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

[cassandra] 12/13: Bulk change of units around the code to support the move to the new configuration framework 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 c51a7c66fc21ca2da08b89ae5f9b4817ee4d8c23
Author: Ekaterina Dimitrova <ek...@datastax.com>
AuthorDate: Sat Feb 5 17:15:18 2022 -0500

    Bulk change of units around the code to support the move to the new configuration framework
    patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe, David Capwell, Michael Semb Wever and Benjamin Lerer for CASSANDRA-15234
---
 conf/cassandra.yaml                                | 206 ++++++++++-----------
 .../apache/cassandra/cache/AutoSavingCache.java    |   2 +-
 .../org/apache/cassandra/cache/CaffeineCache.java  |   2 +-
 .../org/apache/cassandra/cache/ChunkCache.java     |   5 +-
 .../apache/cassandra/cache/SerializingCache.java   |   2 +-
 src/java/org/apache/cassandra/config/Config.java   |  16 +-
 .../org/apache/cassandra/config/Converters.java    |   2 +
 .../org/apache/cassandra/config/DataRateSpec.java  |   2 -
 .../cassandra/config/DatabaseDescriptor.java       | 134 +++++---------
 .../cassandra/config/YamlConfigurationLoader.java  |   3 +-
 .../org/apache/cassandra/cql3/QueryProcessor.java  |  15 +-
 .../org/apache/cassandra/db/RowIndexEntry.java     |   4 +-
 .../cassandra/db/compaction/CompactionTask.java    |   4 +-
 .../db/compaction/LeveledCompactionStrategy.java   |   8 +-
 .../cassandra/db/compaction/SSTableSplitter.java   |   6 +-
 .../SplittingSizeTieredCompactionWriter.java       |   4 +-
 .../db/partitions/AtomicBTreePartition.java        |   2 +-
 .../rows/UnfilteredRowIteratorWithLowerBound.java  |   2 +-
 .../org/apache/cassandra/hadoop/ConfigHelper.java  |  10 +-
 .../cassandra/hadoop/cql3/CqlBulkRecordWriter.java |   2 +-
 .../cassandra/hadoop/cql3/CqlInputFormat.java      |   4 +-
 .../org/apache/cassandra/hints/HintsReader.java    |   2 +-
 .../org/apache/cassandra/hints/HintsWriter.java    |   4 +-
 .../cassandra/index/sasi/conf/IndexMode.java       |   2 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java     |  15 +-
 .../cassandra/io/sstable/IndexSummaryBuilder.java  |   2 +-
 .../cassandra/io/sstable/IndexSummaryManager.java  |   4 +-
 .../io/sstable/IndexSummaryManagerMBean.java       |   2 +-
 .../io/sstable/IndexSummaryRedistribution.java     |   6 +-
 .../org/apache/cassandra/io/sstable/SSTable.java   |   2 +-
 .../cassandra/io/sstable/SSTableRewriter.java      |   4 +-
 .../io/sstable/format/big/BigTableWriter.java      |   2 +-
 .../cassandra/io/util/SequentialWriterOption.java  |   6 +-
 .../cassandra/net/AsyncStreamingOutputPlus.java    |   4 +-
 .../apache/cassandra/schema/CompressionParams.java |   2 +-
 .../org/apache/cassandra/service/CacheService.java |   8 +-
 .../apache/cassandra/service/StorageService.java   |  18 +-
 .../cassandra/service/StorageServiceMBean.java     |   2 +-
 src/java/org/apache/cassandra/tools/NodeProbe.java |   6 +-
 .../apache/cassandra/tools/StandaloneScrubber.java |   4 +-
 .../org/apache/cassandra/tools/nodetool/Info.java  |   4 +-
 .../tools/nodetool/SetBatchlogReplayThrottle.java  |   2 +-
 .../apache/cassandra/utils/memory/BufferPools.java |   4 +-
 .../cassandra/utils/memory/NativeAllocator.java    |   4 +-
 .../cassandra/utils/memory/SlabAllocator.java      |   4 +-
 test/conf/cassandra-seeds.yaml                     |   2 +-
 test/conf/cassandra_encryption.yaml                |  26 +--
 test/conf/unit-test-conf/test-native-port.yaml     |   2 +-
 .../test/HintedHandoffNodetoolTest.java            |   6 +-
 .../distributed/test/MessageFiltersTest.java       |   1 -
 .../distributed/test/ReadRepairQueryTypesTest.java |   1 -
 .../trackwarnings/RowIndexSizeWarningTest.java     |   4 +-
 .../upgrade/MixedModeMessageForwardTest.java       |   2 +-
 .../LongLeveledCompactionStrategyTest.java         |   8 +-
 .../io/sstable/CQLSSTableWriterLongTest.java       |   2 +-
 .../cassandra/config/DatabaseDescriptorTest.java   |  10 +-
 .../LoadOldYAMLBackwardCompatibilityTest.java      |   2 +-
 ...lityTest.java => ParseAndConvertUnitsTest.java} | 105 +++++------
 .../validation/entities/SecondaryIndexTest.java    |  10 +-
 .../unit/org/apache/cassandra/db/KeyCacheTest.java |   8 +-
 .../unit/org/apache/cassandra/db/RowCacheTest.java |  10 +-
 .../commitlog/CommitLogInitWithExceptionTest.java  |   1 -
 .../cassandra/db/commitlog/CommitLogTest.java      |   4 +-
 .../db/compaction/CompactionsCQLTest.java          |   4 +-
 .../compaction/LeveledCompactionStrategyTest.java  |  10 +-
 .../db/streaming/CassandraStreamManagerTest.java   |   4 +-
 .../CompressedSequentialWriterReopenTest.java      |   2 +-
 .../cassandra/io/sstable/CQLSSTableWriterTest.java |  10 +-
 .../cassandra/io/sstable/IndexSummaryTest.java     |   4 +-
 .../cassandra/io/sstable/LargePartitionsTest.java  |  34 ++--
 .../cassandra/io/sstable/SSTableLoaderTest.java    |   4 +-
 .../apache/cassandra/io/util/FileUtilsTest.java    |  20 +-
 .../cassandra/io/util/RandomAccessReaderTest.java  |   4 +-
 .../tools/StandaloneSplitterWithCQLTesterTest.java |   6 +-
 .../tools/nodetool/SetGetColumnIndexSizeTest.java  |   2 +-
 .../org/apache/cassandra/utils/MerkleTreeTest.java |  10 +-
 .../cassandra/utils/memory/BufferPoolTest.java     |   6 +-
 .../io/sstable/StressCQLSSTableWriter.java         |  12 +-
 .../apache/cassandra/stress/CompactionStress.java  |  16 +-
 .../operations/userdefined/SchemaInsert.java       |   2 +-
 80 files changed, 417 insertions(+), 485 deletions(-)

diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index cf4326a..7e39097 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -514,15 +514,15 @@ commitlog_segment_size: 32MiB
 # any class that implements the SeedProvider interface and has a
 # constructor that takes a Map<String, String> of parameters will do.
 seed_provider:
-    # Addresses of hosts that are deemed contact points. 
-    # Cassandra nodes use this list of hosts to find each other and learn
-    # the topology of the ring.  You must change this if you are running
-    # multiple nodes!
-    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
-      parameters:
-          # seeds is actually a comma-delimited list of addresses.
-          # Ex: "<ip1>,<ip2>,<ip3>"
-          - seeds: "127.0.0.1:7000"
+  # Addresses of hosts that are deemed contact points.
+  # Cassandra nodes use this list of hosts to find each other and learn
+  # the topology of the ring.  You must change this if you are running
+  # multiple nodes!
+  - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+    parameters:
+      # seeds is actually a comma-delimited list of addresses.
+      # Ex: "<ip1>,<ip2>,<ip3>"
+      - seeds: "127.0.0.1:7000"
 
 # For workloads with more data than can fit in memory, Cassandra's
 # bottleneck will be reads that need to fetch data from
@@ -660,7 +660,7 @@ memtable_allocation_type: heap_buffers
 # and flush size and frequency. More is not better you just need enough flush writers
 # to never stall waiting for flushing to free memory.
 #
-#memtable_flush_writers: 2
+# memtable_flush_writers: 2
 
 # Total space to use for change-data-capture logs on disk.
 #
@@ -668,7 +668,7 @@ memtable_allocation_type: heap_buffers
 # on Mutations including tables with CDC enabled. A CDCCompactor is responsible
 # for parsing the raw CDC logs and deleting them when parsing is completed.
 #
-# The default value is the min of 4096 mb and 1/8th of the total space
+# The default value is the min of 4096 MiB and 1/8th of the total space
 # of the drive where cdc_raw_directory resides.
 # cdc_total_space: 4096MiB
 
@@ -769,8 +769,8 @@ native_transport_port: 9042
 # native_transport_max_threads: 128
 #
 # The maximum size of allowed frame. Frame (requests) larger than this will
-# be rejected as invalid. The default is 16MB. If you're changing this parameter,
-# you may want to adjust max_value_size_in_mb accordingly. This should be positive and less than 2048.
+# be rejected as invalid. The default is 16MiB. If you're changing this parameter,
+# you may want to adjust max_value_size accordingly. This should be positive and less than 2048.
 # native_transport_max_frame_size: 16MiB
 
 # The maximum number of concurrent client connections.
@@ -793,7 +793,7 @@ native_transport_allow_older_protocols: true
 # values for heartbeat intervals have to be set on the client side.
 #
 # Idle connection timeouts are disabled by default.
-# native_transport_idle_timeout_in_ms: 60000
+# native_transport_idle_timeout: 60000ms
 
 # The address or interface to bind the native transport server to.
 #
@@ -903,7 +903,7 @@ column_index_cache_size: 2KiB
 # 
 # If your data directories are backed by SSD, you should increase this
 # to the number of cores.
-#concurrent_compactors: 1
+# concurrent_compactors: 1
 
 # Number of simultaneous repair validations to allow. If not set or set to
 # a value less than 1, it defaults to the value of concurrent_compactors.
@@ -1212,39 +1212,39 @@ dynamic_snitch_badness_threshold: 1.0
 # Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual
 # auth set require_client_auth=true. Restart all nodes
 server_encryption_options:
-    # On outbound connections, determine which type of peers to securely connect to.
-    #   The available options are :
-    #     none : Do not encrypt outgoing connections
-    #     dc   : Encrypt connections to peers in other datacenters but not within datacenters
-    #     rack : Encrypt connections to peers in other racks but not within racks
-    #     all  : Always use encrypted connections
-    internode_encryption: none
-    # When set to true, encrypted and unencrypted connections are allowed on the storage_port
-    # This should _only be true_ while in unencrypted or transitional operation
-    # optional defaults to true if internode_encryption is none
-    # optional: true
-    # If enabled, will open up an encrypted listening socket on ssl_storage_port. Should only be used
-    # during upgrade to 4.0; otherwise, set to false.
-    legacy_ssl_storage_port_enabled: false
-    # Set to a valid keystore if internode_encryption is dc, rack or all
-    keystore: conf/.keystore
-    keystore_password: cassandra
-    # Verify peer server certificates
-    require_client_auth: false
-    # Set to a valid trustore if require_client_auth is true
-    truststore: conf/.truststore
-    truststore_password: cassandra
-    # Verify that the host name in the certificate matches the connected host
-    require_endpoint_verification: false
-    # More advanced defaults:
-    # protocol: TLS
-    # store_type: JKS
-    # cipher_suites: [
-    #   TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
-    #   TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,
-    #   TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA,
-    #   TLS_RSA_WITH_AES_256_CBC_SHA
-    # ]
+  # On outbound connections, determine which type of peers to securely connect to.
+  #   The available options are :
+  #     none : Do not encrypt outgoing connections
+  #     dc   : Encrypt connections to peers in other datacenters but not within datacenters
+  #     rack : Encrypt connections to peers in other racks but not within racks
+  #     all  : Always use encrypted connections
+  internode_encryption: none
+  # When set to true, encrypted and unencrypted connections are allowed on the storage_port
+  # This should _only be true_ while in unencrypted or transitional operation
+  # optional defaults to true if internode_encryption is none
+  # optional: true
+  # If enabled, will open up an encrypted listening socket on ssl_storage_port. Should only be used
+  # during upgrade to 4.0; otherwise, set to false.
+  legacy_ssl_storage_port_enabled: false
+  # Set to a valid keystore if internode_encryption is dc, rack or all
+  keystore: conf/.keystore
+  keystore_password: cassandra
+  # Verify peer server certificates
+  require_client_auth: false
+  # Set to a valid trustore if require_client_auth is true
+  truststore: conf/.truststore
+  truststore_password: cassandra
+  # Verify that the host name in the certificate matches the connected host
+  require_endpoint_verification: false
+  # More advanced defaults:
+  # protocol: TLS
+  # store_type: JKS
+  # cipher_suites: [
+  #   TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
+  #   TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,
+  #   TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA,
+  #   TLS_RSA_WITH_AES_256_CBC_SHA
+  # ]
 
 # Configure client-to-server encryption.
 #
@@ -1259,29 +1259,29 @@ server_encryption_options:
 # Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual
 # auth set require_client_auth=true. Restart all nodes
 client_encryption_options:
-    # Enable client-to-server encryption
-    enabled: false
-    # When set to true, encrypted and unencrypted connections are allowed on the native_transport_port
-    # This should _only be true_ while in unencrypted or transitional operation
-    # optional defaults to true when enabled is false, and false when enabled is true.
-    # optional: true
-    # Set keystore and keystore_password to valid keystores if enabled is true
-    keystore: conf/.keystore
-    keystore_password: cassandra
-    # Verify client certificates
-    require_client_auth: false
-    # Set trustore and truststore_password if require_client_auth is true
-    # truststore: conf/.truststore
-    # truststore_password: cassandra
-    # More advanced defaults:
-    # protocol: TLS
-    # store_type: JKS
-    # cipher_suites: [
-    #   TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
-    #   TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,
-    #   TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA,
-    #   TLS_RSA_WITH_AES_256_CBC_SHA
-    # ]
+  # Enable client-to-server encryption
+  enabled: false
+  # When set to true, encrypted and unencrypted connections are allowed on the native_transport_port
+  # This should _only be true_ while in unencrypted or transitional operation
+  # optional defaults to true when enabled is false, and false when enabled is true.
+  # optional: true
+  # Set keystore and keystore_password to valid keystores if enabled is true
+  keystore: conf/.keystore
+  keystore_password: cassandra
+  # Verify client certificates
+  require_client_auth: false
+  # Set trustore and truststore_password if require_client_auth is true
+  # truststore: conf/.truststore
+  # truststore_password: cassandra
+  # More advanced defaults:
+  # protocol: TLS
+  # store_type: JKS
+  # cipher_suites: [
+  #   TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
+  #   TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,
+  #   TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA,
+  #   TLS_RSA_WITH_AES_256_CBC_SHA
+  # ]
 
 # internode_compression controls whether traffic between nodes is
 # compressed.
@@ -1332,19 +1332,19 @@ scripted_user_defined_functions_enabled: false
 # Currently, only the following file types are supported for transparent data encryption, although
 # more are coming in future cassandra releases: commitlog, hints
 transparent_data_encryption_options:
-    enabled: false
-    chunk_length_kb: 64
-    cipher: AES/CBC/PKCS5Padding
-    key_alias: testing:1
-    # CBC IV length for AES needs to be 16 bytes (which is also the default size)
-    # iv_length: 16
-    key_provider:
-      - class_name: org.apache.cassandra.security.JKSKeyProvider
-        parameters:
-          - keystore: conf/.keystore
-            keystore_password: cassandra
-            store_type: JCEKS
-            key_password: cassandra
+  enabled: false
+  chunk_length_kb: 64
+  cipher: AES/CBC/PKCS5Padding
+  key_alias: testing:1
+  # CBC IV length for AES needs to be 16 bytes (which is also the default size)
+  # iv_length: 16
+  key_provider:
+    - class_name: org.apache.cassandra.security.JKSKeyProvider
+      parameters:
+        - keystore: conf/.keystore
+          keystore_password: cassandra
+          store_type: JCEKS
+          key_password: cassandra
 
 
 #####################
@@ -1383,11 +1383,11 @@ replica_filtering_protection:
     cached_rows_warn_threshold: 2000
     cached_rows_fail_threshold: 32000
 
-# Log WARN on any multiple-partition batch size exceeding this value. 5kb per batch by default.
+# Log WARN on any multiple-partition batch size exceeding this value. 5KiB per batch by default.
 # Caution should be taken on increasing the size of this threshold as it can lead to node instability.
 batch_size_warn_threshold: 5KiB
 
-# Fail any multiple-partition batch exceeding this value. 50kb (10x warn threshold) by default.
+# Fail any multiple-partition batch exceeding this value. 50KiB (10x warn threshold) by default.
 batch_size_fail_threshold: 50KiB
 
 # Log WARN on any batches not of type LOGGED than span across more partitions than this limit
@@ -1446,23 +1446,23 @@ compaction_tombstone_warning_threshold: 100000
 # Audit logging - Logs every incoming CQL command request, authentication to a node. See the docs
 # on audit_logging for full details about the various configuration options.
 audit_logging_options:
-    enabled: false
-    logger:
-      - class_name: BinAuditLogger
-    # audit_logs_dir:
-    # included_keyspaces:
-    # excluded_keyspaces: system, system_schema, system_virtual_schema
-    # included_categories:
-    # excluded_categories:
-    # included_users:
-    # excluded_users:
-    # roll_cycle: HOURLY
-    # block: true
-    # max_queue_weight: 268435456 # 256 MiB
-    # max_log_size: 17179869184 # 16 GiB
-    ## archive command is "/path/to/script.sh %path" where %path is replaced with the file being rolled:
-    # archive_command:
-    # max_archive_retries: 10
+  enabled: false
+  logger:
+    - class_name: BinAuditLogger
+  # audit_logs_dir:
+  # included_keyspaces:
+  # excluded_keyspaces: system, system_schema, system_virtual_schema
+  # included_categories:
+  # excluded_categories:
+  # included_users:
+  # excluded_users:
+  # roll_cycle: HOURLY
+  # block: true
+  # max_queue_weight: 268435456 # 256 MiB
+  # max_log_size: 17179869184 # 16 GiB
+  ## archive command is "/path/to/script.sh %path" where %path is replaced with the file being rolled:
+  # archive_command:
+  # max_archive_retries: 10
 
 
 # default options for full query logging - these can be overridden from command line when executing
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index d0b897e..66736d1 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -93,7 +93,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
     {
         private final SequentialWriterOption writerOption = SequentialWriterOption.newBuilder()
                                                                     .trickleFsync(DatabaseDescriptor.getTrickleFsync())
-                                                                    .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024)
+                                                                    .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKiB() * 1024)
                                                                     .finishOnClose(true).build();
 
         public InputStream getInputStream(File dataPath, File crcPath) throws IOException
diff --git a/src/java/org/apache/cassandra/cache/CaffeineCache.java b/src/java/org/apache/cassandra/cache/CaffeineCache.java
index b01093f..124e681 100644
--- a/src/java/org/apache/cassandra/cache/CaffeineCache.java
+++ b/src/java/org/apache/cassandra/cache/CaffeineCache.java
@@ -65,7 +65,7 @@ public class CaffeineCache<K extends IMeasurableMemory, V extends IMeasurableMem
         return create(weightedCapacity, (key, value) -> {
             long size = key.unsharedHeapSize() + value.unsharedHeapSize();
             if (size > Integer.MAX_VALUE) {
-                throw new IllegalArgumentException("Serialized size cannot be more than 2GB/Integer.MAX_VALUE");
+                throw new IllegalArgumentException("Serialized size cannot be more than 2GiB/Integer.MAX_VALUE");
             }
             return (int) size;
         });
diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java
index 397850a..51dbdc6 100644
--- a/src/java/org/apache/cassandra/cache/ChunkCache.java
+++ b/src/java/org/apache/cassandra/cache/ChunkCache.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.MoreExecutors;
 
 import com.github.benmanes.caffeine.cache.*;
 import org.apache.cassandra.concurrent.ImmediateExecutor;
@@ -40,8 +39,8 @@ import org.apache.cassandra.utils.memory.BufferPools;
 public class ChunkCache
         implements CacheLoader<ChunkCache.Key, ChunkCache.Buffer>, RemovalListener<ChunkCache.Key, ChunkCache.Buffer>, CacheSize
 {
-    public static final int RESERVED_POOL_SPACE_IN_MB = 32;
-    public static final long cacheSize = 1024L * 1024L * Math.max(0, DatabaseDescriptor.getFileCacheSizeInMB() - RESERVED_POOL_SPACE_IN_MB);
+    public static final int RESERVED_POOL_SPACE_IN_MiB = 32;
+    public static final long cacheSize = 1024L * 1024L * Math.max(0, DatabaseDescriptor.getFileCacheSizeInMiB() - RESERVED_POOL_SPACE_IN_MiB);
     public static final boolean roundUp = DatabaseDescriptor.getFileCacheRoundUp();
 
     private static boolean enabled = DatabaseDescriptor.getFileCacheEnabled() && cacheSize > 0;
diff --git a/src/java/org/apache/cassandra/cache/SerializingCache.java b/src/java/org/apache/cassandra/cache/SerializingCache.java
index cd028e2..be7c4f5 100644
--- a/src/java/org/apache/cassandra/cache/SerializingCache.java
+++ b/src/java/org/apache/cassandra/cache/SerializingCache.java
@@ -71,7 +71,7 @@ public class SerializingCache<K, V> implements ICache<K, V>
         return create(weightedCapacity, (key, value) -> {
             long size = value.size();
             if (size > Integer.MAX_VALUE) {
-                throw new IllegalArgumentException("Serialized size must not be more than 2GB");
+                throw new IllegalArgumentException("Serialized size must not be more than 2GiB");
             }
             return (int) size;
         }, serializer);
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index a6b0413..369fe04 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
 import com.google.common.base.Joiner;
@@ -196,6 +195,7 @@ public class Config
     @Replaces(oldName = "internode_recv_buff_size_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
     public DataStorageSpec internode_socket_receive_buffer_size = new DataStorageSpec("0B");
 
+    // TODO: derive defaults from system memory settings?
     @Replaces(oldName = "internode_application_send_queue_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
     public DataStorageSpec internode_application_send_queue_capacity = new DataStorageSpec("4MiB");
     @Replaces(oldName = "internode_application_send_queue_reserve_endpoint_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
@@ -203,7 +203,6 @@ public class Config
     @Replaces(oldName = "internode_application_send_queue_reserve_global_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
     public DataStorageSpec internode_application_send_queue_reserve_global_capacity = new DataStorageSpec("512MiB");
 
-    // TODO: derive defaults from system memory settings?
     @Replaces(oldName = "internode_application_receive_queue_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
     public DataStorageSpec internode_application_receive_queue_capacity = new DataStorageSpec("4MiB");
     @Replaces(oldName = "internode_application_receive_queue_reserve_endpoint_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true)
@@ -281,12 +280,6 @@ public class Config
     public volatile int concurrent_materialized_view_builders = 1;
     public volatile int reject_repair_compaction_threshold = Integer.MAX_VALUE;
 
-    /**
-     * @deprecated retry support removed on CASSANDRA-10992
-     */
-    @Deprecated
-    public int max_streaming_retries = 3;
-
     @Replaces(oldName = "stream_throughput_outbound_megabits_per_sec", converter = Converters.MEGABITS_TO_MEBIBYTES_PER_SECOND_DATA_RATE, deprecated = true)
     public volatile DataRateSpec stream_throughput_outbound = new DataRateSpec("24MiB/s");
     @Replaces(oldName = "inter_dc_stream_throughput_outbound_megabits_per_sec", converter = Converters.MEGABITS_TO_MEBIBYTES_PER_SECOND_DATA_RATE, deprecated = true)
@@ -365,7 +358,6 @@ public class Config
     @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;
-
     @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)
@@ -513,16 +505,16 @@ public class Config
      * (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;
+    public long user_defined_function_warn_timeout_in_ms = 500;
     /**
      * Time in milliseconds after a fatal UDF run-time situation is detected and action according to
      * user_function_timeout_policy will take place.
      * (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_fail_timeout = 1500;
+    public long user_defined_function_fail_timeout_in_ms = 1500;
     /**
-     * Defines what to do when a UDF ran longer than user_defined_function_fail_timeout.
+     * Defines what to do when a UDF ran longer than user_defined_function_fail_timeout_in_ms.
      * Possible options are:
      * - 'die' - i.e. it is able to emit a warning to the client before the Cassandra Daemon will shut down.
      * - 'die_immediate' - shut down C* daemon immediately (effectively prevent the chance that the client will receive a warning).
diff --git a/src/java/org/apache/cassandra/config/Converters.java b/src/java/org/apache/cassandra/config/Converters.java
index f2f7120..447e731 100644
--- a/src/java/org/apache/cassandra/config/Converters.java
+++ b/src/java/org/apache/cassandra/config/Converters.java
@@ -112,6 +112,7 @@ public enum Converters
      *
      * @param value we will use from cassandra.yaml to create a new {@link Config} parameter of type {@link DurationSpec},
      * {@link DataRateSpec} or {@link DataStorageSpec}
+     *
      * @return new object of type {@link DurationSpec}, {@link DataRateSpec} or {@link DataStorageSpec}
      */
     public Object convert(Object value)
@@ -126,6 +127,7 @@ public enum Converters
      * compatibility
      *
      * @param value we will use to calculate the output value
+     *
      * @return the numeric value
      */
     public Object deconvert(Object value)
diff --git a/src/java/org/apache/cassandra/config/DataRateSpec.java b/src/java/org/apache/cassandra/config/DataRateSpec.java
index bbdbecc..3512513 100644
--- a/src/java/org/apache/cassandra/config/DataRateSpec.java
+++ b/src/java/org/apache/cassandra/config/DataRateSpec.java
@@ -300,8 +300,6 @@ public final class DataRateSpec
 
             public double toMegabitsPerSecond(double d)
             {
-
-
                 if (d > MAX / (MEGABITS_PER_MEBIBYTE))
                     return MAX;
                 return Math.round(d * MEGABITS_PER_MEBIBYTE);
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 25e5c8b..ef9dd43 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -32,11 +32,6 @@ import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.RateLimiter;
 
-import org.apache.cassandra.gms.IFailureDetector;
-import org.apache.cassandra.io.util.File;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.audit.AuditLogOptions;
 import org.apache.cassandra.fql.FullQueryLoggerOptions;
 import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
@@ -54,6 +49,8 @@ import org.apache.cassandra.db.commitlog.CommitLogSegmentManagerCDC;
 import org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.IFailureDetector;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.DiskOptimizationStrategy;
 import org.apache.cassandra.io.util.FileUtils;
@@ -75,6 +72,9 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 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;
@@ -94,7 +94,7 @@ public class DatabaseDescriptor
     private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class);
 
     /**
-     * Tokens are serialized in a Gossip VersionedValue String.  VV are restricted to 64KB
+     * Tokens are serialized in a Gossip VersionedValue String.  VV are restricted to 64KiB
      * when we send them over the wire, which works out to about 1700 tokens.
      */
     private static final int MAX_NUM_TOKENS = 1536;
@@ -379,7 +379,7 @@ public class DatabaseDescriptor
             throw new ConfigurationException("Missing required directive CommitLogSync", false);
         }
 
-        if (conf.commitlog_sync == Config.CommitLogSync.batch)
+        if (conf.commitlog_sync == CommitLogSync.batch)
         {
             if (conf.commitlog_sync_period.toMilliseconds() != 0)
             {
@@ -401,7 +401,7 @@ public class DatabaseDescriptor
         }
         else
         {
-            if (conf.commitlog_sync_period.toMilliseconds() <= 0)
+            if (conf.commitlog_sync_period.toMilliseconds() == 0)
             {
                 throw new ConfigurationException("Missing value for commitlog_sync_period.", false);
             }
@@ -495,7 +495,7 @@ public class DatabaseDescriptor
 
         if (conf.repair_session_space.toMebibytes() < 1)
             throw new ConfigurationException("repair_session_space must be > 0, but was " + conf.repair_session_space);
-        else if (conf.repair_session_space.toMebibytesAsInt() > (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)))
+        else if (conf.repair_session_space.toMebibytes() > (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)))
             logger.warn("A repair_session_space of " + conf.repair_session_space+ " mebibytes is likely to cause heap pressure");
 
         checkForLowestAcceptedTimeouts(conf);
@@ -682,13 +682,13 @@ public class DatabaseDescriptor
                                               ? Math.max(10, (int) (Runtime.getRuntime().maxMemory() / 1024 / 1024 / 256))
                                               : conf.prepared_statements_cache_size.toMebibytes();
 
-            if (preparedStatementsCacheSizeInMiB <= 0)
+            if (preparedStatementsCacheSizeInMiB == 0)
                 throw new NumberFormatException(); // to escape duplicating error message
         }
         catch (NumberFormatException e)
         {
             throw new ConfigurationException("prepared_statements_cache_size option was set incorrectly to '"
-                                             + conf.prepared_statements_cache_size + "', supported values are <integer> >= 0.", false);
+                                             + (conf.prepared_statements_cache_size != null ? conf.prepared_statements_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
         }
 
         try
@@ -720,7 +720,7 @@ public class DatabaseDescriptor
         catch (NumberFormatException e)
         {
             throw new ConfigurationException("counter_cache_size option was set incorrectly to '"
-                                             + conf.counter_cache_size + "', supported values are <integer> >= 0.", false);
+                                             + (conf.counter_cache_size !=null ?conf.counter_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
         }
 
         // if set to empty/"auto" then use 5% of Heap size
@@ -732,13 +732,13 @@ public class DatabaseDescriptor
             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);
-        if (conf.user_defined_function_warn_timeout < 0)
-            throw new ConfigurationException("user_defined_function_warn_timeout must not be negative", false);
+        if (conf.user_defined_function_fail_timeout_in_ms < 0)
+            throw new ConfigurationException("user_defined_function_fail_timeout_in_ms must not be negative", false);
+        if (conf.user_defined_function_warn_timeout_in_ms < 0)
+            throw new ConfigurationException("user_defined_function_warn_timeout_in_ms must not be negative", false);
 
-        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.user_defined_function_fail_timeout_in_ms < conf.user_defined_function_warn_timeout_in_ms)
+            throw new ConfigurationException("user_defined_function_warn_timeout_in_ms must less than user_defined_function_fail_timeout_in_ms", false);
 
         if (conf.commitlog_segment_size.toMebibytes() == 0)
             throw new ConfigurationException("commitlog_segment_size must be positive, but was "
@@ -768,9 +768,9 @@ public class DatabaseDescriptor
         if (conf.snapshot_links_per_second < 0)
             throw new ConfigurationException("snapshot_links_per_second must be >= 0");
 
-        if (conf.max_value_size.toMebibytesAsInt() == 0)
+        if (conf.max_value_size.toMebibytes() == 0)
             throw new ConfigurationException("max_value_size must be positive", false);
-        else if (conf.max_value_size.toMebibytesAsInt() >= 2048)
+        else if (conf.max_value_size.toMebibytes() >= 2048)
             throw new ConfigurationException("max_value_size must be smaller than 2048, but was "
                     + conf.max_value_size.toString(), false);
 
@@ -1552,7 +1552,7 @@ public class DatabaseDescriptor
         return conf.batch_size_fail_threshold.toBytesAsInt();
     }
 
-    public static int getBatchSizeFailThresholdInKB()
+    public static int getBatchSizeFailThresholdInKiB()
     {
         return conf.batch_size_fail_threshold.toKibibytesAsInt();
     }
@@ -1569,7 +1569,7 @@ public class DatabaseDescriptor
         conf.batch_size_warn_threshold = SmallestDataStorageKibibytes.inKibibytes(threshold);
     }
 
-    public static void setBatchSizeFailThresholdInKB(int threshold)
+    public static void setBatchSizeFailThresholdInKiB(int threshold)
     {
         conf.batch_size_fail_threshold = SmallestDataStorageKibibytes.inKibibytes(threshold);
     }
@@ -1662,7 +1662,7 @@ public class DatabaseDescriptor
 
     public static long getRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.request_timeout.to(unit);
     }
 
     public static void setRpcTimeout(long timeOutInMillis)
@@ -1672,7 +1672,7 @@ public class DatabaseDescriptor
 
     public static long getReadRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.read_request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.read_request_timeout.to(unit);
     }
 
     public static void setReadRpcTimeout(long timeOutInMillis)
@@ -1682,7 +1682,7 @@ public class DatabaseDescriptor
 
     public static long getRangeRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.range_request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.range_request_timeout.to(unit);
     }
 
     public static void setRangeRpcTimeout(long timeOutInMillis)
@@ -1692,7 +1692,7 @@ public class DatabaseDescriptor
 
     public static long getWriteRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.write_request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.write_request_timeout.to(unit);
     }
 
     public static void setWriteRpcTimeout(long timeOutInMillis)
@@ -1702,7 +1702,7 @@ public class DatabaseDescriptor
 
     public static long getCounterWriteRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.counter_write_request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.counter_write_request_timeout.to(unit);
     }
 
     public static void setCounterWriteRpcTimeout(long timeOutInMillis)
@@ -1712,7 +1712,7 @@ public class DatabaseDescriptor
 
     public static long getCasContentionTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.cas_contention_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.cas_contention_timeout.to(unit);
     }
 
     public static void setCasContentionTimeout(long timeOutInMillis)
@@ -1722,7 +1722,7 @@ public class DatabaseDescriptor
 
     public static long getTruncateRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.truncate_request_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.truncate_request_timeout.to(unit);
     }
 
     public static void setTruncateRpcTimeout(long timeOutInMillis)
@@ -1740,9 +1740,9 @@ public class DatabaseDescriptor
         conf.internode_timeout = crossNodeTimeout;
     }
 
-    public static long getSlowQueryTimeout(TimeUnit units)
+    public static long getSlowQueryTimeout(TimeUnit unit)
     {
-        return units.convert(conf.slow_query_log_timeout.toMilliseconds(), MILLISECONDS);
+        return conf.slow_query_log_timeout.to(unit);
     }
 
     /**
@@ -2770,7 +2770,7 @@ public class DatabaseDescriptor
         return conf.file_cache_enabled;
     }
 
-    public static int getFileCacheSizeInMB()
+    public static int getFileCacheSizeInMiB()
     {
         if (conf.file_cache_size == null)
         {
@@ -2782,7 +2782,7 @@ public class DatabaseDescriptor
         return conf.file_cache_size.toMebibytesAsInt();
     }
 
-    public static int getNetworkingCacheSizeInMB()
+    public static int getNetworkingCacheSizeInMiB()
     {
         if (conf.networking_cache_size == null)
         {
@@ -2830,13 +2830,14 @@ public class DatabaseDescriptor
         conf.key_cache_migrate_during_compaction = migrateCacheEntry;
     }
 
-    public static int getSSTablePreemptiveOpenIntervalInMB()
+    public static int getSSTablePreemptiveOpenIntervalInMiB()
     {
         return conf.sstable_preemptive_open_interval.toMebibytesAsInt();
     }
-    public static void setSSTablePreemptiveOpenIntervalInMB(int mb)
+
+    public static void setSSTablePreemptiveOpenIntervalInMiB(int mib)
     {
-        conf.sstable_preemptive_open_interval = SmallestDataStorageMebibytes.inMebibytes(mb);
+        conf.sstable_preemptive_open_interval = SmallestDataStorageMebibytes.inMebibytes(mib);
     }
 
     public static boolean getTrickleFsync()
@@ -2844,17 +2845,17 @@ public class DatabaseDescriptor
         return conf.trickle_fsync;
     }
 
-    public static int getTrickleFsyncIntervalInKb()
+    public static int getTrickleFsyncIntervalInKiB()
     {
         return conf.trickle_fsync_interval.toKibibytesAsInt();
     }
 
-    public static long getKeyCacheSizeInMB()
+    public static long getKeyCacheSizeInMiB()
     {
         return keyCacheSizeInMiB;
     }
 
-    public static long getIndexSummaryCapacityInMB()
+    public static long getIndexSummaryCapacityInMiB()
     {
         return indexSummaryCapacityInMiB;
     }
@@ -3075,7 +3076,7 @@ public class DatabaseDescriptor
         return conf.trace_type_query_ttl.toSecondsAsInt();
     }
 
-    public static long getPreparedStatementsCacheSizeMB()
+    public static long getPreparedStatementsCacheSizeMiB()
     {
         return preparedStatementsCacheSizeInMiB;
     }
@@ -3102,12 +3103,12 @@ public class DatabaseDescriptor
 
     public static long getUserDefinedFunctionWarnTimeout()
     {
-        return conf.user_defined_function_warn_timeout;
+        return conf.user_defined_function_warn_timeout_in_ms;
     }
 
     public static void setUserDefinedFunctionWarnTimeout(long userDefinedFunctionWarnTimeout)
     {
-        conf.user_defined_function_warn_timeout = userDefinedFunctionWarnTimeout;
+        conf.user_defined_function_warn_timeout_in_ms = userDefinedFunctionWarnTimeout;
     }
 
     public static boolean getMaterializedViewsEnabled()
@@ -3153,12 +3154,12 @@ public class DatabaseDescriptor
 
     public static long getUserDefinedFunctionFailTimeout()
     {
-        return conf.user_defined_function_fail_timeout;
+        return conf.user_defined_function_fail_timeout_in_ms;
     }
 
     public static void setUserDefinedFunctionFailTimeout(long userDefinedFunctionFailTimeout)
     {
-        conf.user_defined_function_fail_timeout = userDefinedFunctionFailTimeout;
+        conf.user_defined_function_fail_timeout_in_ms = userDefinedFunctionFailTimeout;
     }
 
     public static Config.UserFunctionTimeoutPolicy getUserFunctionTimeoutPolicy()
@@ -3419,43 +3420,6 @@ public class DatabaseDescriptor
     }
 
     /**
-     * Class that primarily tracks overflow thresholds during conversions
-     */
-    private enum ByteUnit {
-        KIBI_BYTES(2048 * 1024, 1024),
-        MEBI_BYTES(2048, 1024 * 1024);
-
-        private final int overflowThreshold;
-        private final int multiplier;
-
-        ByteUnit(int t, int m)
-        {
-            this.overflowThreshold = t;
-            this.multiplier = m;
-        }
-
-        public int overflowThreshold()
-        {
-            return overflowThreshold;
-        }
-
-        public boolean willOverflowInBytes(int val)
-        {
-            return val >= overflowThreshold;
-        }
-
-        public long toBytes(int val)
-        {
-            return val * multiplier;
-        }
-
-        public long fromBytes(int val)
-        {
-            return val / multiplier;
-        }
-    }
-
-    /**
      * Ensures passed in configuration value is positive and will not overflow when converted to Bytes
      */
     private static void checkValidForByteConversion(final SmallestDataStorageKibibytes value, String name)
@@ -3789,22 +3753,22 @@ public class DatabaseDescriptor
         conf.track_warnings.local_read_size.setAbortThresholdKb(value);
     }
 
-    public static int getRowIndexSizeWarnThresholdKb()
+    public static int getRowIndexSizeWarnThresholdKiB()
     {
         return conf.track_warnings.row_index_size.getWarnThresholdKb();
     }
 
-    public static void setRowIndexSizeWarnThresholdKb(int value)
+    public static void setRowIndexSizeWarnThresholdKiB(int value)
     {
         conf.track_warnings.row_index_size.setWarnThresholdKb(value);
     }
 
-    public static int getRowIndexSizeAbortThresholdKb()
+    public static int getRowIndexSizeAbortThresholdKiB()
     {
         return conf.track_warnings.row_index_size.getAbortThresholdKb();
     }
 
-    public static void setRowIndexSizeAbortThresholdKb(int value)
+    public static void setRowIndexSizeAbortThresholdKiB(int value)
     {
         conf.track_warnings.row_index_size.setAbortThresholdKb(value);
     }
diff --git a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
index 4fc82b4..1840054 100644
--- a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
+++ b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
@@ -144,8 +144,7 @@ public class YamlConfigurationLoader implements ConfigurationLoader
         }
         catch (YAMLException e)
         {
-            throw new ConfigurationException("Invalid yaml: " + url + SystemUtils.LINE_SEPARATOR
-                                             + " Error: " + e.getMessage(), false);
+            throw new ConfigurationException("Invalid yaml: " + url, e);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 80ba508..0e0c043 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -30,7 +30,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.*;
 import com.google.common.primitives.Ints;
-import com.google.common.util.concurrent.MoreExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,7 +98,7 @@ public class QueryProcessor implements QueryHandler
     {
         preparedStatements = Caffeine.newBuilder()
                              .executor(ImmediateExecutor.INSTANCE)
-                             .maximumWeight(capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMB()))
+                             .maximumWeight(capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMiB()))
                              .weigher(QueryProcessor::measure)
                              .removalListener((key, prepared, cause) -> {
                                  MD5Digest md5Digest = (MD5Digest) key;
@@ -114,13 +113,13 @@ public class QueryProcessor implements QueryHandler
         ScheduledExecutors.scheduledTasks.scheduleAtFixedRate(() -> {
             long count = lastMinuteEvictionsCount.getAndSet(0);
             if (count > 0)
-                logger.warn("{} prepared statements discarded in the last minute because cache limit reached ({} MB)",
+                logger.warn("{} prepared statements discarded in the last minute because cache limit reached ({} MiB)",
                             count,
-                            DatabaseDescriptor.getPreparedStatementsCacheSizeMB());
+                            DatabaseDescriptor.getPreparedStatementsCacheSizeMiB());
         }, 1, 1, TimeUnit.MINUTES);
 
-        logger.info("Initialized prepared statement caches with {} MB",
-                    DatabaseDescriptor.getPreparedStatementsCacheSizeMB());
+        logger.info("Initialized prepared statement caches with {} MiB",
+                    DatabaseDescriptor.getPreparedStatementsCacheSizeMiB());
     }
 
     private static long capacityToBytes(long cacheSizeMB)
@@ -564,10 +563,10 @@ public class QueryProcessor implements QueryHandler
         // (if the keyspace is null, queryString has to have a fully-qualified keyspace so it's fine.
         long statementSize = ObjectSizes.measureDeep(prepared.statement);
         // don't execute the statement if it's bigger than the allowed threshold
-        if (statementSize > capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMB()))
+        if (statementSize > capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMiB()))
             throw new InvalidRequestException(String.format("Prepared statement of size %d bytes is larger than allowed maximum of %d MB: %s...",
                                                             statementSize,
-                                                            DatabaseDescriptor.getPreparedStatementsCacheSizeMB(),
+                                                            DatabaseDescriptor.getPreparedStatementsCacheSizeMiB(),
                                                             queryString.substring(0, 200)));
         MD5Digest statementId = computeId(queryString, keyspace);
         preparedStatements.put(statementId, prepared);
diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java
index 153dbac..f5f632a 100644
--- a/src/java/org/apache/cassandra/db/RowIndexEntry.java
+++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java
@@ -355,8 +355,8 @@ public class RowIndexEntry<T> implements IMeasurableMemory
             if (command == null || SchemaConstants.isSystemKeyspace(command.metadata().keyspace) || !DatabaseDescriptor.getTrackWarningsEnabled())
                 return;
 
-            int warnThreshold = DatabaseDescriptor.getRowIndexSizeWarnThresholdKb() * 1024;
-            int abortThreshold = DatabaseDescriptor.getRowIndexSizeAbortThresholdKb() * 1024;
+            int warnThreshold = DatabaseDescriptor.getRowIndexSizeWarnThresholdKiB() * 1024;
+            int abortThreshold = DatabaseDescriptor.getRowIndexSizeAbortThresholdKiB() * 1024;
             if (warnThreshold == 0 && abortThreshold == 0)
                 return;
 
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 17c30d4..b0ec8da 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -87,7 +87,7 @@ public class CompactionTask extends AbstractCompactionTask
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+            logger.warn("insufficient space to compact all requested files. {}MiB required, {}",
                         (float) expectedSize / 1024 / 1024,
                         StringUtils.join(transaction.originals(), ", "));
 
@@ -394,7 +394,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
 
             sstablesRemoved++;
-            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+            logger.warn("Not enough space for compaction, {}MiB estimated.  Reducing scope.",
                         (float) expectedWriteSize / 1024 / 1024);
         }
 
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 6faca63..54953e4 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -54,7 +54,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
 
     @VisibleForTesting
     final LeveledManifest manifest;
-    private final int maxSSTableSizeInMB;
+    private final int maxSSTableSizeInMiB;
     private final int levelFanoutSize;
     private final boolean singleSSTableUplevel;
 
@@ -91,11 +91,11 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
                 configuredSingleSSTableUplevel = Boolean.parseBoolean(options.get(SINGLE_SSTABLE_UPLEVEL_OPTION));
             }
         }
-        maxSSTableSizeInMB = configuredMaxSSTableSize;
+        maxSSTableSizeInMiB = configuredMaxSSTableSize;
         levelFanoutSize = configuredLevelFanoutSize;
         singleSSTableUplevel = configuredSingleSSTableUplevel;
 
-        manifest = new LeveledManifest(cfs, this.maxSSTableSizeInMB, this.levelFanoutSize, localOptions);
+        manifest = new LeveledManifest(cfs, this.maxSSTableSizeInMiB, this.levelFanoutSize, localOptions);
         logger.trace("Created {}", manifest);
     }
 
@@ -282,7 +282,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
 
     public long getMaxSSTableBytes()
     {
-        return maxSSTableSizeInMB * 1024L * 1024L;
+        return maxSSTableSizeInMiB * 1024L * 1024L;
     }
 
     public int getLevelFanoutSize()
diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
index 1bb1701..6f68c34 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -42,12 +42,12 @@ public class SSTableSplitter
 
     public static class SplittingCompactionTask extends CompactionTask
     {
-        private final int sstableSizeInMB;
+        private final int sstableSizeInMiB;
 
         public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
         {
             super(cfs, transaction, CompactionManager.NO_GC, false);
-            this.sstableSizeInMB = sstableSizeInMB;
+            this.sstableSizeInMiB = sstableSizeInMB;
 
             if (sstableSizeInMB <= 0)
                 throw new IllegalArgumentException("Invalid target size for SSTables, must be > 0 (got: " + sstableSizeInMB + ")");
@@ -65,7 +65,7 @@ public class SSTableSplitter
                                                               LifecycleTransaction txn,
                                                               Set<SSTableReader> nonExpiredSSTables)
         {
-            return new MaxSSTableSizeWriter(cfs, directories, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, false);
+            return new MaxSSTableSizeWriter(cfs, directories, txn, nonExpiredSSTables, sstableSizeInMiB * 1024L * 1024L, 0, false);
         }
 
         @Override
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index d29061c..565e614 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
  * CompactionAwareWriter that splits input in differently sized sstables
  *
  * Biggest sstable will be total_compaction_size / 2, second biggest total_compaction_size / 4 etc until
- * the result would be sub 50MB, all those are put in the same
+ * the result would be sub 50MiB, all those are put in the same
  */
 public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
 {
@@ -70,7 +70,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
         }
 
         int noPointIndex = 0;
-        // find how many sstables we should create - 50MB min sstable size
+        // find how many sstables we should create - 50MiB min sstable size
         for (double ratio : potentialRatios)
         {
             noPointIndex++;
diff --git a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
index c5e9fe4..92e2f8e 100644
--- a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
@@ -73,7 +73,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition
 
     /**
      * (clock + allocation) granularity are combined to give us an acceptable (waste) allocation rate that is defined by
-     * the passage of real time of ALLOCATION_GRANULARITY_BYTES/CLOCK_GRANULARITY, or in this case 7.63Kb/ms, or 7.45Mb/s
+     * the passage of real time of ALLOCATION_GRANULARITY_BYTES/CLOCK_GRANULARITY, or in this case 7.63KiB/ms, or 7.45Mb/s
      *
      * in wasteTracker we maintain within EXCESS_WASTE_OFFSET before the current time; whenever we waste bytes
      * we increment the current value if it is within this window, and set it to the min of the window plus our waste
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
index b6f4254..2842e66 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
@@ -185,7 +185,7 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
         // CASSANDRA-11369 is there to fix this afterwards.
 
         // Creating the iterator ensures that rowIndexEntry is loaded if available (partitions bigger than
-        // DatabaseDescriptor.column_index_size_in_kb)
+        // DatabaseDescriptor.column_index_size)
         if (!canUseMetadataLowerBound())
             maybeInit();
 
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index f01197d..cc539b1 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -45,7 +45,7 @@ public class ConfigHelper
     private static final String INPUT_PREDICATE_CONFIG = "cassandra.input.predicate";
     private static final String INPUT_KEYRANGE_CONFIG = "cassandra.input.keyRange";
     private static final String INPUT_SPLIT_SIZE_CONFIG = "cassandra.input.split.size";
-    private static final String INPUT_SPLIT_SIZE_IN_MB_CONFIG = "cassandra.input.split.size_mb";
+    private static final String INPUT_SPLIT_SIZE_IN_MIB_CONFIG = "cassandra.input.split.size_mb";
     private static final String INPUT_WIDEROWS_CONFIG = "cassandra.input.widerows";
     private static final int DEFAULT_SPLIT_SIZE = 64 * 1024;
     private static final String RANGE_BATCH_SIZE_CONFIG = "cassandra.range.batch.size";
@@ -185,21 +185,21 @@ public class ConfigHelper
      * the overhead of each map will take up the bulk of the job time.
      *
      * @param conf          Job configuration you are about to run
-     * @param splitSizeMb   Input split size in MB
+     * @param splitSizeMb   Input split size in MiB
      */
     public static void setInputSplitSizeInMb(Configuration conf, int splitSizeMb)
     {
-        conf.setInt(INPUT_SPLIT_SIZE_IN_MB_CONFIG, splitSizeMb);
+        conf.setInt(INPUT_SPLIT_SIZE_IN_MIB_CONFIG, splitSizeMb);
     }
 
     /**
      * cassandra.input.split.size will be used if the value is undefined or negative.
      * @param conf  Job configuration you are about to run
-     * @return      split size in MB or -1 if it is undefined.
+     * @return      split size in MiB or -1 if it is undefined.
      */
     public static int getInputSplitSizeInMb(Configuration conf)
     {
-        return conf.getInt(INPUT_SPLIT_SIZE_IN_MB_CONFIG, -1);
+        return conf.getInt(INPUT_SPLIT_SIZE_IN_MIB_CONFIG, -1);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index b34048d..82d5e8a 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -169,7 +169,7 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
                                      .using(insertStatement)
                                      .withPartitioner(ConfigHelper.getOutputPartitioner(conf))
                                      .inDirectory(outputDir)
-                                     .withBufferSizeInMB(Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64")))
+                                     .withBufferSizeInMiB(Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64")))
                                      .withPartitioner(partitioner)
                                      .build();
         }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index bb6881b..584a80c 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@ -330,8 +330,8 @@ public class CqlInputFormat extends org.apache.hadoop.mapreduce.InputFormat<Long
     private Map<TokenRange, Long> getSubSplits(String keyspace, String cfName, TokenRange range, Host host, Configuration conf, Session session)
     {
         int splitSize = ConfigHelper.getInputSplitSize(conf);
-        int splitSizeMb = ConfigHelper.getInputSplitSizeInMb(conf);
-        return describeSplits(keyspace, cfName, range, host, splitSize, splitSizeMb, session);
+        int splitSizeMiB = ConfigHelper.getInputSplitSizeInMb(conf);
+        return describeSplits(keyspace, cfName, range, host, splitSize, splitSizeMiB, session);
     }
 
     private static Map<TokenRange, List<Host>> getRangeMap(String keyspace, Metadata metadata, String targetDC)
diff --git a/src/java/org/apache/cassandra/hints/HintsReader.java b/src/java/org/apache/cassandra/hints/HintsReader.java
index a3944eb..708a916 100644
--- a/src/java/org/apache/cassandra/hints/HintsReader.java
+++ b/src/java/org/apache/cassandra/hints/HintsReader.java
@@ -53,7 +53,7 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
 {
     private static final Logger logger = LoggerFactory.getLogger(HintsReader.class);
 
-    // don't read more than 512 KB of hints at a time.
+    // don't read more than 512 KiB of hints at a time.
     private static final int PAGE_SIZE = 512 << 10;
 
     private final HintsDescriptor descriptor;
diff --git a/src/java/org/apache/cassandra/hints/HintsWriter.java b/src/java/org/apache/cassandra/hints/HintsWriter.java
index 8c70893..468eee2 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriter.java
@@ -284,7 +284,7 @@ class HintsWriter implements AutoCloseable
 
         private void maybeFsync()
         {
-            if (position() >= lastSyncPosition + DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024L)
+            if (position() >= lastSyncPosition + DatabaseDescriptor.getTrickleFsyncIntervalInKiB() * 1024L)
                 fsync();
         }
 
@@ -294,7 +294,7 @@ class HintsWriter implements AutoCloseable
 
             // don't skip page cache for tiny files, on the assumption that if they are tiny, the target node is probably
             // alive, and if so, the file will be closed and dispatched shortly (within a minute), and the file will be dropped.
-            if (position >= DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024L)
+            if (position >= DatabaseDescriptor.getTrickleFsyncIntervalInKiB() * 1024L)
                 NativeLibrary.trySkipCache(fd, 0, position - (position % PAGE_SIZE), file.path());
         }
     }
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
index 875d2f7..7d4f5c9 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
@@ -190,7 +190,7 @@ public class IndexMode
 
         if (maxMemBytes > 100L * 1073741824)
         {
-            logger.error("{} configured as {} is above 100GB, reverting to default 1GB", INDEX_MAX_FLUSH_MEMORY_OPTION, maxMemBytes);
+            logger.error("{} configured as {} is above 100GiB, reverting to default 1GB", INDEX_MAX_FLUSH_MEMORY_OPTION, maxMemBytes);
             maxMemBytes = DEFAULT_MAX_MEM_BYTES;
         }
         return new IndexMode(mode, isLiteral, isAnalyzed, analyzerClass, maxMemBytes);
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 919c199..153f0d5 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -52,7 +52,6 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -356,7 +355,7 @@ public class CQLSSTableWriter implements Closeable
         private IPartitioner partitioner;
 
         private boolean sorted = false;
-        private long bufferSizeInMB = 128;
+        private long bufferSizeInMiB = 128;
 
         protected Builder() {
             this.typeStatements = new ArrayList<>();
@@ -469,15 +468,15 @@ public class CQLSSTableWriter implements Closeable
          * a new SSTable. This correspond roughly to the data size that will have the created
          * sstable.
          * <p>
-         * The default is 128MB, which should be reasonable for a 1GB heap. If you experience
+         * The default is 128MiB, which should be reasonable for a 1GiB heap. If you experience
          * OOM while using the writer, you should lower this value.
          *
-         * @param size the size to use in MB.
+         * @param size the size to use in MiB.
          * @return this builder.
          */
-        public Builder withBufferSizeInMB(int size)
+        public Builder withBufferSizeInMiB(int size)
         {
-            this.bufferSizeInMB = size;
+            this.bufferSizeInMiB = size;
             return this;
         }
 
@@ -494,7 +493,7 @@ public class CQLSSTableWriter implements Closeable
          * the rows in order, which is rarely the case. If you can provide the
          * rows in order however, using this sorted might be more efficient.
          * <p>
-         * Note that if used, some option like withBufferSizeInMB will be ignored.
+         * Note that if used, some option like withBufferSizeInMiB will be ignored.
          *
          * @return this builder.
          */
@@ -548,7 +547,7 @@ public class CQLSSTableWriter implements Closeable
                 TableMetadataRef ref = TableMetadataRef.forOfflineTools(tableMetadata);
                 AbstractSSTableSimpleWriter writer = sorted
                                                    ? new SSTableSimpleWriter(directory, ref, preparedInsert.updatedColumns())
-                                                   : new SSTableSimpleUnsortedWriter(directory, ref, preparedInsert.updatedColumns(), bufferSizeInMB);
+                                                   : new SSTableSimpleUnsortedWriter(directory, ref, preparedInsert.updatedColumns(), bufferSizeInMiB);
 
                 if (formatType != null)
                     writer.setSSTableFormatType(formatType);
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index cb6fcc0..75cca84 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -211,7 +211,7 @@ public class IndexSummaryBuilder implements AutoCloseable
             else
             {
                 // we cannot fully sample this sstable due to too much memory in the index summary, so let's tell the user
-                logger.error("Memory capacity of index summary exceeded (2GB), index summary will not cover full sstable, " +
+                logger.error("Memory capacity of index summary exceeded (2GiB), index summary will not cover full sstable, " +
                              "you should increase min_sampling_level");
             }
         }
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index ee6f349..4e3ae30 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -82,12 +82,12 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
     {
         executor = executorFactory().scheduled(false, "IndexSummaryManager", Thread.MIN_PRIORITY);
 
-        long indexSummarySizeInMB = DatabaseDescriptor.getIndexSummaryCapacityInMB();
+        long indexSummarySizeInMB = DatabaseDescriptor.getIndexSummaryCapacityInMiB();
         int interval = DatabaseDescriptor.getIndexSummaryResizeIntervalInMinutes();
         logger.info("Initializing index summary manager with a memory pool size of {} MB and a resize interval of {} minutes",
                     indexSummarySizeInMB, interval);
 
-        setMemoryPoolCapacityInMB(DatabaseDescriptor.getIndexSummaryCapacityInMB());
+        setMemoryPoolCapacityInMB(DatabaseDescriptor.getIndexSummaryCapacityInMiB());
         setResizeIntervalInMinutes(DatabaseDescriptor.getIndexSummaryResizeIntervalInMinutes());
     }
 
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
index ad79185..8edba75 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
@@ -27,7 +27,7 @@ public interface IndexSummaryManagerMBean
 
     /**
      * Returns the current actual off-heap memory usage of the index summaries for all non-compacting sstables.
-     * @return The amount of memory used in MB.
+     * @return The amount of memory used in MiB.
      */
     public double getMemoryPoolSizeInMB();
 
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
index bac3280..91a3b54 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
@@ -94,7 +94,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
         for (SSTableReader sstable : redistribute)
             total += sstable.getIndexSummaryOffHeapSize();
 
-        logger.info("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
+        logger.info("Beginning redistribution of index summaries for {} sstables with memory pool size {} MiB; current spaced used is {} MiB",
                      redistribute.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
 
         final Map<SSTableReader, Double> readRates = new HashMap<>(redistribute.size());
@@ -119,7 +119,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
 
         long remainingBytes = memoryPoolBytes - nonRedistributingOffHeapSize;
 
-        logger.trace("Index summaries for compacting SSTables are using {} MB of space",
+        logger.trace("Index summaries for compacting SSTables are using {} MiB of space",
                      (memoryPoolBytes - remainingBytes) / 1024.0 / 1024.0);
         List<SSTableReader> newSSTables;
         try (Refs<SSTableReader> refs = Refs.ref(sstablesByHotness))
@@ -133,7 +133,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
         for (SSTableReader sstable : newSSTables)
             total += sstable.getIndexSummaryOffHeapSize();
 
-        logger.info("Completed resizing of index summaries; current approximate memory used: {} MB, time spent: {}ms",
+        logger.info("Completed resizing of index summaries; current approximate memory used: {} MiB, time spent: {}ms",
                     total / 1024.0 / 1024.0, TimeUnit.NANOSECONDS.toMillis(nanoTime() - start));
 
         return newSSTables;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index b56a286..04f22d9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -273,7 +273,7 @@ public abstract class SSTable
     /** @return An estimate of the number of keys contained in the given index file. */
     public static long estimateRowsFromIndex(RandomAccessReader ifile, Descriptor descriptor) throws IOException
     {
-        // collect sizes for the first 10000 keys, or first 10 megabytes of data
+        // collect sizes for the first 10000 keys, or first 10 mebibytes of data
         final int SAMPLES_CAP = 10000, BYTES_CAP = (int)Math.min(10000000, ifile.length());
         int keys = 0;
         while (ifile.getFilePointer() < BYTES_CAP && keys < SAMPLES_CAP)
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index 06b5dcd..e394bbd 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -62,7 +62,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
     private final ILifecycleTransaction transaction; // the readers we are rewriting (updated as they are replaced)
     private final List<SSTableReader> preparedForCommit = new ArrayList<>();
 
-    private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
+    private long currentlyOpenedEarlyAt; // the position (in MiB) in the target file we last (re)opened at
 
     private final List<SSTableWriter> writers = new ArrayList<>();
     private final boolean keepOriginals; // true if we do not want to obsolete the originals
@@ -106,7 +106,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
 
     private static long calculateOpenInterval(boolean shouldOpenEarly)
     {
-        long interval = DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMB() * (1L << 20);
+        long interval = DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMiB() * (1L << 20);
         if (disableEarlyOpeningForTests || !shouldOpenEarly || interval < 0)
             interval = Long.MAX_VALUE;
         return interval;
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index dc43380..85eb8ed 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -71,7 +71,7 @@ public class BigTableWriter extends SSTableWriter
 
     private final SequentialWriterOption writerOption = SequentialWriterOption.newBuilder()
                                                         .trickleFsync(DatabaseDescriptor.getTrickleFsync())
-                                                        .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024)
+                                                        .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKiB() * 1024)
                                                         .build();
 
     public BigTableWriter(Descriptor descriptor,
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriterOption.java b/src/java/org/apache/cassandra/io/util/SequentialWriterOption.java
index 61f375b..599c654 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriterOption.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriterOption.java
@@ -32,10 +32,10 @@ public class SequentialWriterOption
      * Default write option.
      *
      * <ul>
-     *   <li>buffer size: 64 KB
+     *   <li>buffer size: 64 KiB
      *   <li>buffer type: on heap
      *   <li>trickle fsync: false
-     *   <li>trickle fsync byte interval: 10 MB
+     *   <li>trickle fsync byte interval: 10 MiB
      *   <li>finish on close: false
      * </ul>
      */
@@ -108,7 +108,7 @@ public class SequentialWriterOption
         private BufferType bufferType = BufferType.ON_HEAP;
         /* default: no trickle fsync */
         private boolean trickleFsync = false;
-        /* default tricle fsync byte interval: 10MB */
+        /* default tricle fsync byte interval: 10MiB */
         private int trickleFsyncByteInterval = 10 * 1024 * 1024;
         private boolean finishOnClose = false;
 
diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
index 2a51ae3..915e8a3 100644
--- a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
+++ b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
@@ -147,7 +147,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus implements
     /**
      * Writes all data in file channel to stream: <br>
      * * For zero-copy-streaming, 1MiB at a time, with at most 2MiB in flight at once. <br>
-     * * For streaming with SSL, 64kb at a time, with at most 32+64kb (default low water mark + batch size) in flight. <br>
+     * * For streaming with SSL, 64KiB at a time, with at most 32+64KiB (default low water mark + batch size) in flight. <br>
      * <p>
      * This method takes ownership of the provided {@link FileChannel}.
      * <p>
@@ -157,7 +157,7 @@ public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus implements
     public long writeFileToChannel(FileChannel file, RateLimiter limiter) throws IOException
     {
         if (channel.pipeline().get(SslHandler.class) != null)
-            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            // each batch is loaded into ByteBuffer, 64KiB is more BufferPool friendly.
             return writeFileToChannel(file, limiter, 1 << 16);
         else
             // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
diff --git a/src/java/org/apache/cassandra/schema/CompressionParams.java b/src/java/org/apache/cassandra/schema/CompressionParams.java
index fca7e4c..be79ec6 100644
--- a/src/java/org/apache/cassandra/schema/CompressionParams.java
+++ b/src/java/org/apache/cassandra/schema/CompressionParams.java
@@ -370,7 +370,7 @@ public final class CompressionParams
     }
 
     /**
-     * Parse the chunk length (in KB) and returns it as bytes.
+     * Parse the chunk length (in KiB) and returns it as bytes.
      *
      * @param chLengthKB the length of the chunk to parse
      * @return the chunk length in bytes
diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java
index 2bacc343..27d3322 100644
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@ -99,9 +99,9 @@ public class CacheService implements CacheServiceMBean
      */
     private AutoSavingCache<KeyCacheKey, RowIndexEntry> initKeyCache()
     {
-        logger.info("Initializing key cache with capacity of {} MBs.", DatabaseDescriptor.getKeyCacheSizeInMB());
+        logger.info("Initializing key cache with capacity of {} MiBs.", DatabaseDescriptor.getKeyCacheSizeInMiB());
 
-        long keyCacheInMemoryCapacity = DatabaseDescriptor.getKeyCacheSizeInMB() * 1024 * 1024;
+        long keyCacheInMemoryCapacity = DatabaseDescriptor.getKeyCacheSizeInMiB() * 1024 * 1024;
 
         // as values are constant size we can use singleton weigher
         // where 48 = 40 bytes (average size of the key) + 8 bytes (size of value)
@@ -121,7 +121,7 @@ public class CacheService implements CacheServiceMBean
      */
     private AutoSavingCache<RowCacheKey, IRowCacheEntry> initRowCache()
     {
-        logger.info("Initializing row cache with capacity of {} MBs", DatabaseDescriptor.getRowCacheSizeInMiB());
+        logger.info("Initializing row cache with capacity of {} MiBs", DatabaseDescriptor.getRowCacheSizeInMiB());
 
         CacheProvider<RowCacheKey, IRowCacheEntry> cacheProvider;
         String cacheProviderClassName = DatabaseDescriptor.getRowCacheSizeInMiB() > 0
@@ -150,7 +150,7 @@ public class CacheService implements CacheServiceMBean
 
     private AutoSavingCache<CounterCacheKey, ClockAndCount> initCounterCache()
     {
-        logger.info("Initializing counter cache with capacity of {} MBs", DatabaseDescriptor.getCounterCacheSizeInMiB());
+        logger.info("Initializing counter cache with capacity of {} MiBs", DatabaseDescriptor.getCounterCacheSizeInMiB());
 
         long capacity = DatabaseDescriptor.getCounterCacheSizeInMiB() * 1024 * 1024;
 
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 1236279..3f956dd 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -5827,12 +5827,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public void setSSTablePreemptiveOpenIntervalInMB(int intervalInMB)
     {
-        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMB(intervalInMB);
+        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(intervalInMB);
     }
 
     public int getSSTablePreemptiveOpenIntervalInMB()
     {
-        return DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMB();
+        return DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMiB();
     }
 
     public boolean getMigrateKeycacheOnCompaction()
@@ -5889,7 +5889,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         logger.info("updated replica_filtering_protection.cached_rows_fail_threshold to {}", threshold);
     }
 
-    public int getColumnIndexSizeInKB()
+    public int getColumnIndexSizeInKiB()
     {
         return DatabaseDescriptor.getColumnIndexSizeInKiB();
     }
@@ -5914,12 +5914,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int getBatchSizeFailureThreshold()
     {
-        return DatabaseDescriptor.getBatchSizeFailThresholdInKB();
+        return DatabaseDescriptor.getBatchSizeFailThresholdInKiB();
     }
 
     public void setBatchSizeFailureThreshold(int threshold)
     {
-        DatabaseDescriptor.setBatchSizeFailThresholdInKB(threshold);
+        DatabaseDescriptor.setBatchSizeFailThresholdInKiB(threshold);
         logger.info("updated batch_size_fail_threshold to {}", threshold);
     }
 
@@ -6318,7 +6318,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     @Override
     public int getRowIndexSizeWarnThresholdKb()
     {
-        return DatabaseDescriptor.getRowIndexSizeWarnThresholdKb();
+        return DatabaseDescriptor.getRowIndexSizeWarnThresholdKiB();
     }
 
     @Override
@@ -6326,14 +6326,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         if (value < 0)
             throw new IllegalArgumentException("value " + value + " is less than 0; must be positive or zero");
-        DatabaseDescriptor.setRowIndexSizeWarnThresholdKb(value);
+        DatabaseDescriptor.setRowIndexSizeWarnThresholdKiB(value);
         logger.info("updated track_warnings.row_index_size.warn_threshold_kb to {}", value);
     }
 
     @Override
     public int getRowIndexSizeAbortThresholdKb()
     {
-        return DatabaseDescriptor.getRowIndexSizeAbortThresholdKb();
+        return DatabaseDescriptor.getRowIndexSizeAbortThresholdKiB();
     }
 
     @Override
@@ -6341,7 +6341,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         if (value < 0)
             throw new IllegalArgumentException("value " + value + " is less than 0; must be positive or zero");
-        DatabaseDescriptor.setRowIndexSizeAbortThresholdKb(value);
+        DatabaseDescriptor.setRowIndexSizeAbortThresholdKiB(value);
         logger.info("updated track_warnings.row_index_size.abort_threshold_kb to {}", value);
     }
 
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 103b4dd..213c5a9 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -777,7 +777,7 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void setCachedReplicaRowsFailThreshold(int threshold);
 
     /** Returns the granularity of the collation index of rows within a partition **/
-    public int getColumnIndexSizeInKB();
+    public int getColumnIndexSizeInKiB();
     /** Sets the granularity of the collation index of rows within a partition **/
     public void setColumnIndexSize(int columnIndexSizeInKB);
 
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 90f8bb3..f66566d 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1227,14 +1227,14 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.isInitialized();
     }
 
-    public void setColumnIndexSize(int columnIndexSizeInKB)
+    public void setColumnIndexSize(int columnIndexSizeInKiB)
     {
-        ssProxy.setColumnIndexSize(columnIndexSizeInKB);
+        ssProxy.setColumnIndexSize(columnIndexSizeInKiB);
     }
 
     public int getColumnIndexSizeInKB()
     {
-        return ssProxy.getColumnIndexSizeInKB();
+        return ssProxy.getColumnIndexSizeInKiB();
     }
 
     public void setCompactionThroughput(int value)
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 6ee320e..b1f8e6c 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -262,7 +262,7 @@ public class StandaloneScrubber
     {
         if (strategyManager.getCompactionParams().klass().equals(LeveledCompactionStrategy.class))
         {
-            int maxSizeInMB = (int)((cfs.getCompactionStrategyManager().getMaxSSTableBytes()) / (1024L * 1024L));
+            int maxSizeInMiB = (int)((cfs.getCompactionStrategyManager().getMaxSSTableBytes()) / (1024L * 1024L));
             int fanOut = cfs.getCompactionStrategyManager().getLevelFanoutSize();
             for (AbstractStrategyHolder.GroupedSSTableContainer sstableGroup : strategyManager.groupSSTables(sstables))
             {
@@ -270,7 +270,7 @@ public class StandaloneScrubber
                 {
                     List<SSTableReader> groupSSTables = new ArrayList<>(sstableGroup.getGroup(i));
                     // creating the manifest makes sure the leveling is sane:
-                    LeveledManifest.create(cfs, maxSizeInMB, fanOut, groupSSTables);
+                    LeveledManifest.create(cfs, maxSizeInMiB, fanOut, groupSSTables);
                 }
             }
         }
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Info.java b/src/java/org/apache/cassandra/tools/nodetool/Info.java
index 1ee6bac..6966157 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Info.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@ -162,8 +162,8 @@ public class Info extends NodeToolCmd
     }
 
     /**
-     * Returns the total off heap memory used in MB.
-     * @return the total off heap memory used in MB.
+     * Returns the total off heap memory used in MiB.
+     * @return the total off heap memory used in MiB.
      */
     private static double getOffHeapMemoryUsed(NodeProbe probe)
     {
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
index 3c6370c..b96146c 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetBatchlogReplayThrottle.java
@@ -26,7 +26,7 @@ import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
                                                            "This will be reduced proportionally to the number of nodes in the cluster.")
 public class SetBatchlogReplayThrottle extends NodeToolCmd
 {
-    @Arguments(title = "batchlog_replay_throttle", usage = "<value_in_kb_per_sec>", description = "Value in KB per second, 0 to disable throttling", required = true)
+    @Arguments(title = "batchlog_replay_throttle", usage = "<value_in_kb_per_sec>", description = "Value in KiB per second, 0 to disable throttling", required = true)
     private Integer batchlogReplayThrottle = null;
 
     @Override
diff --git a/src/java/org/apache/cassandra/utils/memory/BufferPools.java b/src/java/org/apache/cassandra/utils/memory/BufferPools.java
index 736e1cd..5d80881 100644
--- a/src/java/org/apache/cassandra/utils/memory/BufferPools.java
+++ b/src/java/org/apache/cassandra/utils/memory/BufferPools.java
@@ -35,13 +35,13 @@ public class BufferPools
     /**
      * Used by chunk cache to store decompressed data and buffers may be held by chunk cache for arbitrary period.
      */
-    private static final long FILE_MEMORY_USAGE_THRESHOLD = DatabaseDescriptor.getFileCacheSizeInMB() * 1024L * 1024L;
+    private static final long FILE_MEMORY_USAGE_THRESHOLD = DatabaseDescriptor.getFileCacheSizeInMiB() * 1024L * 1024L;
     private static final BufferPool CHUNK_CACHE_POOL = new BufferPool("chunk-cache", FILE_MEMORY_USAGE_THRESHOLD, true);
 
     /**
      * Used by client-server or inter-node requests, buffers should be released immediately after use.
      */
-    private static final long NETWORKING_MEMORY_USAGE_THRESHOLD = DatabaseDescriptor.getNetworkingCacheSizeInMB() * 1024L * 1024L;
+    private static final long NETWORKING_MEMORY_USAGE_THRESHOLD = DatabaseDescriptor.getNetworkingCacheSizeInMiB() * 1024L * 1024L;
     private static final BufferPool NETWORKING_POOL = new BufferPool("networking", NETWORKING_MEMORY_USAGE_THRESHOLD, false);
 
     static
diff --git a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
index cb5c425..29853e4 100644
--- a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
@@ -32,8 +32,8 @@ import static org.apache.cassandra.utils.concurrent.Semaphore.newSemaphore;
 
 /**
  * This NativeAllocator uses global slab allocation strategy
- * with slab size that scales exponentially from 8kb to 1Mb to
- * serve allocation of up to 128kb.
+ * with slab size that scales exponentially from 8KiB to 1MiB to
+ * serve allocation of up to 128KiB.
  * <p>
  * </p>
  * The slab allocation reduces heap fragmentation from small
diff --git a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
index 538cd3f..17eb29c 100644
--- a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
@@ -32,8 +32,8 @@ import org.apache.cassandra.utils.concurrent.OpOrder;
 
 /**
 + * The SlabAllocator is a bump-the-pointer allocator that allocates
-+ * large (1MB) global regions and then doles them out to threads that
-+ * request smaller sized (up to 128kb) slices into the array.
++ * large (1MiB) global regions and then doles them out to threads that
++ * request smaller sized (up to 128KiB) slices into the array.
  * <p></p>
  * The purpose of this class is to combat heap fragmentation in long lived
  * objects: by ensuring that all allocations with similar lifetimes
diff --git a/test/conf/cassandra-seeds.yaml b/test/conf/cassandra-seeds.yaml
index 0cfd06b..1c38f8e 100644
--- a/test/conf/cassandra-seeds.yaml
+++ b/test/conf/cassandra-seeds.yaml
@@ -17,7 +17,7 @@ listen_address: 127.0.0.1
 storage_port: 7012
 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
diff --git a/test/conf/cassandra_encryption.yaml b/test/conf/cassandra_encryption.yaml
index 47e1312..3b8d08d 100644
--- a/test/conf/cassandra_encryption.yaml
+++ b/test/conf/cassandra_encryption.yaml
@@ -1,14 +1,14 @@
 transparent_data_encryption_options:
-    enabled: true
-    chunk_length_kb: 2
-    cipher: AES/CBC/PKCS5Padding
-    key_alias: testing:1
-    # CBC requires iv length to be 16 bytes
-    # iv_length: 16
-    key_provider: 
-      - class_name: org.apache.cassandra.security.JKSKeyProvider
-        parameters: 
-          - keystore: test/conf/cassandra.keystore
-            keystore_password: cassandra
-            store_type: JCEKS
-            key_password: cassandra
+  enabled: true
+  chunk_length_kb: 2
+  cipher: AES/CBC/PKCS5Padding
+  key_alias: testing:1
+  # CBC requires iv length to be 16 bytes
+  # iv_length: 16
+  key_provider:
+    - class_name: org.apache.cassandra.security.JKSKeyProvider
+      parameters:
+        - keystore: test/conf/cassandra.keystore
+          keystore_password: cassandra
+          store_type: JCEKS
+          key_password: cassandra
diff --git a/test/conf/unit-test-conf/test-native-port.yaml b/test/conf/unit-test-conf/test-native-port.yaml
index a234ad5..c8ed929 100644
--- a/test/conf/unit-test-conf/test-native-port.yaml
+++ b/test/conf/unit-test-conf/test-native-port.yaml
@@ -39,7 +39,7 @@ server_encryption_options:
   truststore_password: cassandra
 incremental_backups: true
 concurrent_compactors: 4
-compaction_throughput_mb_per_sec: 0
+compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
 row_cache_size: 16MiB
 user_defined_functions_enabled: true
diff --git a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
index 5ab83ba..a401bd0 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
@@ -122,10 +122,10 @@ public class HintedHandoffNodetoolTest extends TestBaseImpl
     @Test
     public void testThrottle()
     {
-        Integer throttleInKB = cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKiB);
-        cluster.get(node).nodetoolResult("sethintedhandoffthrottlekb", String.valueOf(throttleInKB * 2)).asserts().success();
+        Integer throttleInKiB = cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKiB);
+        cluster.get(node).nodetoolResult("sethintedhandoffthrottlekb", String.valueOf(throttleInKiB * 2)).asserts().success();
         Integer newThrottleInKB = cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKiB);
-        assertEquals(throttleInKB * 2, newThrottleInKB.intValue());
+        assertEquals(throttleInKiB * 2, newThrottleInKB.intValue());
     }
 
     @SuppressWarnings("Convert2MethodRef")
diff --git a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
index 9ecea99..e1a0a91 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
@@ -30,7 +30,6 @@ import com.google.common.collect.Sets;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.config.SmallestDurationMilliseconds;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICluster;
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
index 221ac86..10a8976 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
@@ -32,7 +32,6 @@ import org.junit.runners.Parameterized;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
 
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertEquals;
 import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
 import static org.apache.cassandra.distributed.shared.AssertUtils.row;
diff --git a/test/distributed/org/apache/cassandra/distributed/test/trackwarnings/RowIndexSizeWarningTest.java b/test/distributed/org/apache/cassandra/distributed/test/trackwarnings/RowIndexSizeWarningTest.java
index e2158e5..5a2957d 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/trackwarnings/RowIndexSizeWarningTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/trackwarnings/RowIndexSizeWarningTest.java
@@ -36,8 +36,8 @@ public class RowIndexSizeWarningTest extends AbstractClientSizeWarning
         AbstractClientSizeWarning.setupClass();
 
         CLUSTER.stream().forEach(i -> i.runOnInstance(() -> {
-            DatabaseDescriptor.setRowIndexSizeWarnThresholdKb(1);
-            DatabaseDescriptor.setRowIndexSizeAbortThresholdKb(2);
+            DatabaseDescriptor.setRowIndexSizeWarnThresholdKiB(1);
+            DatabaseDescriptor.setRowIndexSizeAbortThresholdKiB(2);
 
             // hack to force multiple index entries
             DatabaseDescriptor.setColumnIndexCacheSize(1 << 20);
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
index 935cc8e..7da2b79 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
@@ -79,7 +79,7 @@ public class MixedModeMessageForwardTest extends UpgradeTestBase
                                   .collect(Collectors.joining(","));
 
         new TestCase()
-        .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK).set("request_timeout_in_ms", 30000))
+        .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK).set("request_timeout", "30000ms"))
         .withBuilder(b -> b.withRacks(numDCs, 1, nodesPerDc))
         .nodes(numDCs * nodesPerDc)
         .singleUpgrade(v30)
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index 399f3a9..4d692c6 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -75,7 +75,7 @@ public class LongLeveledCompactionStrategyTest
         CompactionStrategyManager mgr = store.getCompactionStrategyManager();
         LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) mgr.getStrategies().get(1).get(0);
 
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         populateSSTables(store);
 
@@ -138,7 +138,7 @@ public class LongLeveledCompactionStrategyTest
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARDLVL2);
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         populateSSTables(store);
 
@@ -147,7 +147,7 @@ public class LongLeveledCompactionStrategyTest
         CompactionStrategyManager mgr = store.getCompactionStrategyManager();
         LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) mgr.getStrategies().get(1).get(0);
 
-        value = ByteBuffer.wrap(new byte[10 * 1024]); // 10 KB value
+        value = ByteBuffer.wrap(new byte[10 * 1024]); // 10 KiB value
 
         // Adds 10 partitions
         for (int r = 0; r < 10; r++)
@@ -245,7 +245,7 @@ public class LongLeveledCompactionStrategyTest
 
     private void populateSSTables(ColumnFamilyStore store)
     {
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
         int rows = 128;
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index f2bbfa6..2cdc447 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -76,7 +76,7 @@ public class CQLSSTableWriterLongTest
                                                   .inDirectory(dataDir)
                                                   .forTable(schema)
                                                   .using(insert)
-                                                  .withBufferSizeInMB(1)
+                                                  .withBufferSizeInMiB(1)
                                                   .build();
 
         long high = 100;
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index eea55f7..1c7cf0a 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -442,25 +442,25 @@ public class DatabaseDescriptorTest
     }
 
     @Test
-    public void testCalculateDefaultSpaceInMB()
+    public void testCalculateDefaultSpaceInMiB()
     {
         // check prefered size is used for a small storage volume
-        int preferredInMB = 667;
+        int preferredInMiB = 667;
         int numerator = 2;
         int denominator = 3;
         int spaceInBytes = 999 * 1024 * 1024;
 
         assertEquals(666, // total size is less than preferred, so return lower limit
-                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", "/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
+                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", "/path", "setting_name", preferredInMiB, spaceInBytes, numerator, denominator));
 
         // check preferred size is used for a small storage volume
-        preferredInMB = 100;
+        preferredInMiB = 100;
         numerator = 1;
         denominator = 3;
         spaceInBytes = 999 * 1024 * 1024;
 
         assertEquals(100, // total size is more than preferred so keep the configured limit
-                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", "/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
+                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", "/path", "setting_name", preferredInMiB, spaceInBytes, numerator, denominator));
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
index da1d094..d57009e 100644
--- a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
@@ -22,9 +22,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
 
 public class LoadOldYAMLBackwardCompatibilityTest
 {
diff --git a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
similarity index 51%
copy from test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
copy to test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
index da1d094..0421e3e 100644
--- a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/config/ParseAndConvertUnitsTest.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.config;
 
 import org.junit.BeforeClass;
@@ -23,26 +22,23 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
 
-public class LoadOldYAMLBackwardCompatibilityTest
+public class ParseAndConvertUnitsTest
 {
     @BeforeClass
     public static void setupDatabaseDescriptor()
     {
-        System.setProperty("cassandra.config", "cassandra-old.yaml");
         DatabaseDescriptor.daemonInitialization();
     }
 
     // CASSANDRA-15234
     @Test
-    public void testConfigurationLoaderBackwardCompatibility()
+    public void testConfigurationLoaderParser()
     {
         Config config = DatabaseDescriptor.loadConfig();
 
+        //Confirm duration parameters were successfully parsed with the default values in cassandra.yaml
         assertEquals(DurationSpec.inMilliseconds(10800000), config.max_hint_window);
-        assertEquals(DurationSpec.inHours(3), config.max_hint_window);
         assertEquals(DurationSpec.inMilliseconds(0), config.native_transport_idle_timeout);
         assertEquals(DurationSpec.inMilliseconds(10000), config.request_timeout);
         assertEquals(DurationSpec.inMilliseconds(5000), config.read_request_timeout);
@@ -53,62 +49,19 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DurationSpec.inMilliseconds(60000), config.truncate_request_timeout);
         assertEquals(DurationSpec.inSeconds(300), config.streaming_keep_alive_period);
         assertEquals(DurationSpec.inMilliseconds(500), config.slow_query_log_timeout);
-        assertNull(config.memtable_heap_space);
-        assertNull(config.memtable_offheap_space);
-        assertNull( config.repair_session_space);
-        assertEquals(DataStorageSpec.inBytes(4194304), config.internode_application_send_queue_capacity);
-        assertEquals(DataStorageSpec.inBytes(134217728), config.internode_application_send_queue_reserve_endpoint_capacity);
-        assertEquals(DataStorageSpec.inBytes(536870912), config.internode_application_send_queue_reserve_global_capacity);
-        assertEquals(DataStorageSpec.inBytes(4194304), config.internode_application_receive_queue_capacity);
-        assertEquals(DataStorageSpec.inBytes(134217728), config.internode_application_receive_queue_reserve_endpoint_capacity);
-        assertEquals(DataStorageSpec.inBytes(536870912), config.internode_application_receive_queue_reserve_global_capacity);
         assertEquals(DurationSpec.inMilliseconds(2000), config.internode_tcp_connect_timeout);
         assertEquals(DurationSpec.inMilliseconds(30000), config.internode_tcp_user_timeout);
-        assertEquals(DurationSpec.inMilliseconds(300000), config.internode_streaming_tcp_user_timeout);
-        assertEquals(DataStorageSpec.inMebibytes(16), config.native_transport_max_frame_size);
-        assertEquals(DataStorageSpec.inMebibytes(256), config.max_value_size);
-        assertEquals(DataStorageSpec.inKibibytes(4), config.column_index_size);
-        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(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);
-        assertEquals(DurationSpec.inDoubleMilliseconds(0), config.commitlog_sync_group_window);
+        assertEquals(DurationSpec.inMilliseconds(0), config.commitlog_sync_group_window);
         assertEquals(DurationSpec.inMilliseconds(0), config.commitlog_sync_period);
-        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);
+        assertNull(config.periodic_commitlog_sync_lag_block);
         assertEquals(DurationSpec.inMilliseconds(250), config.cdc_free_space_check_interval);
         assertEquals(DurationSpec.inMilliseconds(100), config.dynamic_snitch_update_interval);
         assertEquals(DurationSpec.inMilliseconds(600000), config.dynamic_snitch_reset_interval);
-        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.inKibibytes(10240), config.trickle_fsync_interval);
-        assertEquals(DataStorageSpec.inMebibytes(50), config.sstable_preemptive_open_interval);
-        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);
-        assertNull(config.index_summary_capacity);
         assertEquals(DurationSpec.inMilliseconds(200), config.gc_log_threshold);
+        assertEquals(DurationSpec.inMilliseconds(10000), config.hints_flush_period);
         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);
-        assertTrue(config.user_defined_functions_enabled);
-        assertTrue(config.scripted_user_defined_functions_enabled);
-        assertTrue(config.materialized_views_enabled);
-        assertFalse(config.transient_replication_enabled);
-        assertTrue(config.sasi_indexes_enabled);
-        assertTrue(config.drop_compact_storage_enabled);
-        assertTrue(config.user_defined_functions_threads_enabled);
         assertEquals(DurationSpec.inMilliseconds(2000), config.permissions_validity);
         assertEquals(DurationSpec.inMilliseconds(0), config.permissions_update_interval);
         assertEquals(DurationSpec.inMilliseconds(2000), config.roles_validity);
@@ -116,13 +69,43 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DurationSpec.inMilliseconds(2000), config.credentials_validity);
         assertEquals(DurationSpec.inMilliseconds(0), config.credentials_update_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("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);
+
+        //Confirm space parameters were successfully parsed with the default values in cassandra.yaml
+        assertNull(config.memtable_heap_space);
+        assertNull(config.memtable_offheap_space);
+        assertNull(config.repair_session_space); //null everywhere so should be correct, let's check whether it will bomb
+        assertEquals(new DataStorageSpec("4194304B"), config.internode_application_send_queue_capacity);
+        assertEquals(new DataStorageSpec("134217728B"), config.internode_application_send_queue_reserve_endpoint_capacity);
+        assertEquals(new DataStorageSpec("536870912B"), config.internode_application_send_queue_reserve_global_capacity);
+        assertEquals(new DataStorageSpec("4194304B"), config.internode_application_receive_queue_capacity);
+        assertEquals(new DataStorageSpec("134217728B"), config.internode_application_receive_queue_reserve_endpoint_capacity);
+        assertEquals(new DataStorageSpec("536870912B"), config.internode_application_receive_queue_reserve_global_capacity);
+        assertEquals(new DataStorageSpec("16MiB"), config.native_transport_max_frame_size);
+        assertEquals(new DataStorageSpec("256MiB"), config.max_value_size);
+        assertEquals(new DataStorageSpec("4KiB"), config.column_index_size);
+        assertEquals(new DataStorageSpec("2KiB"), config.column_index_cache_size);
+        assertEquals(new DataStorageSpec("5KiB"), config.batch_size_warn_threshold);
+        assertEquals(new DataStorageSpec("50KiB"), config.batch_size_fail_threshold);
+        assertEquals(new DataStorageSpec("100MiB"), config.compaction_large_partition_warning_threshold);
+        assertNull(config.commitlog_total_space);
+        assertEquals(new DataStorageSpec("5MiB"), config.commitlog_segment_size);
+        assertNull(config.max_mutation_size); //not set explicitly in the default yaml, check the config; not set there too
+        assertEquals(new DataStorageSpec("0MiB"), config.cdc_total_space);
+        assertEquals(new DataStorageSpec("1024KiB"), config.hinted_handoff_throttle);
+        assertEquals(new DataStorageSpec("1024KiB"), config.batchlog_replay_throttle);
+        assertEquals(new DataStorageSpec("10240KiB"), config.trickle_fsync_interval);
+        assertEquals(new DataStorageSpec("50MiB"), config.sstable_preemptive_open_interval);
+        assertNull(config.counter_cache_size);
+        assertNull(config.file_cache_size);
+        assertNull(config.index_summary_capacity);
+        assertEquals(new DataStorageSpec("1MiB"), config.prepared_statements_cache_size);
+        assertNull(config.key_cache_size);
+        assertEquals(new DataStorageSpec("16MiB"), config.row_cache_size);
+
+        //Confirm rate parameters were successfully parsed with the default values in cassandra.yaml
+        assertEquals(DataRateSpec.inMebibytesPerSecond(0), config.compaction_throughput);
+        assertEquals(DataRateSpec.inMebibytesPerSecond(23841858), config.stream_throughput_outbound);
+        assertEquals(DataRateSpec.inMebibytesPerSecond(24), config.inter_dc_stream_throughput_outbound);
     }
-}
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index bfe4815..e5fda16 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -612,7 +612,7 @@ public class SecondaryIndexTest extends CQLTester
     /**
      * Test for CASSANDRA-5732, Can not query secondary index
      * migrated from cql_tests.py:TestCQL.bug_5732_test(),
-     * which was executing with a row cache size of 100 MB
+     * which was executing with a row cache size of 100 MiB
      * and restarting the node, here we just cleanup the cache.
      */
     @Test
@@ -672,14 +672,14 @@ public class SecondaryIndexTest extends CQLTester
         long batchSizeThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
         try
         {
-            DatabaseDescriptor.setBatchSizeFailThresholdInKB( (TOO_BIG / 1024) * 2);
+            DatabaseDescriptor.setBatchSizeFailThresholdInKiB((TOO_BIG / 1024) * 2);
             succeedInsert("BEGIN BATCH\n" +
                           "INSERT INTO %s (a, b, c) VALUES (1, 1, ?) IF NOT EXISTS;\n" +
                           "APPLY BATCH", ByteBuffer.allocate(TOO_BIG));
         }
         finally
         {
-            DatabaseDescriptor.setBatchSizeFailThresholdInKB((int) (batchSizeThreshold / 1024));
+            DatabaseDescriptor.setBatchSizeFailThresholdInKiB((int) (batchSizeThreshold / 1024));
         }
     }
 
@@ -725,14 +725,14 @@ public class SecondaryIndexTest extends CQLTester
         long batchSizeThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
         try
         {
-            DatabaseDescriptor.setBatchSizeFailThresholdInKB( (TOO_BIG / 1024) * 2);
+            DatabaseDescriptor.setBatchSizeFailThresholdInKiB((TOO_BIG / 1024) * 2);
             succeedInsert("BEGIN BATCH\n" +
                           "INSERT INTO %s (a, b, c) VALUES (1, 1, ?) IF NOT EXISTS;\n" +
                           "APPLY BATCH", ByteBuffer.allocate(TOO_BIG));
         }
         finally
         {
-            DatabaseDescriptor.setBatchSizeFailThresholdInKB((int)(batchSizeThreshold / 1024));
+            DatabaseDescriptor.setBatchSizeFailThresholdInKiB((int)(batchSizeThreshold / 1024));
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index ee56d6e..23a94f5 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -317,7 +317,7 @@ public class KeyCacheTest
             throw new IllegalStateException();
 
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
-        boolean noEarlyOpen = DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMB() < 0;
+        boolean noEarlyOpen = DatabaseDescriptor.getSSTablePreemptiveOpenIntervalInMiB() < 0;
 
         // after compaction cache should have entries for new SSTables,
         // but since we have kept a reference to the old sstables,
@@ -339,16 +339,16 @@ public class KeyCacheTest
     }
 
     @Test
-    public void testKeyCacheLoadNegativeCacheLoadTime() throws Exception
+    public void testKeyCacheLoadZeroCacheLoadTime() throws Exception
     {
-        DatabaseDescriptor.setCacheLoadTimeout(-1);
+        DatabaseDescriptor.setCacheLoadTimeout(0);
         String cf = COLUMN_FAMILY7;
 
         createAndInvalidateCache(Collections.singletonList(Pair.create(KEYSPACE1, cf)), 100);
 
         CacheService.instance.keyCache.loadSaved();
 
-        // Here max time to load cache is negative which means no time left to load cache. So the keyCache size should
+        // Here max time to load cache is zero which means no time left to load cache. So the keyCache size should
         // be zero after loadSaved().
         assertKeyCacheSize(0, KEYSPACE1, cf);
         assertEquals(0, CacheService.instance.keyCache.size());
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index e6585a4..a015788 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -94,7 +94,7 @@ public class RowCacheTest
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
-        // set global row cache size to 1 MB
+        // set global row cache size to 1 MiB
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
@@ -139,7 +139,7 @@ public class RowCacheTest
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
-        // set global row cache size to 1 MB
+        // set global row cache size to 1 MiB
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into both column families
@@ -221,7 +221,7 @@ public class RowCacheTest
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
-        // set global row cache size to 1 MB
+        // set global row cache size to 1 MiB
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into column family
@@ -409,7 +409,7 @@ public class RowCacheTest
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
-        // set global row cache size to 1 MB
+        // set global row cache size to 1 MiB
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
@@ -484,7 +484,7 @@ public class RowCacheTest
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
-        // set global row cache size to 1 MB
+        // set global row cache size to 1 MiB
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into both column families
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
index 7cd1445..fbad0b4 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
@@ -90,7 +90,6 @@ public class CommitLogInitWithExceptionTest
             Assert.fail();
         }
 
-
         Assert.assertTrue(CommitLog.instance.segmentManager.executor.isTerminated()); // exit successfully
     }
 
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 31b44cd..a70d06c 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -338,7 +338,7 @@ public abstract class CommitLogTest
         ColumnFamilyStore cfs1 = ks.getColumnFamilyStore(STANDARD1);
         ColumnFamilyStore cfs2 = ks.getColumnFamilyStore(STANDARD2);
 
-        // Roughly 32 MB mutation
+        // Roughly 32 MiB mutation
         Mutation m = new RowUpdateBuilder(cfs1.metadata(), 0, "k")
                      .clustering("bytes")
                      .add("val", ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize() / 4))
@@ -374,7 +374,7 @@ public abstract class CommitLogTest
         ColumnFamilyStore cfs1 = ks.getColumnFamilyStore(STANDARD1);
         ColumnFamilyStore cfs2 = ks.getColumnFamilyStore(STANDARD2);
 
-        // Roughly 32 MB mutation
+        // Roughly 32 MiB mutation
          Mutation rm = new RowUpdateBuilder(cfs1.metadata(), 0, "k")
                   .clustering("bytes")
                   .add("val", ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1))
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
index 25ec662..0af6be3 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
@@ -385,7 +385,7 @@ public class CompactionsCQLTest extends CQLTester
     {
         // write enough data to make sure we use an IndexedReader when doing a read, and make sure it fails when reading a corrupt range tombstone
         DatabaseDescriptor.setCorruptedTombstoneStrategy(Config.CorruptedTombstoneStrategy.exception);
-        final int maxSizePreKB = DatabaseDescriptor.getColumnIndexSizeInKiB();
+        final int maxSizePreKiB = DatabaseDescriptor.getColumnIndexSizeInKiB();
         DatabaseDescriptor.setColumnIndexSize(1024);
         prepareWide();
         RangeTombstone rt = new RangeTombstone(Slice.ALL, new DeletionTime(System.currentTimeMillis(), -1));
@@ -394,7 +394,7 @@ public class CompactionsCQLTest extends CQLTester
         getCurrentColumnFamilyStore().forceBlockingFlush();
         readAndValidate(true);
         readAndValidate(false);
-        DatabaseDescriptor.setColumnIndexSize(maxSizePreKB);
+        DatabaseDescriptor.setColumnIndexSize(maxSizePreKiB);
     }
 
 
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 8dc75f5..40402df 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -115,7 +115,7 @@ public class LeveledCompactionStrategyTest
      */
     @Test
     public void testGrouperLevels() throws Exception{
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         //Need entropy to prevent compression so size is predictable with compression enabled/disabled
         new Random().nextBytes(value.array());
@@ -173,7 +173,7 @@ public class LeveledCompactionStrategyTest
     {
         byte [] b = new byte[100 * 1024];
         new Random().nextBytes(b);
-        ByteBuffer value = ByteBuffer.wrap(b); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(b); // 100 KiB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
         int rows = 40;
@@ -284,7 +284,7 @@ public class LeveledCompactionStrategyTest
     public void testMutateLevel() throws Exception
     {
         cfs.disableAutoCompaction();
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
         int rows = 40;
@@ -325,7 +325,7 @@ public class LeveledCompactionStrategyTest
     {
         byte [] b = new byte[100 * 1024];
         new Random().nextBytes(b);
-        ByteBuffer value = ByteBuffer.wrap(b); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(b); // 100 KiB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
         int rows = 40;
@@ -395,7 +395,7 @@ public class LeveledCompactionStrategyTest
         // Disable auto compaction so cassandra does not compact
         CompactionManager.instance.disableAutoCompaction();
 
-        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KiB value, make it easy to have multiple files
 
         DecoratedKey key1 = Util.dk(String.valueOf(1));
         DecoratedKey key2 = Util.dk(String.valueOf(2));
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
index 7c59628..30c77a7 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
@@ -202,7 +202,7 @@ public class CassandraStreamManagerTest
         Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         Assert.assertEquals(1, allSSTables.size());
         final Token firstToken = allSSTables.iterator().next().first.getToken();
-        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMB(1);
+        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(1);
 
         Set<SSTableReader> sstablesBeforeRewrite = getReadersForRange(new Range<>(firstToken, firstToken));
         Assert.assertEquals(1, sstablesBeforeRewrite.size());
@@ -234,7 +234,7 @@ public class CassandraStreamManagerTest
         }
         finally
         {
-            DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMB(50);
+            DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(50);
             done.set(true);
             t.join(20);
         }
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java
index 461c13c..a751984 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java
@@ -84,7 +84,7 @@ public class CompressedSequentialWriterReopenTest extends CQLTester
             execute("insert into %s (id, t) values (?, ?)", i, ByteBuffer.wrap(blob));
         }
         getCurrentColumnFamilyStore().forceBlockingFlush();
-        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMB(1);
+        DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMiB(1);
         getCurrentColumnFamilyStore().forceMajorCompaction();
     }
 
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index ad476fd..92b627a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -173,8 +173,8 @@ public class CQLSSTableWriterTest
     public void testSyncWithinPartition() throws Exception
     {
         // Check that the write respect the buffer size even if we only insert rows withing the same partition (#7360)
-        // To do that simply, we use a writer with a buffer of 1MB, and write 2 rows in the same partition with a value
-        // > 1MB and validate that this created more than 1 sstable.
+        // To do that simply, we use a writer with a buffer of 1MiB, and write 2 rows in the same partition with a value
+        // > 1MiB and validate that this created more than 1 sstable.
         String schema = "CREATE TABLE " + qualifiedTable + " ("
                       + "  k int PRIMARY KEY,"
                       + "  v blob"
@@ -184,7 +184,7 @@ public class CQLSSTableWriterTest
                                                   .inDirectory(dataDir)
                                                   .using(insert)
                                                   .forTable(schema)
-                                                  .withBufferSizeInMB(1)
+                                                  .withBufferSizeInMiB(1)
                                                   .build();
 
         ByteBuffer val = ByteBuffer.allocate(1024 * 1050);
@@ -212,7 +212,7 @@ public class CQLSSTableWriterTest
                                                   .inDirectory(dataDir)
                                                   .forTable(schema)
                                                   .using(insert)
-                                                  .withBufferSizeInMB(1)
+                                                  .withBufferSizeInMiB(1)
                                                   .build();
 
         for (int i = 0 ; i < 50000 ; i++) {
@@ -644,7 +644,7 @@ public class CQLSSTableWriterTest
                                                   .inDirectory(dataDir)
                                                   .forTable(schema)
                                                   .using(insert)
-                                                  .withBufferSizeInMB(1)
+                                                  .withBufferSizeInMiB(1)
                                                   .build();
 
         final int ID_OFFSET = 1000;
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
index 1318227..d0680f8 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
@@ -112,7 +112,7 @@ public class IndexSummaryTest
     }
 
     /**
-     * Test an index summary whose total size is bigger than 2GB,
+     * Test an index summary whose total size is bigger than 2GiB,
      * the index summary builder should log an error but it should still
      * create an index summary, albeit one that does not cover the entire sstable.
      */
@@ -146,7 +146,7 @@ public class IndexSummaryTest
     }
 
     /**
-     * Test an index summary whose total size is bigger than 2GB,
+     * Test an index summary whose total size is bigger than 2GiB,
      * having updated IndexSummaryBuilder.defaultExpectedKeySize to match the size,
      * the index summary should be downsampled automatically.
      */
diff --git a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
index 7ff7845..17fb0f2 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
@@ -59,24 +59,24 @@ public class LargePartitionsTest extends CQLTester
         return new String(ch);
     }
 
-    private static final int rowKBytes = 8;
+    private static final int rowKibibytes = 8;
 
-    private void withPartitionSize(long partitionKBytes, long totalMBytes) throws Throwable
+    private void withPartitionSize(long partitionKibibytes, long totalMBytes) throws Throwable
     {
-        long totalKBytes = totalMBytes * 1024L;
+        long totalKibibytes = totalMBytes * 1024L;
 
         createTable("CREATE TABLE %s (pk text, ck text, val text, PRIMARY KEY (pk, ck))");
 
-        String name = "part=" + partitionKBytes + "k total=" + totalMBytes + 'M';
+        String name = "part=" + partitionKibibytes + "k total=" + totalMBytes + 'M';
 
         measured("INSERTs for " + name, () -> {
-            for (long writtenKBytes = 0L; writtenKBytes < totalKBytes; writtenKBytes += partitionKBytes)
+            for (long writtenKibibytes = 0L; writtenKibibytes < totalKibibytes; writtenKibibytes += partitionKibibytes)
             {
-                String pk = Long.toBinaryString(writtenKBytes);
-                for (long kbytes = 0L; kbytes < partitionKBytes; kbytes += rowKBytes)
+                String pk = Long.toBinaryString(writtenKibibytes);
+                for (long kibibytes = 0L; kibibytes < partitionKibibytes; kibibytes += rowKibibytes)
                 {
-                    String ck = Long.toBinaryString(kbytes);
-                    execute("INSERT INTO %s (pk, ck, val) VALUES (?,?,?)", pk, ck, randomText(rowKBytes * 1024));
+                    String ck = Long.toBinaryString(kibibytes);
+                    execute("INSERT INTO %s (pk, ck, val) VALUES (?,?,?)", pk, ck, randomText(rowKibibytes * 1024));
                 }
             }
         });
@@ -91,20 +91,20 @@ public class LargePartitionsTest extends CQLTester
             keyCacheMetrics("after compaction");
         });
 
-        measured("SELECTs 1 for " + name, () -> selects(partitionKBytes, totalKBytes));
+        measured("SELECTs 1 for " + name, () -> selects(partitionKibibytes, totalKibibytes));
 
-        measured("SELECTs 2 for " + name, () -> selects(partitionKBytes, totalKBytes));
+        measured("SELECTs 2 for " + name, () -> selects(partitionKibibytes, totalKibibytes));
 
         CacheService.instance.keyCache.clear();
-        measured("Scan for " + name, () -> scan(partitionKBytes, totalKBytes));
+        measured("Scan for " + name, () -> scan(partitionKibibytes, totalKibibytes));
     }
 
-    private void selects(long partitionKBytes, long totalKBytes) throws Throwable
+    private void selects(long partitionKibibytes, long totalKibibytes) throws Throwable
     {
         for (int i = 0; i < 50000; i++)
         {
-            long pk = ThreadLocalRandom.current().nextLong(totalKBytes / partitionKBytes) * partitionKBytes;
-            long ck = ThreadLocalRandom.current().nextLong(partitionKBytes / rowKBytes) * rowKBytes;
+            long pk = ThreadLocalRandom.current().nextLong(totalKibibytes / partitionKibibytes) * partitionKibibytes;
+            long ck = ThreadLocalRandom.current().nextLong(partitionKibibytes / rowKibibytes) * rowKibibytes;
             execute("SELECT val FROM %s WHERE pk=? AND ck=?",
                     Long.toBinaryString(pk),
                     Long.toBinaryString(ck)).one();
@@ -114,9 +114,9 @@ public class LargePartitionsTest extends CQLTester
         keyCacheMetrics("after all selects");
     }
 
-    private void scan(long partitionKBytes, long totalKBytes) throws Throwable
+    private void scan(long partitionKibibytes, long totalKibibytes) throws Throwable
     {
-        long pk = ThreadLocalRandom.current().nextLong(totalKBytes / partitionKBytes) * partitionKBytes;
+        long pk = ThreadLocalRandom.current().nextLong(totalKibibytes / partitionKibibytes) * partitionKibibytes;
         Iterator<UntypedResultSet.Row> iter = execute("SELECT val FROM %s WHERE pk=?", Long.toBinaryString(pk)).iterator();
         int i = 0;
         while (iter.hasNext())
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index b67cc8b..3dc6151 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -168,10 +168,10 @@ public class SSTableLoaderTest
                                                   .inDirectory(dataDir)
                                                   .forTable(String.format(schema, KEYSPACE1, CF_STANDARD2))
                                                   .using(String.format(query, KEYSPACE1, CF_STANDARD2))
-                                                  .withBufferSizeInMB(1)
+                                                  .withBufferSizeInMiB(1)
                                                   .build();
 
-        int NB_PARTITIONS = 5000; // Enough to write >1MB and get at least one completed sstable before we've closed the writer
+        int NB_PARTITIONS = 5000; // Enough to write >1MiB and get at least one completed sstable before we've closed the writer
 
         for (int i = 0; i < NB_PARTITIONS; i++)
         {
diff --git a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
index 46bdded..6f25b2e 100644
--- a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
+++ b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
@@ -52,26 +52,26 @@ public class FileUtilsTest
         // test straightforward conversions for each unit
         assertEquals("FileUtils.parseFileSize() failed to parse a whole number of bytes",
             256L, FileUtils.parseFileSize("256 bytes"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of kilobytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of kibibytes",
             2048L, FileUtils.parseFileSize("2 KiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of megabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of mebibytes",
             4194304L, FileUtils.parseFileSize("4 MiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of gigabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of gibibytes",
             3221225472L, FileUtils.parseFileSize("3 GiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of terabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a whole number of tebibytes",
             5497558138880L, FileUtils.parseFileSize("5 TiB"));
         // test conversions of fractional units
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of kilobytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of kibibytes",
             1536L, FileUtils.parseFileSize("1.5 KiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of kilobytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of kibibytes",
             4434L, FileUtils.parseFileSize("4.33 KiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of megabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of mebibytes",
             2359296L, FileUtils.parseFileSize("2.25 MiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of megabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of mebibytes",
             3292529L, FileUtils.parseFileSize("3.14 MiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of gigabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of gibibytes",
             1299227607L, FileUtils.parseFileSize("1.21 GiB"));
-        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of terabytes",
+        assertEquals("FileUtils.parseFileSize() failed to parse a rational number of tebibytes",
             6621259022467L, FileUtils.parseFileSize("6.022 TiB"));
     }
 
diff --git a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
index 967ddac..f933cf1 100644
--- a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
@@ -138,8 +138,8 @@ public class RandomAccessReaderTest
     @Test
     public void testVeryLarge() throws IOException
     {
-        final long SIZE = 1L << 32; // 2GB
-        Parameters params = new Parameters(SIZE, 1 << 20); // 1MB
+        final long SIZE = 1L << 32; // 2GiB
+        Parameters params = new Parameters(SIZE, 1 << 20); // 1MiB
 
 
         try (ChannelProxy channel = new ChannelProxy("abc", new FakeFileChannel(SIZE));
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java b/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
index b9269cd..5b02ccf 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
@@ -70,7 +70,7 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
         List<File> splitFiles = Arrays.asList(sstablesDir.tryList());
         splitFiles.stream().forEach(f -> {
             if (f.name().endsWith("Data.db") && !origSstables.contains(f))
-                assertTrue(f.name() + " is way bigger than 1MB: [" + f.length() + "] bytes",
+                assertTrue(f.name() + " is way bigger than 1MiB: [" + f.length() + "] bytes",
                            f.length() <= 1024 * 1024 * 1.2); //give a 20% margin on size check
         });
         assertTrue(origSstables.size() < splitFiles.size());
@@ -93,7 +93,7 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
         List<File> splitFiles = Arrays.asList(sstablesDir.tryList());
         splitFiles.stream().forEach(f -> {
             if (f.name().endsWith("Data.db") && !origSstables.contains(f))
-                assertTrue(f.name() + " is way bigger than 1MB: [" + f.length() + "] bytes",
+                assertTrue(f.name() + " is way bigger than 1MiB: [" + f.length() + "] bytes",
                            f.length() <= 1024 * 1024 * 1.2); //give a 20% margin on size check
         });
         assertTrue(origSstables.size() < splitFiles.size());
@@ -123,7 +123,7 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
 
         Set<SSTableReader> sstables = cfs.getLiveSSTables();
         sstableFileName = sstables.iterator().next().getFilename();
-        assertTrue("Generated sstable must be at least 1MB", (new File(sstableFileName)).length() > 1024*1024);
+        assertTrue("Generated sstable must be at least 1MiB", (new File(sstableFileName)).length() > 1024*1024);
         sstablesDir = new File(sstableFileName).parent();
         origSstables = Arrays.asList(sstablesDir.tryList());
         System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetGetColumnIndexSizeTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetGetColumnIndexSizeTest.java
index b30cf06..45736c9 100644
--- a/test/unit/org/apache/cassandra/tools/nodetool/SetGetColumnIndexSizeTest.java
+++ b/test/unit/org/apache/cassandra/tools/nodetool/SetGetColumnIndexSizeTest.java
@@ -90,7 +90,7 @@ public class SetGetColumnIndexSizeTest extends CQLTester
 
         assertGetThroughput(columnIndexSizeInKB);
 
-        assertThat(StorageService.instance.getColumnIndexSizeInKB()).isEqualTo(columnIndexSizeInKB);
+        assertThat(StorageService.instance.getColumnIndexSizeInKiB()).isEqualTo(columnIndexSizeInKB);
     }
 
     private static void assertSetInvalidColumnIndexSize(String columnIndexSizeInKB, String expectedErrorMessage, int expectedErrorCode)
diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
index 1cdcc22..ae70fd4 100644
--- a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
@@ -600,19 +600,19 @@ public class MerkleTreeTest
         Assert.assertEquals(1, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance, 0, 32));
         Assert.assertEquals(1, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance, 1, 32));
 
-        // The minimum of 1 megabyte split between RF=3 should yield trees of around 10
+        // The minimum of 1 mebibyte split between RF=3 should yield trees of around 10
         Assert.assertEquals(10, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance,
                                                                      1048576 / 3, 32));
 
-        // With a single megabyte of space we should get 12
+        // With a single mebibyte of space we should get 12
         Assert.assertEquals(12, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance,
                                                                      1048576, 32));
 
-        // With 100 megabytes we should get a limit of 19
+        // With 100 mebibytes we should get a limit of 19
         Assert.assertEquals(19, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance,
                                                                      100 * 1048576, 32));
 
-        // With 300 megabytes we should get the old limit of 20
+        // With 300 mebibytes we should get the old limit of 20
         Assert.assertEquals(20, MerkleTree.estimatedMaxDepthForBytes(Murmur3Partitioner.instance,
                                                                      300 * 1048576, 32));
         Assert.assertEquals(20, MerkleTree.estimatedMaxDepthForBytes(RandomPartitioner.instance,
@@ -635,7 +635,7 @@ public class MerkleTreeTest
         Range<Token> fullRange = new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken());
         MerkleTree tree = new MerkleTree(partitioner, fullRange, RECOMMENDED_DEPTH, 0);
 
-        // Test 16 kilobyte -> 16 megabytes
+        // Test 16 kibibyte -> 16 mebibytes
         for (int i = 14; i < 24; i ++)
         {
             long numBytes = 1 << i;
diff --git a/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java b/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java
index eb3cc1b..4e68d8b 100644
--- a/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java
+++ b/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java
@@ -353,7 +353,7 @@ public class BufferPoolTest
     @Test
     public void testChunkExhausted()
     {
-        final int size = BufferPool.NORMAL_CHUNK_SIZE / 64; // 1kbit
+        final int size = BufferPool.NORMAL_CHUNK_SIZE / 64; // 1kibibit
         int[] sizes = new int[128];
         Arrays.fill(sizes, size);
 
@@ -852,8 +852,8 @@ public class BufferPoolTest
     @Test
     public void testRecyclePartialFreeChunk()
     {
-        // normal chunk size is 128kb
-        int halfNormalChunk = BufferPool.NORMAL_CHUNK_SIZE / 2; // 64kb, half of normal chunk
+        // normal chunk size is 128KiB
+        int halfNormalChunk = BufferPool.NORMAL_CHUNK_SIZE / 2; // 64KiB, half of normal chunk
         List<ByteBuffer> toRelease = new ArrayList<>();
 
         // allocate three buffers on different chunks
diff --git a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 84ac5f1..c4ecb07 100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@ -353,7 +353,7 @@ public class StressCQLSSTableWriter implements Closeable
         private IPartitioner partitioner;
 
         private boolean sorted = false;
-        private long bufferSizeInMB = 128;
+        private long bufferSizeInMiB = 128;
 
         protected Builder()
         {
@@ -502,15 +502,15 @@ public class StressCQLSSTableWriter implements Closeable
          * a new SSTable. This correspond roughly to the data size that will have the created
          * sstable.
          * <p>
-         * The default is 128MB, which should be reasonable for a 1GB heap. If you experience
+         * The default is 128MiB, which should be reasonable for a 1GB heap. If you experience
          * OOM while using the writer, you should lower this value.
          *
          * @param size the size to use in MB.
          * @return this builder.
          */
-        public Builder withBufferSizeInMB(int size)
+        public Builder withBufferSizeInMiB(int size)
         {
-            this.bufferSizeInMB = size;
+            this.bufferSizeInMiB = size;
             return this;
         }
 
@@ -527,7 +527,7 @@ public class StressCQLSSTableWriter implements Closeable
          * the rows in order, which is rarely the case. If you can provide the
          * rows in order however, using this sorted might be more efficient.
          * <p>
-         * Note that if used, some option like withBufferSizeInMB will be ignored.
+         * Note that if used, some option like withBufferSizeInMiB will be ignored.
          *
          * @return this builder.
          */
@@ -558,7 +558,7 @@ public class StressCQLSSTableWriter implements Closeable
                 UpdateStatement preparedInsert = prepareInsert();
                 AbstractSSTableSimpleWriter writer = sorted
                                                      ? new SSTableSimpleWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.updatedColumns())
-                                                     : new SSTableSimpleUnsortedWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.updatedColumns(), bufferSizeInMB);
+                                                     : new SSTableSimpleUnsortedWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.updatedColumns(), bufferSizeInMiB);
 
                 if (formatType != null)
                     writer.setSSTableFormatType(formatType);
diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
index a42829a..25ad29a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
@@ -273,10 +273,10 @@ public abstract class CompactionStress implements Runnable
     @Command(name = "write", description = "write data directly to disk")
     public static class DataWriter extends CompactionStress
     {
-        private static double BYTES_IN_GB = 1024 * 1014 * 1024;
+        private static double BYTES_IN_GIB = 1024 * 1014 * 1024;
 
         @Option(name = { "-g", "--gbsize"}, description = "Total GB size on disk you wish to write", required = true)
-        Integer totalSizeGb;
+        Integer totalSizeGiB;
 
         @Option(name = { "-t", "--threads" }, description = "Number of sstable writer threads (default 2)")
         Integer threads = 2;
@@ -284,7 +284,7 @@ public abstract class CompactionStress implements Runnable
         @Option(name = { "-c", "--partition-count"}, description = "Number of partitions to loop over (default 1000000)")
         Integer partitions = 1000000;
 
-        @Option(name = { "-b", "--buffer-size-mb"}, description = "Buffer in MB writes before writing new sstable (default 128)")
+        @Option(name = { "-b", "--buffer-size-mb"}, description = "Buffer in MiB writes before writing new sstable (default 128)")
         Integer bufferSize = 128;
 
         @Option(name = { "-r", "--range-aware"}, description = "Splits the local ranges in number of data directories and makes sure we never write the same token in two different directories (default true)")
@@ -326,13 +326,13 @@ public abstract class CompactionStress implements Runnable
                 });
             }
 
-            double currentSizeGB;
-            while ((currentSizeGB = directories.getRawDiretoriesSize() / BYTES_IN_GB) < totalSizeGb)
+            double currentSizeGiB;
+            while ((currentSizeGiB = directories.getRawDiretoriesSize() / BYTES_IN_GIB) < totalSizeGiB)
             {
                 if (finished.getCount() == 0)
                     break;
 
-                System.out.println(String.format("Written %.2fGB of %dGB", currentSizeGB, totalSizeGb));
+                System.out.println(String.format("Written %.2fGB of %dGB", currentSizeGiB, totalSizeGiB));
 
                 Uninterruptibles.sleepUninterruptibly(3, TimeUnit.SECONDS);
             }
@@ -340,8 +340,8 @@ public abstract class CompactionStress implements Runnable
             workManager.stop();
             Uninterruptibles.awaitUninterruptibly(finished);
 
-            currentSizeGB = directories.getRawDiretoriesSize() / BYTES_IN_GB;
-            System.out.println(String.format("Finished writing %.2fGB", currentSizeGB));
+            currentSizeGiB = directories.getRawDiretoriesSize() / BYTES_IN_GIB;
+            System.out.println(String.format("Finished writing %.2fGB", currentSizeGiB));
         }
     }
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaInsert.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaInsert.java
index 4cfea82..258e688 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaInsert.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaInsert.java
@@ -151,7 +151,7 @@ public class SchemaInsert extends SchemaStatement
     {
         return StressCQLSSTableWriter.builder()
                                .withCfs(cfs)
-                               .withBufferSizeInMB(bufferSize)
+                               .withBufferSizeInMiB(bufferSize)
                                .forTable(tableSchema)
                                .using(insertStatement)
                                .rangeAware(makeRangeAware)

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