private static void applySimpleConfig()

in src/java/org/apache/cassandra/config/DatabaseDescriptor.java [565:1212]


    private static void applySimpleConfig()
    {
        //Doing this first before all other things in case other pieces of config want to construct
        //InetAddressAndPort and get the right defaults
        InetAddressAndPort.initializeDefaultPort(getStoragePort());

        validateUpperBoundStreamingConfig();

        if (conf.auto_snapshot_ttl != null)
        {
            try
            {
                autoSnapshoTtl = new DurationSpec.IntSecondsBound(conf.auto_snapshot_ttl);
            }
            catch (IllegalArgumentException e)
            {
                throw new ConfigurationException("Invalid value of auto_snapshot_ttl: " + conf.auto_snapshot_ttl, false);
            }
        }

        if (conf.commitlog_sync == null)
        {
            throw new ConfigurationException("Missing required directive CommitLogSync", false);
        }

        if (conf.commitlog_sync == CommitLogSync.batch)
        {
            if (conf.commitlog_sync_period.toMilliseconds() != 0)
            {
                throw new ConfigurationException("Batch sync specified, but commitlog_sync_period found.", false);
            }
            logger.debug("Syncing log with batch mode");
        }
        else if (conf.commitlog_sync == CommitLogSync.group)
        {
            if (conf.commitlog_sync_group_window.toMilliseconds() == 0)
            {
                throw new ConfigurationException("Missing value for commitlog_sync_group_window.", false);
            }
            else if (conf.commitlog_sync_period.toMilliseconds() != 0)
            {
                throw new ConfigurationException("Group sync specified, but commitlog_sync_period found. Only specify commitlog_sync_group_window when using group sync", false);
            }
            logger.debug("Syncing log with a group window of {}", conf.commitlog_sync_period.toString());
        }
        else
        {
            if (conf.commitlog_sync_period.toMilliseconds() == 0)
            {
                throw new ConfigurationException("Missing value for commitlog_sync_period.", false);
            }
            logger.debug("Syncing log with a period of {}", conf.commitlog_sync_period.toString());
        }

        if (conf.accord.journal.flushPeriod == null)
        {
            conf.accord.journal.flushPeriod = conf.commitlog_sync_period;
            if (conf.accord.journal.flushMode == PERIODIC && conf.commitlog_sync_period.toMilliseconds() == 0)
            {
                logger.warn("Accord journal is configured in periodic mode, while Cassandra commit log is configured in {} mode", conf.commitlog_sync);
                conf.accord.journal.flushPeriod = conf.accord.journal.periodicFlushLagBlock;
            }
        }

        /* evaluate the DiskAccessMode Config directive, which also affects indexAccessMode selection */
        if (conf.disk_access_mode == DiskAccessMode.auto || conf.disk_access_mode == DiskAccessMode.mmap_index_only)
        {
            conf.disk_access_mode = DiskAccessMode.standard;
            indexAccessMode = DiskAccessMode.mmap;
        }
        else if (conf.disk_access_mode == DiskAccessMode.legacy)
        {
            conf.disk_access_mode = hasLargeAddressSpace() ? DiskAccessMode.mmap : DiskAccessMode.standard;
            indexAccessMode = conf.disk_access_mode;
        }
        else if (conf.disk_access_mode == DiskAccessMode.direct)
        {
            throw new ConfigurationException(String.format("DiskAccessMode '%s' is not supported", DiskAccessMode.direct));
        }
        else
        {
            indexAccessMode = conf.disk_access_mode;
        }
        logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);

        /* phi convict threshold for FailureDetector */
        if (conf.phi_convict_threshold < 5 || conf.phi_convict_threshold > 16)
        {
            throw new ConfigurationException("phi_convict_threshold must be between 5 and 16, but was " + conf.phi_convict_threshold, false);
        }

        /* Thread per pool */
        if (conf.concurrent_reads < 2)
        {
            throw new ConfigurationException("concurrent_reads must be at least 2, but was " + conf.concurrent_reads, false);
        }

        if (conf.concurrent_writes < 2 && TEST_FAIL_MV_LOCKS_COUNT.getString("").isEmpty())
        {
            throw new ConfigurationException("concurrent_writes must be at least 2, but was " + conf.concurrent_writes, false);
        }

        if (conf.concurrent_counter_writes < 2)
            throw new ConfigurationException("concurrent_counter_writes must be at least 2, but was " + conf.concurrent_counter_writes, false);

        if (conf.concurrent_accord_operations < 1)
            throw new ConfigurationException("concurrent_accord_operations must be at least 1, but was " + conf.concurrent_accord_operations, false);

        if (conf.networking_cache_size == null)
            conf.networking_cache_size = new DataStorageSpec.IntMebibytesBound(Math.min(128, (int) (Runtime.getRuntime().maxMemory() / (16 * 1048576))));

        if (conf.file_cache_size == null)
            conf.file_cache_size = new DataStorageSpec.IntMebibytesBound(Math.min(512, (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576))));

        // round down for SSDs and round up for spinning disks
        if (conf.file_cache_round_up == null)
            conf.file_cache_round_up = conf.disk_optimization_strategy == Config.DiskOptimizationStrategy.spinning;

        if (conf.memtable_offheap_space == null)
            conf.memtable_offheap_space = new DataStorageSpec.IntMebibytesBound((int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)));
        // for the moment, we default to twice as much on-heap space as off-heap, as heap overhead is very large
        if (conf.memtable_heap_space == null)
            conf.memtable_heap_space = new DataStorageSpec.IntMebibytesBound((int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)));
        if (conf.memtable_heap_space.toMebibytes() == 0)
            throw new ConfigurationException("memtable_heap_space must be positive, but was " + conf.memtable_heap_space, false);
        logger.info("Global memtable on-heap threshold is enabled at {}", conf.memtable_heap_space);
        if (conf.memtable_offheap_space.toMebibytes() == 0)
            logger.info("Global memtable off-heap threshold is disabled, HeapAllocator will be used instead");
        else
            logger.info("Global memtable off-heap threshold is enabled at {}", conf.memtable_offheap_space);

        if (conf.repair_session_max_tree_depth != null)
        {
            logger.warn("repair_session_max_tree_depth has been deprecated and should be removed from cassandra.yaml. Use repair_session_space instead");
            if (conf.repair_session_max_tree_depth < 10)
                throw new ConfigurationException("repair_session_max_tree_depth should not be < 10, but was " + conf.repair_session_max_tree_depth);
            if (conf.repair_session_max_tree_depth > 20)
                logger.warn("repair_session_max_tree_depth of " + conf.repair_session_max_tree_depth + " > 20 could lead to excessive memory usage");
        }
        else
        {
            conf.repair_session_max_tree_depth = 20;
        }

        if (conf.repair_session_space == null)
            conf.repair_session_space = new DataStorageSpec.IntMebibytesBound(Math.max(1, (int) (Runtime.getRuntime().maxMemory() / (16 * 1048576))));

        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.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);

        long valueInBytes = conf.native_transport_max_frame_size.toBytes();
        if (valueInBytes < 0 || valueInBytes > Integer.MAX_VALUE - 1)
        {
            throw new ConfigurationException(String.format("native_transport_max_frame_size must be positive value < %dB, but was %dB",
                                                           Integer.MAX_VALUE,
                                                           valueInBytes),
                                             false);
        }

        if (conf.column_index_size != null)
            checkValidForByteConversion(conf.column_index_size, "column_index_size");
        checkValidForByteConversion(conf.column_index_cache_size, "column_index_cache_size");
        checkValidForByteConversion(conf.batch_size_warn_threshold, "batch_size_warn_threshold");

        // if data dirs, commitlog dir, or saved caches dir are set in cassandra.yaml, use that.  Otherwise,
        // use -Dcassandra.storagedir (set in cassandra-env.sh) as the parent dir for data/, commitlog/, and saved_caches/
        if (conf.commitlog_directory == null)
        {
            conf.commitlog_directory = storagedirFor("commitlog");
        }

        initializeCommitLogDiskAccessMode();
        if (commitLogWriteDiskAccessMode != conf.commitlog_disk_access_mode)
            logger.info("commitlog_disk_access_mode resolved to: {}", commitLogWriteDiskAccessMode);

        if (conf.accord.journal_directory == null)
        {
            conf.accord.journal_directory = storagedirFor("accord_journal");
        }

        if (conf.hints_directory == null)
        {
            conf.hints_directory = storagedirFor("hints");
        }

        if (conf.native_transport_max_request_data_in_flight == null)
        {
            conf.native_transport_max_request_data_in_flight = new DataStorageSpec.LongBytesBound(Runtime.getRuntime().maxMemory() / 10);
        }

        if (conf.native_transport_max_request_data_in_flight_per_ip == null)
        {
            conf.native_transport_max_request_data_in_flight_per_ip = new DataStorageSpec.LongBytesBound(Runtime.getRuntime().maxMemory() / 40);
        }

        if (conf.native_transport_rate_limiting_enabled)
            logger.info("Native transport rate-limiting enabled at {} requests/second.", conf.native_transport_max_requests_per_second);
        else
            logger.info("Native transport rate-limiting disabled.");

        if (conf.commitlog_total_space == null)
        {
            final int preferredSizeInMiB = 8192;
            // use 1/4 of available space.  See discussion on #10013 and #10199
            final long totalSpaceInBytes = tryGetSpace(conf.commitlog_directory, FileStore::getTotalSpace);
            int defaultSpaceInMiB = calculateDefaultSpaceInMiB("commitlog",
                                                               conf.commitlog_directory,
                                                               "commitlog_total_space",
                                                               preferredSizeInMiB,
                                                               totalSpaceInBytes, 1, 4);
            conf.commitlog_total_space = new DataStorageSpec.IntMebibytesBound(defaultSpaceInMiB);
        }

        if (conf.cdc_enabled)
        {
            if (conf.cdc_raw_directory == null)
            {
                conf.cdc_raw_directory = storagedirFor("cdc_raw");
            }

            if (conf.cdc_total_space.toMebibytes() == 0)
            {
                final int preferredSizeInMiB = 4096;
                // use 1/8th of available space.  See discussion on #10013 and #10199 on the CL, taking half that for CDC
                final long totalSpaceInBytes = tryGetSpace(conf.cdc_raw_directory, FileStore::getTotalSpace);
                int defaultSpaceInMiB = calculateDefaultSpaceInMiB("cdc",
                                                                   conf.cdc_raw_directory,
                                                                   "cdc_total_space",
                                                                   preferredSizeInMiB,
                                                                   totalSpaceInBytes, 1, 8);
                conf.cdc_total_space = new DataStorageSpec.IntMebibytesBound(defaultSpaceInMiB);
            }

            logger.info("cdc_enabled is true. Starting casssandra node with Change-Data-Capture enabled.");
        }

        if (conf.saved_caches_directory == null)
        {
            conf.saved_caches_directory = storagedirFor("saved_caches");
        }
        if (conf.data_file_directories == null || conf.data_file_directories.length == 0)
        {
            conf.data_file_directories = new String[]{ storagedir("data_file_directories") + File.pathSeparator() + "data" };
        }

        long dataFreeBytes = 0;
        /* data file and commit log directories. they get created later, when they're needed. */
        for (String datadir : conf.data_file_directories)
        {
            if (datadir == null)
                throw new ConfigurationException("data_file_directories must not contain empty entry", false);
            if (datadir.equals(conf.local_system_data_file_directory))
                throw new ConfigurationException("local_system_data_file_directory must not be the same as any data_file_directories", false);
            if (datadir.equals(conf.commitlog_directory))
                throw new ConfigurationException("commitlog_directory must not be the same as any data_file_directories", false);
            if (datadir.equals(conf.accord.journal_directory))
                throw new ConfigurationException("accord.journal_directory must not be the same as any data_file_directories", false);
            if (datadir.equals(conf.hints_directory))
                throw new ConfigurationException("hints_directory must not be the same as any data_file_directories", false);
            if (datadir.equals(conf.saved_caches_directory))
                throw new ConfigurationException("saved_caches_directory must not be the same as any data_file_directories", false);

            dataFreeBytes = saturatedSum(dataFreeBytes, tryGetSpace(datadir, FileStore::getUnallocatedSpace));
        }
        if (dataFreeBytes < 64 * ONE_GIB) // 64 GB
            logger.warn("Only {} free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
                        FBUtilities.prettyPrintMemory(dataFreeBytes));

        if (conf.local_system_data_file_directory != null)
        {
            if (conf.local_system_data_file_directory.equals(conf.commitlog_directory))
                throw new ConfigurationException("local_system_data_file_directory must not be the same as the commitlog_directory", false);
            if (conf.local_system_data_file_directory.equals(conf.accord.journal_directory))
                throw new ConfigurationException("local_system_data_file_directory must not be the same as the accord.journal_directory", false);
            if (conf.local_system_data_file_directory.equals(conf.saved_caches_directory))
                throw new ConfigurationException("local_system_data_file_directory must not be the same as the saved_caches_directory", false);
            if (conf.local_system_data_file_directory.equals(conf.hints_directory))
                throw new ConfigurationException("local_system_data_file_directory must not be the same as the hints_directory", false);

            long freeBytes = tryGetSpace(conf.local_system_data_file_directory, FileStore::getUnallocatedSpace);

            if (freeBytes < ONE_GIB)
                logger.warn("Only {} free in the system data volume. Consider adding more capacity or removing obsolete snapshots",
                            FBUtilities.prettyPrintMemory(freeBytes));
        }

        if (conf.commitlog_directory.equals(conf.accord.journal_directory))
            throw new ConfigurationException("accord.journal_directory must not be the same as the commitlog_directory", false);
        if (conf.commitlog_directory.equals(conf.hints_directory))
            throw new ConfigurationException("hints_directory must not be the same as the commitlog_directory", false);
        if (conf.commitlog_directory.equals(conf.saved_caches_directory))
            throw new ConfigurationException("saved_caches_directory must not be the same as the commitlog_directory", false);

        if (conf.accord.journal_directory.equals(conf.hints_directory))
            throw new ConfigurationException("hints_directory must not be the same as the accord.journal_directory", false);
        if (conf.accord.journal_directory.equals(conf.saved_caches_directory))
            throw new ConfigurationException("saved_caches_directory must not be the same as the accord.journal_directory", false);

        if (conf.hints_directory.equals(conf.saved_caches_directory))
            throw new ConfigurationException("saved_caches_directory must not be the same as the hints_directory", false);

        if (conf.memtable_flush_writers == 0)
        {
            conf.memtable_flush_writers = conf.data_file_directories.length == 1 ? 2 : 1;
        }

        if (conf.memtable_flush_writers < 1)
            throw new ConfigurationException("memtable_flush_writers must be at least 1, but was " + conf.memtable_flush_writers, false);

        if (conf.memtable_cleanup_threshold == null)
        {
            conf.memtable_cleanup_threshold = (float) (1.0 / (1 + conf.memtable_flush_writers));
        }
        else
        {
            logger.warn("memtable_cleanup_threshold has been deprecated and should be removed from cassandra.yaml");
        }

        if (conf.memtable_cleanup_threshold < 0.01f)
            throw new ConfigurationException("memtable_cleanup_threshold must be >= 0.01, but was " + conf.memtable_cleanup_threshold, false);
        if (conf.memtable_cleanup_threshold > 0.99f)
            throw new ConfigurationException("memtable_cleanup_threshold must be <= 0.99, but was " + conf.memtable_cleanup_threshold, false);
        if (conf.memtable_cleanup_threshold < 0.1f)
            logger.warn("memtable_cleanup_threshold is set very low [{}], which may cause performance degradation", conf.memtable_cleanup_threshold);

        if (conf.concurrent_compactors == null)
            conf.concurrent_compactors = Math.min(8, Math.max(2, Math.min(FBUtilities.getAvailableProcessors(), conf.data_file_directories.length)));

        if (conf.concurrent_compactors <= 0)
            throw new ConfigurationException("concurrent_compactors should be strictly greater than 0, but was " + conf.concurrent_compactors, false);

        applyConcurrentValidations(conf);
        applyRepairCommandPoolSize(conf);
        applyReadThresholdsValidations(conf);

        if (conf.concurrent_materialized_view_builders <= 0)
            throw new ConfigurationException("concurrent_materialized_view_builders should be strictly greater than 0, but was " + conf.concurrent_materialized_view_builders, false);

        if (conf.num_tokens != null && conf.num_tokens > MAX_NUM_TOKENS)
            throw new ConfigurationException(String.format("A maximum number of %d tokens per node is supported", MAX_NUM_TOKENS), false);

        try
        {
            // if prepared_statements_cache_size option was set to "auto" then size of the cache should be "max(1/256 of Heap (in MiB), 10MiB)"
            preparedStatementsCacheSizeInMiB = (conf.prepared_statements_cache_size == null)
                                               ? Math.max(10, (int) (Runtime.getRuntime().maxMemory() / 1024 / 1024 / 256))
                                               : conf.prepared_statements_cache_size.toMebibytes();

            if (preparedStatementsCacheSizeInMiB == 0)
                throw new NumberFormatException(); // to escape duplicating error message

            // we need this assignment for the Settings virtual table - CASSANDRA-17734
            conf.prepared_statements_cache_size = new DataStorageSpec.LongMebibytesBound(preparedStatementsCacheSizeInMiB);
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("prepared_statements_cache_size option was set incorrectly to '"
                                             + (conf.prepared_statements_cache_size != null ? conf.prepared_statements_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if key_cache_size option was set to "auto" then size of the cache should be "min(5% of Heap (in MiB), 100MiB)
            keyCacheSizeInMiB = (conf.key_cache_size == null)
                                ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024)), 100)
                                : conf.key_cache_size.toMebibytes();

            if (keyCacheSizeInMiB < 0)
                throw new NumberFormatException(); // to escape duplicating error message

            // we need this assignment for the Settings Virtual Table - CASSANDRA-17734
            conf.key_cache_size = new DataStorageSpec.LongMebibytesBound(keyCacheSizeInMiB);
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("key_cache_size option was set incorrectly to '"
                                             + (conf.key_cache_size != null ? conf.key_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if counter_cache_size option was set to "auto" then size of the cache should be "min(2.5% of Heap (in MiB), 50MiB)
            counterCacheSizeInMiB = (conf.counter_cache_size == null)
                                    ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.025 / 1024 / 1024)), 50)
                                    : conf.counter_cache_size.toMebibytes();

            if (counterCacheSizeInMiB < 0)
                throw new NumberFormatException(); // to escape duplicating error message
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("counter_cache_size option was set incorrectly to '"
                                             + (conf.counter_cache_size != null ? conf.counter_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if paxosCacheSizeInMiB option was set to "auto" then size of the cache should be "min(1% of Heap (in MB), 50MB)
            paxosCacheSizeInMiB = (conf.paxos_cache_size == null)
                                  ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.01 / 1024 / 1024)), 50)
                                  : conf.paxos_cache_size.toMebibytes();

            if (paxosCacheSizeInMiB < 0)
                throw new NumberFormatException(); // to escape duplicating error message
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("paxos_cache_size option was set incorrectly to '"
                                             + conf.paxos_cache_size + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if accordCacheSizeInMiB option was set to "auto" then size of the cache should be "max(10% of Heap (in MB), 1MB)
            accordCacheSizeInMiB = (conf.accord.cache_size == null)
                                  ? Math.max(1, (int) ((Runtime.getRuntime().totalMemory() * 0.10) / 1024 / 1024))
                                  : conf.accord.cache_size.toMebibytes();

            if (accordCacheSizeInMiB < 0)
                throw new NumberFormatException(); // to escape duplicating error message
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("accord.cache_size option was set incorrectly to '"
                                             + conf.accord.cache_size + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if accordWorkingSetSizeInMiB option was set to "auto" then size of the working set should be "max(5% of Heap (in MB), 1MB)
            // if negative, there is no limit
            accordWorkingSetSizeInMiB = (conf.accord.working_set_size == null)
                                  ? Math.max(1, (int) ((Runtime.getRuntime().totalMemory() * 0.05) / 1024 / 1024))
                                  : conf.accord.working_set_size.toMebibytes();
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("accord.working_set_size option was set incorrectly to '"
                                             + conf.accord.working_set_size + "', supported values are <integer> >= 0.", false);
        }

        try
        {
            // if consensusMigrationCacheSizeInMiB option was set to "auto" then size of the cache should be "min(1% of Heap (in MB), 50MB)
            consensusMigrationCacheSizeInMiB = (conf.consensus_migration_cache_size == null)
                                               ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.01 / 1024 / 1024)), 50)
                                               : conf.consensus_migration_cache_size.toMebibytes();

            if (consensusMigrationCacheSizeInMiB < 0)
                throw new NumberFormatException(); // to escape duplicating error message
        }
        catch (NumberFormatException e)
        {
            throw new ConfigurationException("consensus_migration_cache_size option was set incorrectly to '"
                                             + conf.consensus_migration_cache_size + "', supported values are <integer> >= 0.", false);
        }

        // we need this assignment for the Settings virtual table - CASSANDRA-17735
        conf.counter_cache_size = new DataStorageSpec.LongMebibytesBound(counterCacheSizeInMiB);

        // if set to empty/"auto" then use 5% of Heap size
        indexSummaryCapacityInMiB = (conf.index_summary_capacity == null)
                                    ? Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024))
                                    : conf.index_summary_capacity.toMebibytes();

        if (indexSummaryCapacityInMiB < 0)
            throw new ConfigurationException("index_summary_capacity option was set incorrectly to '"
                                             + conf.index_summary_capacity.toString() + "', it should be a non-negative integer.", false);

        // we need this assignment for the Settings virtual table - CASSANDRA-17735
        conf.index_summary_capacity = new DataStorageSpec.LongMebibytesBound(indexSummaryCapacityInMiB);

        if (conf.user_defined_functions_fail_timeout.toMilliseconds() < conf.user_defined_functions_warn_timeout.toMilliseconds())
            throw new ConfigurationException("user_defined_functions_warn_timeout must less than user_defined_function_fail_timeout", false);

        if (!conf.allow_insecure_udfs && !conf.user_defined_functions_threads_enabled)
            throw new ConfigurationException("To be able to set enable_user_defined_functions_threads: false you need to set allow_insecure_udfs: true - this is an unsafe configuration and is not recommended.");

        if (conf.allow_extra_insecure_udfs)
            logger.warn("Allowing java.lang.System.* access in UDFs is dangerous and not recommended. Set allow_extra_insecure_udfs: false to disable.");

        if (conf.scripted_user_defined_functions_enabled)
            throw new ConfigurationException("JavaScript user-defined functions were removed in CASSANDRA-18252. " +
                                             "Hooks are planned to be introduced as part of CASSANDRA-17280");

        if (conf.commitlog_segment_size.toMebibytes() == 0)
            throw new ConfigurationException("commitlog_segment_size must be positive, but was "
                                             + conf.commitlog_segment_size.toString(), false);
        else if (conf.commitlog_segment_size.toMebibytes() >= 2048)
            throw new ConfigurationException("commitlog_segment_size must be smaller than 2048, but was "
                                             + conf.commitlog_segment_size.toString(), false);

        if (conf.max_mutation_size == null)
            conf.max_mutation_size = new DataStorageSpec.IntKibibytesBound(conf.commitlog_segment_size.toKibibytes() / 2);
        else if (conf.commitlog_segment_size.toKibibytes() < 2 * conf.max_mutation_size.toKibibytes())
            throw new ConfigurationException("commitlog_segment_size must be at least twice the size of max_mutation_size / 1024", false);

        if (conf.native_transport_max_message_size == null)
        {
            conf.native_transport_max_message_size = new DataStorageSpec.LongBytesBound(calculateDefaultNativeTransportMaxMessageSizeInBytes());
        }
        else
        {
            nativeTransportMaxMessageSizeConfiguredExplicitly = true;
            long maxCqlMessageSize = conf.native_transport_max_message_size.toBytes();
            if (maxCqlMessageSize > conf.native_transport_max_request_data_in_flight.toBytes())
                throw new ConfigurationException("native_transport_max_message_size must not exceed native_transport_max_request_data_in_flight", false);

            if (maxCqlMessageSize > conf.native_transport_max_request_data_in_flight_per_ip.toBytes())
                throw new ConfigurationException("native_transport_max_message_size must not exceed native_transport_max_request_data_in_flight_per_ip", false);

        }
        nativeTransportMaxMessageSizeInBytes = conf.native_transport_max_message_size.toBytes();

        // native transport encryption options
        if (conf.client_encryption_options != null)
            conf.client_encryption_options.applyConfig();

        if (conf.jmx_server_options == null)
        {
            conf.jmx_server_options = JMXServerOptions.createParsingSystemProperties();
        }
        else if (JMXServerOptions.isEnabledBySystemProperties())
        {
                throw new ConfigurationException("Configure either jmx_server_options in cassandra.yaml and comment out " +
                                                 "configure_jmx function call in cassandra-env.sh or keep cassandra-env.sh " +
                                                 "to call configure_jmx function but you have to keep jmx_server_options " +
                                                 "in cassandra.yaml commented out.");
        }

        conf.jmx_server_options.jmx_encryption_options.applyConfig();

        if (conf.snapshot_links_per_second < 0)
            throw new ConfigurationException("snapshot_links_per_second must be >= 0");

        if (conf.max_value_size.toMebibytes() == 0)
            throw new ConfigurationException("max_value_size must be positive", false);
        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);

        switch (conf.disk_optimization_strategy)
        {
            case ssd:
                diskOptimizationStrategy = new SsdDiskOptimizationStrategy(conf.disk_optimization_page_cross_chance);
                break;
            case spinning:
                diskOptimizationStrategy = new SpinningDiskOptimizationStrategy();
                break;
        }

        if (conf.compressed_read_ahead_buffer_size.toKibibytes() > 0 && conf.compressed_read_ahead_buffer_size.toKibibytes() < 256)
            throw new ConfigurationException("compressed_read_ahead_buffer_size must be at least 256KiB (set to 0 to disable), but was " + conf.compressed_read_ahead_buffer_size, false);

        if (conf.server_encryption_options != null)
        {
            conf.server_encryption_options.applyConfig();

            if (conf.server_encryption_options.legacy_ssl_storage_port_enabled &&
                conf.server_encryption_options.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
            {
                throw new ConfigurationException("legacy_ssl_storage_port_enabled is true (enabled) with internode encryption disabled (none). Enable encryption or disable the legacy ssl storage port.");
            }
        }

        if (conf.internode_max_message_size != null)
        {
            long maxMessageSize = conf.internode_max_message_size.toBytes();

            if (maxMessageSize > conf.internode_application_receive_queue_reserve_endpoint_capacity.toBytes())
                throw new ConfigurationException("internode_max_message_size must no exceed internode_application_receive_queue_reserve_endpoint_capacity", false);

            if (maxMessageSize > conf.internode_application_receive_queue_reserve_global_capacity.toBytes())
                throw new ConfigurationException("internode_max_message_size must no exceed internode_application_receive_queue_reserve_global_capacity", false);

            if (maxMessageSize > conf.internode_application_send_queue_reserve_endpoint_capacity.toBytes())
                throw new ConfigurationException("internode_max_message_size must no exceed internode_application_send_queue_reserve_endpoint_capacity", false);

            if (maxMessageSize > conf.internode_application_send_queue_reserve_global_capacity.toBytes())
                throw new ConfigurationException("internode_max_message_size must no exceed internode_application_send_queue_reserve_global_capacity", false);
        }
        else
        {
            long maxMessageSizeInBytes =
            Math.min(conf.internode_application_receive_queue_reserve_endpoint_capacity.toBytes(),
                     conf.internode_application_send_queue_reserve_endpoint_capacity.toBytes());

            conf.internode_max_message_size = new DataStorageSpec.IntBytesBound(maxMessageSizeInBytes);
        }

        validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades);

        if (conf.default_keyspace_rf < conf.minimum_replication_factor_fail_threshold)
        {
            throw new ConfigurationException(String.format("default_keyspace_rf (%d) cannot be less than minimum_replication_factor_fail_threshold (%d)",
                                                           conf.default_keyspace_rf, conf.minimum_replication_factor_fail_threshold));
        }

        if (conf.paxos_repair_parallelism <= 0)
            conf.paxos_repair_parallelism = Math.max(1, conf.concurrent_writes / 8);

        Paxos.setPaxosVariant(conf.paxos_variant);
        if (conf.paxos_state_purging == null)
            conf.paxos_state_purging = PaxosStatePurging.legacy;

        logInitializationOutcome(logger);

        if (conf.max_space_usable_for_compactions_in_percentage < 0 || conf.max_space_usable_for_compactions_in_percentage > 1)
            throw new ConfigurationException("max_space_usable_for_compactions_in_percentage must be between 0 and 1", false);

        if (conf.dump_heap_on_uncaught_exception && DatabaseDescriptor.getHeapDumpPath() == null)
            throw new ConfigurationException(String.format("Invalid configuration. Heap dump is enabled but cannot create heap dump output path: %s.", conf.heap_dump_path != null ? conf.heap_dump_path : "null"));

        conf.sai_options.validate();

        List<ConsistencyLevel> progressBarrierCLsArr = Arrays.asList(ALL, EACH_QUORUM, LOCAL_QUORUM, QUORUM, ONE, NODE_LOCAL);
        Set<ConsistencyLevel> progressBarrierCls = new HashSet<>(progressBarrierCLsArr);
        if (!progressBarrierCls.contains(conf.progress_barrier_min_consistency_level))
        {
            throw new ConfigurationException(String.format("Invalid value for progress_barrier_min_consistency_level %s. Allowed values: %s",
                                                           conf.progress_barrier_min_consistency_level, progressBarrierCLsArr));
        }

        if (!progressBarrierCls.contains(conf.progress_barrier_default_consistency_level))
        {
            throw new ConfigurationException(String.format("Invalid value for.progress_barrier_default_consistency_level %s. Allowed values: %s",
                                                           conf.progress_barrier_default_consistency_level, progressBarrierCLsArr));
        }

        if (conf.native_transport_min_backoff_on_queue_overload.toMilliseconds() <= 0)
            throw new ConfigurationException(" be positive");

        if (conf.native_transport_min_backoff_on_queue_overload.toMilliseconds() >= conf.native_transport_max_backoff_on_queue_overload.toMilliseconds())
            throw new ConfigurationException(String.format("native_transport_min_backoff_on_queue_overload should be strictly less than native_transport_max_backoff_on_queue_overload, but %s >= %s",
                                                           conf.native_transport_min_backoff_on_queue_overload,
                                                           conf.native_transport_max_backoff_on_queue_overload));

        if (conf.use_deterministic_table_id)
            logger.warn("use_deterministic_table_id is no longer supported and should be removed from cassandra.yaml.");

        // run audit logging options through sanitation and validation
        if (conf.audit_logging_options != null)
            setAuditLoggingOptions(conf.audit_logging_options);
    }