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