cassandra.yaml configuration file
The cassandra.yaml
file is the main configuration file for DataStax Enterprise.
The dse.yaml file is the primary configuration file for security, DSE Search, DSE Graph, and DSE Analytics.
After changing properties in the |
Syntax
For the properties in each section, the parent setting has zero spaces. Each child entry requires at least two spaces. Adhere to the YAML syntax and retain the spacing.
-
Default values that are not defined are shown as Default: none.
-
Internally defined default values are described.
Default values can be defined internally, commented out, or have implementation dependencies on other properties in the
cassandra.yaml
file. Additionally, some commented-out values may not match the actual default values. The commented out values are recommended alternatives to the default values.
Organization
The configuration properties are grouped into the following sections:
-
The minimal properties needed for configuring a cluster.
-
If you have changed any of the default directories during installation, set these properties to the new locations. Make sure you have root access.
-
Properties most frequently used when configuring DataStax Enterprise.
-
Tuning performance and system resource utilization, including commit log, compaction, memory, disk I/O, CPU, reads, and writes.
-
Properties for advanced users or properties that are less commonly used.
-
Configure authentication, authorization, and role management provided with DSE Unified Authentication.
-
User-defined functions (UDF) properties
Configure how UDF code is executed inside Cassandra daemons.
-
Continuous paging options Configure memory, threads, and duration when pushing pages continuously to the client.
-
Memory leak detection settingsConfigure memory leak detection.
Data directory configuration
Distributing data across multiple disks maximizes throughput and ensures efficient disk I/O. Cassandra allows you to specify multiple directories for storing your data to achieve this.
To configure a single data directory in the cassandra.yaml
file:
data_file_directories:
- /var/lib/cassandra01/data
For multiple data directories:
data_file_directories:
- /disk1/datadir
- /disk2/datadir
- /disk3/datadir
Quick start properties
The minimal properties needed for configuring a cluster.
- cluster_name
-
The name of the cluster. This setting prevents nodes in one logical cluster from joining another. All nodes in a cluster must have the same value.
Default: 'Test Cluster'
- listen_address
-
The IP address or hostname that the database binds to for connecting this node to other nodes.
-
Never set listen_address to 0.0.0.0.
-
Set listen_address or listen_interface, do not set both.
Default: localhost
-
- listen_interface
-
The interface that the database binds to for connecting to other nodes. Interfaces must correspond to a single address. IP aliasing is not supported.
Set listen_address or listen_interface, not both.
- listen_interface_prefer_ipv6
-
Use IPv4 or IPv6 when interface is specified by name.
-
false - Use first IPv4 address.
-
true - Use first IPv6 address.
-
-
When only a single address is used, that address is selected without regard to this setting.
Default: false
Default directories
If you have changed any of the default directories during installation, set these properties to the new locations. Make sure you have root access.
- data_file_directories
-
The directory where table data is stored on disk. The database distributes data evenly across the location, subject to the granularity of the configured compaction strategy.
For production, DataStax recommends RAID 0 and SSDs.
Default: - /var/lib/cassandra/data
- commitlog_directory
-
The directory where the commit log is stored.
For optimal write performance, place the commit log on a separate disk partition, or ideally on a separate physical device, from the data file directories. Because the commit log is append only, a hard disk drive (HDD) is acceptable.
Default: /var/lib/cassandra/commitlog
The
commitlog_directory
and thecdc_raw_directory
must reside on the same partition. Keep these directories in separate sub-folders that are not nested.
- cdc_raw_directory
-
The directory where the change data capture (CDC) commit log segments are stored on flush. DataStax recommends a physical device that is separate from the data directories. See Change Data Capture (CDC) logging.
Default: /var/lib/cassandra/cdc_raw
The
cdc_raw_directory
and thecommitlog_directory
must reside on the same partition. Keep these directories in separate sub-folders that are not nested. - hints_directory
-
The directory in which hints are stored.
Default: /var/lib/cassandra/hints
- metadata_directory
-
The metadata directory that holds information about the cluster, the local node and its peers.
Default: /var/lib/cassandra/metadata
If not set, the default directory is <$CASSANDRA_HOME>/data/metadata.
- saved_caches_directory
-
The directory location where table key and row caches are stored.
Default: /var/lib/cassandra/saved_caches
Commonly used properties
Properties most frequently used when configuring DataStax Enterprise.
Before starting a node for the first time, DataStax recommends that you carefully evaluate your requirements.
Common initialization properties
Be sure to set the properties in the Quick start section as well. |
- commit_failure_policy
-
Policy for commit disk failures:
-
die - Shut down the node and kill the JVM, so the node can be replaced.
-
stop - Shut down the node, leaving the node effectively dead, available for inspection using JMX.
-
stop_commit - Shut down the commit log, letting writes collect but continuing to service reads.
-
ignore - Ignore fatal errors and let the batches fail.
Default: stop
-
- disk_optimization_strategy
-
The strategy for optimizing disk reads.
-
ssd - Solid state disks.
-
spinning - Spinning disks. Default: ssd
-
- disk_failure_policy
-
How the database responds to disk failure. Recommend settings: stop or best_effort.
-
die - Shut down gossip and client transports, and kill the JVM for any file system errors or single SSTable errors, so the node can be replaced.
-
stop_paranoid - Shut down the node, even for single SSTable errors.
-
stop - Shut down the node, leaving the node effectively dead, but available for inspection using JMX.
-
best_effort - Stop using the failed disk and respond to requests based on the remaining available SSTables. This setting allows obsolete data at consistency level of ONE.
-
ignore - Ignore fatal errors and lets the requests fail; all file system errors are logged but otherwise ignored.
Default: stop
-
- endpoint_snitch
-
A class that implements the IEndpointSnitch interface. The database uses the snitch to locate nodes and route requests.
Use only snitch implementations bundled with DSE.
-
DseSimpleSnitch
Appropriate only for development deployments. Proximity is determined by DSE workload, which places transactional, analytics, and search nodes into their separate datacenters. Does not recognize datacenter or rack information.
-
GossipingPropertyFileSnitch
Recommended for production. Reads rack and datacenter for the local node in
cassandra-rackdc.properties
file and propagates these values to other nodes via gossip. For migration from the PropertyFileSnitch, uses thecassandra-topology.properties
file if it is present. -
PropertyFileSnitch
Determines proximity by rack and datacenter that are explicitly configured in
cassandra-topology.properties
file. -
Ec2Snitch
For EC2 deployments in a single region. Loads region and availability zone information from the Amazon EC2 API. The region is treated as the datacenter, the availability zone is treated as the rack, and uses only private IP addresses. For this reason, Ec2Snitch does not work across multiple regions.
-
Ec2MultiRegionSnitch
Uses the public IP as the broadcast_address to allow cross-region connectivity. This means you must also set seed addresses to the public IP and open the storage_port or ssl_storage_port on the public IP firewall. For intra-region traffic, the database switches to the private IP after establishing a connection.
-
RackInferringSnitch
Proximity is determined by rack and datacenter, which are assumed to correspond to the 3rd and 2nd octet of each node’s IP address, respectively. Best used as an example for writing a custom snitch class (unless this happens to match your deployment conventions).
-
GoogleCloudSnitch
Use for deployments on Google Cloud Platform across one or more regions. The region is treated as a datacenter and the availability zones are treated as racks within the datacenter. All communication occurs over private IP addresses within the same logical network.
-
CloudstackSnitch
Use the CloudstackSnitch for Apache Cloudstack environments.
See Snitches.
Default: com.datastax.bdp.snitch.DseSimpleSnitch
-
- seed_provider
-
The addresses of hosts that are designated as contact points in the cluster. A joining node contacts one of the nodes in the -seeds list to learn the topology of the ring.
Use only seed provider implementations bundled with DSE.
-
class_name - The class that handles the seed logic. It can be customized, but this is typically not required.
Default: org.apache.cassandra.locator.SimpleSeedProvider
-
seeds - A comma delimited list of addresses that are used by gossip for bootstrapping new nodes joining a cluster. If your cluster includes multiple nodes, you must change the list from the default value to the IP address of one of the nodes.
Default: "127.0.0.1"
Making every node a seed node is not recommended because of increased maintenance and reduced gossip performance. Gossip optimization is not critical, but it is recommended to use a small seed list (approximately three nodes per datacenter).
Default: org.apache.cassandra.locator.SimpleSeedProvider
-
Common compaction settings
- compaction_throughput_mb_per_sec
-
The MB per second to throttle compaction for the entire system. The faster the database inserts data, the faster the system must compact in order to keep the SSTable count down.
-
16 to 32 x rate of write throughput in MB/second, recommended value.
-
0 - Disable compaction throttling.
Default: 16
-
Memtable settings
- memtable_space_in_mb
-
Total permitted memory to use for memtables. When this threshold is exceeded, writes are not accepted until a flush completes. A flush is triggered based on memtable_cleanup_threshold. When not set:
-
If the deprecated settings are not present, the calculated default is 1/4 of the heap size.
-
If the deprecated memtable_heap_space_in_mb or memtable_offheap_space_in_mb settings are present, an error is logged and the appropriate value is used based on memtable_allocation_type.
See Tuning the Java heap.
Default: 2048
-
- memtable_cleanup_threshold
-
Ratio used for automatic memtable flush.
With the caveat below in mind, the
memtable_cleanup_threshold
setting is valid in DSE 6.7 and 6.8; this setting is removed in Apache Cassandra® 4.0.Generally, the calculated default value is appropriate and does not need adjusting. DataStax recommends contacting the DataStax Services team before changing this value.
Default: 0.15
- memtable_allocation_type
-
The method the database uses to allocate and manage memtable memory.
-
offheap_objects - Native memory, eliminating NIO buffer heap overhead.
-
heap_buffers - On heap NIO (non-blocking I/O) buffers.
-
offheap_buffers - Off heap (direct) NIO buffers.
Default: offheap_objects
-
- memtable_flush_writers
-
The number of memtable flush writer threads per disk and the total number of memtables that can be flushed concurrently, generally a combination of computer and I/O bound. Memtable flushing is more CPU efficient than memtable ingest. A single thread can keep up with the ingest rate of a server on a single fast disk, until the server temporarily becomes IO bound under contention, typically with compaction. Generally, the default value is appropriate and does not need adjusting. However, you may need to increase the number of
memtable_flush_writers
if you observe slow performance with write operations.Default: 8
- memtable_heap_space_in_mb (deprecated)
This setting is deprecated. Use memtable_space_in_mb instead. |
- memtable_offheap_space_in_mb (deprecated)
This setting is deprecated. Use memtable_space_in_mb instead. |
Common automatic backup settings
- incremental_backups
-
Enables incremental backups.
-
true - Enable incremental backups to create a hard link to each SSTable flushed or streamed locally in a
backups
subdirectory of the keyspace data. Incremental backups enable storing backups off site without transferring entire snapshots.The database does not automatically clear incremental backup files. DataStax recommends setting up a process to clear incremental backup hard links each time a new snapshot is created.
-
false - Do not enable incremental backups.
Default: false
-
- snapshot_before_compaction
-
Enables a snapshot before each compaction. A snapshot is useful to back up data when there is a data format change.
Be careful using this option, the database does not clean up older snapshots automatically.
Default: false
- snapshot_before_dropping_column
-
When enabled, every time the user drops a column/columns from a table, a snapshot is created on each node in the cluster before the change in schema is applied. Those snapshots have the same name on each node. For example:
auto-snapshot_drop-column-<columnname>_20200515143511000
. The name includes the name of the dropped column and the timestamp (UTC) when the column was dropped.The database does not automatically clear incremental backup files. DataStax recommends setting up a process to clear incremental backup hard links each time a new snapshot is created.
Default: false
Performance tuning properties
Tuning performance and system resource utilization, including commit log, compaction, memory, disk I/O, CPU, reads, and writes.
Performing tuning properties include:
Commit log settings
- commitlog_sync
-
Commit log synchronization method:
-
periodic - Send ACK signal for writes immediately. Commit log is synced every
commitlog_sync_period_in_ms
. -
group - Send ACK signal for writes after the commit log has been flushed to disk. Wait up to
commitlog_sync_group_window_in_ms
between flushes. -
batch - Send ACK signal for writes after the commit log has been flushed to disk. Each incoming write triggers the flush task.
Default: periodic
-
- commitlog_sync_period_in_ms
-
Use with
commitlog_sync: periodic
. Time interval between syncing the commit log to disk. Periodic syncs are acknowledged immediately.Default: 10000
- commitlog_sync_group_window_in_ms
-
Use with
commitlog_sync: group
. The time that the database waits between flushing the commit log to disk. DataStax recommends usinggroup
instead ofbatch
.Default: 1000
- commitlog_sync_batch_window_in_ms
-
Deprecated. Use with
commitlog_sync: batch
. The maximum time that queries may be batched together.Default: commented out (
2
) - commitlog_segment_size_in_mb
-
The size of an individual commitlog file segment. A commitlog segment may be archived, deleted, or recycled after all its data has been flushed to SSTables. This data can potentially include commitlog segments from every table in the system. The default size is usually suitable, but for commitlog archiving you might want a finer granularity; 8 or 16 MB is reasonable.
Restriction: If you set max_mutation_size_in_kb explicitly, then you must set commitlog_segment_size_in_mb to:
2 * max_mutation_size_in_kb / 1024
The value must be positive and less than 2048.
Default: 32
- max_mutation_size_in_kb
-
The maximum size of a mutation before the mutation is rejected. Before increasing the commitlog segment size of the commitlog segments, investigate why the mutations are larger than expected. Look for underlying issues with access patterns and data model, because increasing the commitlog segment size is a limited fix.
Default: calculated (commitlog_segment_size_in_mb * 1024) / 2
- commitlog_total_space_in_mb
-
Disk usage threshold for commit logs before triggering the database flushing memtables to disk. If the total space used by all commit logs exceeds this threshold, the database flushes memtables to disk for the oldest commitlog segments to reclaim disk space by removing those log segments from the commit log. This flushing reduces the amount of data to replay on start-up, and prevents infrequently updated tables from keeping commitlog segments indefinitely. If the
commitlog_total_space_in_mb
is small, the result is more flush activity on less-active tables.Default for 64-bit JVMs: calculated (8192 or 25% of the total space of the commit log value, whichever is smaller)
Default for 32-bit JVMs: calculated (32 or 25% of the total space of the commit log value, whichever is smaller )
- commitlog_compression
-
The compressor to use if commit log is compressed. To make changes, uncomment the
commitlog_compression
section and make changes to these options:# commitlog_compression: # - class_name: LZ4Compressor # parameters: # -
-
class_name: LZ4Compressor, Snappy, or Deflate
-
parameters: optional parameters for the compressor
When not set, the default compression for the commit log is uncompressed.
Default: commented out
-
Lightweight transactions (LWT) settings
- concurrent_lw_transactions
-
Maximum number of permitted concurrent lightweight transactions (LWT).
-
A higher number might improve throughput if non-contending LWTs are in heavy use, but will use more memory and might be less successful with contention.
-
When not set, the default value is 8x the number of TPC cores. This default value is appropriate for most environments (8 x the number of TPC cores). Default: 128
-
- max_pending_lw_transactions
-
Maximum number of lightweight transactions (LWT) in the queue before node reports
OverloadedException
for LWTs.Default: 10000
Change-data-capture (CDC) space settings
See also cdc_raw_directory.
- cdc_enabled
-
Enables change data capture (CDC) functionality on a per-node basis. This modifies the logic used for write path allocation rejection.
-
true - use CDC functionality to reject mutations that contain a CDC-enabled table if at space limit threshold in
cdc_raw_directory
. -
false - standard behavior, never reject.
Default: false
-
- cdc_total_space_in_mb
-
Total space to use for change-data-capture (CDC) logs on disk. If space allocated for CDC exceeds this value, the database throws WriteTimeoutException on mutations, including CDC-enabled tables. A CDCCompactor (a consumer) is responsible for parsing the raw CDC logs and deleting them when parsing is completed.
Default: calculated (4096 or 1/8th of the total space of the drive where the cdc_raw_directory resides)
- cdc_free_space_check_interval_ms
-
Interval between checks for new available space for CDC-tracked tables when the cdc_total_space_in_mb threshold is reached and the CDCCompactor is running behind or experiencing back pressure.
Default: 250
Compaction settings
See also compaction_throughput_mb_per_sec in the common compaction settings section and Configuring compaction. |
- concurrent_compactors
-
The number of concurrent compaction processes allowed to run simultaneously on a node, not including validation compactions for anti-entropy repair. Simultaneous compactions help preserve read performance in a mixed read-write workload by limiting the number of small SSTables that accumulate during a single long-running compaction. If your data directories are backed by SSDs, increase this value to the number of cores. If compaction running too slowly or too fast, adjust compaction_throughput_mb_per_sec first.
Increasing concurrent compactors leads to more use of available disk space for compaction, because concurrent compactions happen in parallel, especially for STCS. Ensure that adequate disk space is available before increasing this configuration.
Generally, the calculated default value is appropriate and does not need adjusting. DataStax recommends contacting the DataStax Services team before changing this value.
Default: calculated (The fewest number of disks or number of cores, with a minimum of 2 and a maximum of 8 per CPU core.)
- concurrent_validations
-
Configures simultaneous repair validations.
-
values greater than 1 - Number of simultaneous repair validations to allow.
-
0 - Unbounded.
Default: 0
-
- concurrent_materialized_view_builders
-
Number of simultaneous materialized view builder tasks allowed to run concurrently. When a view is created, the node ranges are split into (num_processors x 4) builder tasks and submitted to this executor.
Default: 2
- sstable_preemptive_open_interval_in_mb
-
The size of the SSTables to trigger preemptive opens. The compaction process opens SSTables before they are completely written and uses them in place of the prior SSTables for any range previously written. This process helps to smoothly transfer reads between the SSTables by reducing cache churn and keeps hot rows hot.
A low value has a negative performance impact and will eventually cause heap pressure and GC activity. The optimal value depends on hardware and workload.
Default: 50
- pick_level_on_streaming
-
The compaction level for streamed-in SSTables.
-
true - Streamed-in SSTables of tables using LeveledCompactionStrategy (LCS) are placed on the same level as the source node. For operational tasks like nodetool refresh or replacing a node, true improves performance for compaction work.
-
false - Streamed-in SSTables are placed in level 0.
Default: true
-
Cache and index settings
- column_index_size_in_kb
-
Granularity of the index of rows within a partition. For huge rows, decrease this setting to improve seek time. Lower density nodes might benefit from decreasing this value to 4, 2, or 1.
Default: 16
- file_cache_size_in_mb
-
Buffer pool is split into two pools, this setting defines the maximum memory to use file buffers that are stored in the file cache, also known as chunk cache. Memory is allocated only when needed but is not released. The other buffer pool is direct_reads_size_in_mb.
Default: calculated (0.5 of -XX:MaxDirectMemorySize)
- direct_reads_size_in_mb
-
Buffer pool is split into two pools; this setting defines the buffer pool for transient read operations. A buffer is typically used by a read operation and then returned to this pool when the operation is finished so that it can be reused by other operations. The other buffer pool is file_cache_size_in_mb.
When not set, before DSE 6.8.2, the default
direct_reads_size_in_mb
value is calculated as 2 MB per TPC core thread, plus 2 MB shared by non-TPC threads, with a maximum value of 128 MB. In DSE 6.8.2 and later, the default is calculated as 4 MB per TPC core thread, plus 4 MB shared by non-TPC threads, with a maximum value of 128 MB.Default: calculated
Streaming settings
- input_stream_channel_timeout_in_ms
-
Max amount of time to wait before the inbound channel times out. Set this value to Long.MAX_VALUE when streaming large data, which effectively disables this timeout. Configure this value to a lower value when factors such as SSTable size and network latency change.
Default: streaming_keep_alive_period_in_secs * 1000 * 2 milliseconds
- inter_dc_stream_throughput_outbound_megabits_per_sec
-
Throttle for all streaming file transfers between datacenters, and for network stream traffic as configured with stream_throughput_outbound_megabits_per_sec. When not set, the value is 200 Mbps.
Should be set to a value less than or equal to stream_throughput_outbound_megabits_per_sec since it is a subset of total throughput.
Default: 200 Mbps
- streaming_keep_alive_period_in_secs
-
Interval to send keep-alive messages to prevent reset connections during streaming. The stream session fails when a keep-alive message is not received for two keep-alive cycles. A stalled stream times out in ten minutes.
Default: 300 seconds
- stream_max_outbound_buffers_in_kb
-
Maximum amount of pending data written before pausing outbound streaming. This value is shared among all outbound streaming session to cap the overall memory used by all streaming processes, such as bootstrap and repair.
Default: 131072 KB
- stream_outbound_buffer_in_kb
-
Buffer size for stream writes. Each outbound streaming session buffers writes according to this size.
Default: 1024 KB
- stream_outbound_permits_in_mb
-
Maximum size for the outbound streaming buffer when sending data to another node. For large data, set this value to 8 MB to reduce the probability of a timeout. A reduced value allows data to stream faster than waiting for a 128 MB buffer to be reached.
Default: 128 MB
- stream_throughput_outbound_megabits_per_sec
-
Throttle for the throughput of all outbound streaming file transfers on a node. When streaming data during bootstrap or repair, the database does mostly sequential I/O, which can saturate the network connection and degrade client (RPC) performance.
Default: 200 MB/second
- zerocopy_streaming_enabled
-
Enable zero-copy streaming of SSTables and their components. For each SSTable to stream, only the required ranges are actually streamed as separate SSTables, while the SSTable metadata is streamed in its entirety and linked to every SSTable produced on the destination node, avoiding the costly rebuilding of such metadata at the expense of additional disk usage (see zerocopy_max_unused_metadata_in_mb). All SSTables and their components are also copied via zero-copy operations, greatly reducing GC pressure and improving overall speed.
Default: true
- zerocopy_max_unused_metadata_in_mb
-
Number of megabytes per SSTable of excess metadata allowed in order to use zero-copy rather than legacy streaming.
Default: 200 MB
- zerocopy_max_sstables
-
Maximum number of SSTables a single SStable can be split into when using zero-copy rather than legacy streaming.
Default: 256
- stream_outbound_buffer_in_kb
-
Buffer size for stream writes. Each outbound streaming session will buffer writes according to this size.
Default: 1024
- stream_max_outbound_buffers_in_kb
-
Max amount of pending data written before pausing outbound streaming. This value is shared among all outbound streaming session, in order to cap the overall memory used by all streaming processes (bootstrap, repair, etc.).
Default: 131072
Fsync settings
- trickle_fsync
-
When set to true, causes fsync to force the operating system to flush the dirty buffers at the set interval trickle_fsync_interval_in_kb. Enable this parameter to prevent sudden dirty buffer flushing from impacting read latencies. Recommended for use with SSDs, but not with HDDs.
Default: false
- trickle_fsync_interval_in_kb
-
The size of the fsync in kilobytes.
Default: 10240
- max_value_size_in_mb
-
The maximum size of any value in SSTables. SSTables are marked as corrupted when the threshold is exceeded.
Default: 256
Thread Per Core (TPC) parameters
#tpc_cores:
# tpc_io_cores:
io_global_queue_depth: 128
- tpc_cores
-
The number of concurrent
CoreThreads
. TheCoreThreads
are the main workers in a DSE 6.x node, and process various asynchronous tasks from their queue. If not set, the default is the number of cores (processors on the machine) minus one. Note that configuringtpc_cores
affects the default value fortpc_io_cores
.To achieve optimal throughput and latency, for a given workload, set
tpc_cores
to half the number of CPUs (minimum) to double the number of CPUs (maximum). In cases where there are a large number of incoming client connections, increasingtpc_cores
to more than the default usually results inCoreThreads
receiving more CPU time.DSE Search workloads only: set
tpc_cores
to the number of physical CPUs. See Tuning search for maximum indexing throughput.Default: commented out; defaults to the number of cores minus one.
- tpc_io_cores
-
The subset of
tpc_cores
that process asynchronous IO tasks. (That is, disk reads.) Must be smaller or equal totpc_cores
. Lower this value to decrease parallel disk IO requests.Default: commented out; by default, calculated as
min(io_global_queue_depth/4, tpc_cores)
- io_global_queue_depth
-
Global IO queue depth used for reads when AIO is enabled, which is the default for SSDs. The optimal queue depth as found with the
fio
tool for a given disk setup.Default:
128
NodeSync parameters
By default, the NodeSync service runs on every node.
Manage the NodeSync service using the Manage NodeSync service using nodetool subcommands command. |
- rate_in_kb
-
The maximum bytes per second for data validation on the local node. The optimum validation rate for each node may vary.
Default: 1024
Advanced properties
Properties for advanced users or properties that are less commonly used.
Advanced initialization properties
- auto_bootstrap
-
This setting has been removed from default configuration.
-
true - causes new (non-seed) nodes migrate the right data to themselves automatically
-
false - When initializing a fresh cluster without data
Default: true
-
- broadcast_address
-
The public IP address this node uses to broadcast to other nodes outside the network or across regions in multiple-region EC2 deployments. If this property is commented out, the node uses the same IP address or hostname as listen_address. A node does not need a separate broadcast_address in a single-node or single-datacenter installation, or in an EC2-based network that supports automatic switching between private and public communication. It is necessary to set a separate listen_address and broadcast_address on a node with multiple physical network interfaces or other topologies where not all nodes have access to other nodes by their private IP addresses. For specific configurations, see the instructions for listen_address.
Default: listen_address
- listen_on_broadcast_address
-
Enables the node to communicate on both interfaces.
-
true - If this node uses multiple physical network interfaces, set a unique IP address for broadcast_address
-
false - If this node is on a network that automatically routes between public and private networks, like Amazon EC2 does
See listen_address.
Default: false
-
- initial_token
-
Enables single-node-per-token architecture, in which a node owns exactly one contiguous range in the ring space. Overrides num_tokens.
-
token - The initial token to start the contiguous range.
-
1 - Disable single-node-per-token architecture.
-
-
If your installation is not using vnodes
Use this parameter only with num_tokens (vnodes ) in special cases such as Restoring from a snapshot.
Default: 1
- num_tokens
-
Define virtual node (vnode) token architecture.
All other nodes in the datacenter must have the same token architecture.
-
1 - Disable vnodes and use 1 token for legacy compatibility. You must set an
initial_token
value when setting up a production cluster for the first time, and when adding capacity. See Generating tokens. -
a number between 2 and 128 - the number of token ranges to assign to this virtual node (vnode). A higher value increases the probability that the data and workload are evenly distributed.
Restriction: DataStax recommends not using vnodes with DSE Search. However, if you decide to use vnodes with DSE Search, do not use more than 8 vnodes and ensure that allocate_tokens_for_local_replication_factor option in cassandra.yaml is correctly configured for your environment.
-
Using vnodes can impact performance for your cluster. DataStax recommends testing the configuration before enabling vnodes in production environments. |
When the token number varies between nodes in a datacenter, the vnode logic assigns a proportional number of ranges relative to other nodes in the datacenter.
In general, if all nodes have equal hardware capability, each node should have the same num_tokens
value.
-
To migrate an existing cluster from single node per token range to vnodes, see Enabling virtual nodes on an existing production cluster.
Default: 1
- allocate_tokens_for_local_replication_factor
-
-
RF of keyspaces in datacenter - triggers the recommended algorithmic allocation for the RF and num_tokens for this node.
The allocation algorithm optimizes the workload balance using the target keyspace replication factor. DataStax recommends setting the number of tokens to 8 to distribute the workload with ~10% variance between nodes. The allocation algorithm attempts to choose tokens in a way that optimizes replicated load over the nodes in the datacenter for the specified RF. The load assigned to each node is close to proportional to the number of vnodes.
The allocation algorithm is supported only for the Murmur3Partitioner and RandomPartitioner partitioners. The Murmur3Partitioner is the default partitioning strategy for new clusters and the right choice for new clusters in almost all cases.
-
commented out - uses the random selection algorithm to assign token ranges randomly.
Over time, loads in a datacenter using the random selection algorithm become unevenly distributed. DataStax recommends using only the allocation algorithm.
-
-
Default: use random selection algorithm
See Virtual node (vnode) configuration, and for set up instructions see Adding nodes to vnode-enabled cluster or Adding a datacenter to a cluster using a designated datacenter as a data source.
- partitioner
-
The class that distributes rows (by partition key) across all nodes in the cluster. Any IPartitioner may be used, including your own as long as it is in the class path. For new clusters use the default partitioner.
DataStax Enterprise provides the following partitioners for backward compatibility:
-
RandomPartitioner
-
ByteOrderedPartitioner (deprecated)
-
OrderPreservingPartitioner (deprecated)
Use only partitioner implementations bundled with DSE.
See Partitioners.
Default: org.apache.cassandra.dht.Murmur3Partitioner
-
- tracetype_query_ttl
-
TTL for different trace types used during logging of the query process.
Default: 86400
- tracetype_repair_ttl
-
TTL for different trace types used during logging of the repair process.
Default: 604800
Advanced automatic backup setting
- auto_snapshot
-
Enables snapshots of the data before truncating a keyspace or dropping a table. To prevent data loss, DataStax strongly advises using the default setting. If you set auto_snapshot to false, you lose data on truncation or drop.
Default: true
Global row properties
When creating or modifying tables, you can enable or disable the row cache for that table by setting the caching parameter. Other row cache tuning and configuration options are set at the global (node) level. The database uses these settings to automatically distribute memory for each table on the node based on the overall workload and specific table usage. You can also configure the save periods for these caches globally.
See Configuring caches. |
- column_index_cache_size_in_kb
-
Applies only to BIG format SSTables, the threshold for the total size of all index entries for a partition that the database stores in the partition key cache. If the total size of all index entries for a partition exceeds this amount, the database stops putting entries for this partition into the partition key cache.
Default: 2
- row_cache_class_name
-
The classname of the row cache provider to use.
-
org.apache.cassandra.cache.OHCProvider - Fully off-heap.
-
org.apache.cassandra.cache.SerializingCacheProvider - Partially off-heap, available in earlier releases.
Use only row cache provider implementations bundled with DSE.
Default: org.apache.cassandra.cache.OHCProvider
-
- row_cache_size_in_mb
-
Maximum size of the row cache in memory. The row cache can save time, but it is space-intensive because it contains the entire row. Use the row cache only for hot rows or static rows. If you reduce the size, you may not get you hottest keys loaded on start up.
-
0 - Disable row caching
-
MB - Maximum size of the row cache in memory
Default: 0
-
- row_cache_save_period
-
The number of seconds that rows are kept in cache. Caches are saved to saved_caches_directory. This setting has limited use as described in row_cache_size_in_mb.
Default: 0
- row_cache_keys_to_save
-
The number of keys from the row cache to save. All keys are saved.
Default: 100
Counter caches properties
Counter cache helps to reduce counter locks' contention for hot counter cells. In case of RF = 1 a counter cache hit causes the database to skip the read before write entirely. With RF > 1 a counter cache hit still helps to reduce the duration of the lock hold, helping with hot counter cell updates, but does not allow skipping the read entirely. Only the local (clock, count) tuple of a counter cell is kept in memory, not the whole counter, so it is relatively cheap.
If you reduce the counter cache size, the database may load the hottest keys start-up. |
- counter_cache_size_in_mb
-
The size of counter cache.
-
size - The smaller of minimum of 2.5% of heap or 50 megabytes (MB).
-
0 - Disable. If your system performs counter deletes and relies on low gc_grace_seconds, you should disable the counter cache.
Default: calculated
-
- counter_cache_save_period
-
The time, in seconds, after which the database saves the counter cache (keys only). The database saves caches to saved_caches_directory.
Default: 7200 (2 hours)
- counter_cache_keys_to_save
-
The number of keys from the counter cache to save.
Default: save all keys
Tombstone settings
When executing a scan, within or across a partition, the database must keep tombstones in memory to allow them to return to the coordinator. The coordinator uses tombstones to ensure that other replicas know about the deleted rows. Workloads that generate numerous tombstones may cause performance problems and exhaust the server heap. Adjust these thresholds only if you understand the impact and want to scan more tombstones. You can adjust these thresholds at runtime using the StorageServiceMBean.
Because tombstone warning and failure thresholds are configured via |
See the DataStax Developer Blog post Cassandra anti-patterns: Queues and queue-like datasets. |
Network timeout settings
- read_request_timeout_in_ms
-
How long the coordinator waits for read operations to complete before timing it out.
Default: 5000 (5 seconds)
- range_request_timeout_in_ms
-
How long the coordinator waits for sequential or index scans to complete before timing it out.
Default: 10000 (10 seconds)
- aggregated_request_timeout_in_ms
-
How long the coordinator waits for sequential or index scans to complete. Lowest acceptable value is 10 ms. This timeout does not apply to aggregated queries such as SELECT, COUNT(*), MIN(x), and so on.
Default: 120000 (2 minutes)
- write_request_timeout_in_ms
-
How long the coordinator waits for write requests to complete with at least one node in the local datacenter. Lowest acceptable value is 10 ms.
Default: 2000 (2 seconds)
- counter_write_request_timeout_in_ms
-
How long the coordinator waits for counter writes to complete before timing it out.
Default: 5000 (5 seconds)
- cas_contention_timeout_in_ms
-
How long the coordinator continues to retry a CAS (compare and set) operation that contends with other proposals for the same row. If the coordinator cannot complete the operation within this timespan, it aborts the operation.
Default: 1000 (1 second)
- truncate_request_timeout_in_ms
-
How long the coordinator waits for a truncate (the removal of all data from a table) to complete before timing it out. The long default value allows the database to take a snapshot before removing the data. If auto_snapshot is disabled (not recommended), you can reduce this time.
Default: 60000 (1 minute)
- request_timeout_in_ms
-
The default timeout value for other miscellaneous operations. Lowest acceptable value is 10 ms.
Default: 10000
- cross_dc_rtt_in_ms
-
How much to increase the cross-datacenter timeout (
write_request_timeout_in_ms + cross_dc_rtt_in_ms
) for requests that involve only nodes in a remote datacenter. This setting is intended to reduce hint pressure.DataStax recommends using
LOCAL_*
consistency levels (CL) for read and write requests in multi-datacenter deployments to avoid timeouts that may occur when remote nodes are chosen to satisfy the CL, such as QUORUM.Default: 0
- slow_query_log_timeout_in_ms
-
The time of a slow query. Queries that exceed this value generate an aggregated log message to identify slow queries.
-
milliseconds - How long before a node logs slow queries.
-
0 - Disable slow query log timeout.
Default: 500
-
Internode settings
- storage_port
-
The port for internode communication. Follow security best practices, do not expose this port to the internet. Apply firewall rules.
Default: 7000
- cross_node_timeout
-
Enables operation timeout information exchange between nodes to accurately measure request timeouts.
-
true - Enable operation timeout information exchange between nodes.
-
false - The replica assumes any requests are forwarded to it instantly by the coordinator. During overload conditions this means extra time is required for processing already-timed-out requests.
Before enabling this property make sure NTP (network time protocol) is installed and the times are synchronized among the nodes.
Default: false
-
- internode_send_buff_size_in_bytes
-
The sending socket buffer size, in bytes, for internode calls.
See TCP settings.
The sending socket buffer size and internode_recv_buff_size_in_bytes is limited by
net.core.wmem_max
. For more details runman tcp
and refer to:-
/proc/sys/net/core/wmem_max
-
/proc/sys/net/core/rmem_max
-
/proc/sys/net/ipv4/tcp_wmem
-
/proc/sys/net/ipv4/tcp_wmem
Default: determined by
net.ipv4.tcp_wmem
-
- internode_recv_buff_size_in_bytes
-
The receiving socket buffer size in bytes for internode calls.
Default: not set
- internode_compression
-
Enables compressing traffic between nodes
-
all - Compress all traffic.
-
dc - Compress traffic between datacenters only.
-
none - No compression.
Default: dc
-
- inter_dc_tcp_nodelay
-
Enables tcp_nodelay for inter-datacenter communication.
-
true - Enable tcp_nodelay for inter-datacenter communication. Reduce the number of packets that need to be sent over the network.
-
false - The network sends larger, but fewer, network packets. This reduces overhead from the TCP protocol itself. May increase latency by blocking cross datacenter responses.
Default: false
-
Native transport (CQL Binary Protocol)
See also native_transport_port_ssl in SSL Ports. |
- start_native_transport
-
Enables or disables the native transport server.
Default: true
- native_transport_port
-
The port where the CQL native transport listens for clients. For security reasons, do not expose this port to the internet. Firewall it if needed.
Default: 9042
- native_transport_max_frame_size_in_mb
-
The maximum allowed size of a frame. Frame (requests) larger than this are rejected as invalid.
Default: 256
- native_transport_max_concurrent_connections
-
The maximum number of concurrent client connections.
Default: -1 (unlimited)
- native_transport_max_concurrent_connections_per_ip
-
The maximum number of concurrent client connections per source IP address.
Default: -1 (unlimited)
- native_transport_allow_older_protocols
-
Honor older protocol versions.
Default: true
- native_transport_address
-
When left blank, uses the configured hostname of the node. Unlike the
listen_address
, this value can be set to 0.0.0.0, but you must set the native_transport_broadcast_address to a value other than 0.0.0.0.Set native_transport_address OR native_transport_interface, not both.
Default: localhost
- native_transport_interface
-
IP aliasing is not supported.
Set native_transport_address OR native_transport_interface, not both.
Default: eth0
- native_transport_interface_prefer_ipv6
-
Use IPv4 or IPv6 when interface is specified by name.
-
false - use first IPv4 address.
-
true - use first IPv6 address.
-
-
When only a single address is used, that address is selected without regard to this setting.
Default: false
- native_transport_broadcast_address
-
Native transport address to broadcast to drivers and other DSE nodes. This cannot be set to 0.0.0.0.
-
blank - will be set to the value of native_transport_address
-
<IP_address> - when native_transport_address is set to 0.0.0.0
Default:1.2.3.4
-
- native_transport_keepalive
-
Enables keepalive on native connections.
Default: true
Advanced fault detection settings
Settings to handle poorly performing or failing components.
- gc_log_threshold_in_ms
-
The threshold for log messages at the INFO level. Adjust to minimize logging.
Default: 200
- gc_warn_threshold_in_ms
-
Threshold for GC pause. Any GC pause longer than this interval is logged at the WARN level. By default, the database logs any GC pause greater than 200 ms at the INFO level.
See Configuring logging.
Default: 1000
- otc_coalescing_strategy
-
Strategy to combine multiple network messages into a single packet for outbound TCP connections to nodes in the same data center. See the DataStax Developer Blog post Performance doubling with message coalescing.
Use only strategy implementations bundled with DSE.
Supported strategies are:
-
FIXED
-
MOVINGAVERAGE
-
TIMEHORIZON
-
DISABLED Default: DISABLED
-
- otc_coalescing_window_us
-
How many microseconds to wait for coalescing messages to nodes in the same datacenter.
-
For FIXED strategy - the amount of time after the first message is received before it is sent with any accompanying messages.
-
For MOVING average - the maximum wait time and the interval that messages must arrive on average to enable coalescing.
Default: 100
-
- otc_coalescing_enough_coalesced_messages
-
The threshold for the number of messages to nodes in the same data center. Do not coalesce messages when this value is exceeded. Should be more than 2 and less than 128.
Default: 32
- otc_large_max_message_buffer_kb
-
Size in KB of the direct buffer used to write messages to large outbound internode connections (one for every node in the cluster per connection type). Messages larger than this buffer will require the allocation of a new buffer, so it should be large enough to accommodate the average message size (and possibly more, to allow for batch flushing), but not too large, to avoid running out of memory.
Default: 1024
- otc_small_max_message_buffer_kb
-
Size in KB of the direct buffer used to write messages to small outbound internode connections (one for every node in the cluster per connection type).
Default: 64
- seed_gossip_probability
-
The percentage of time that gossip messages are sent to a seed node during each round of gossip. Decreases the time to propagate gossip changes across the cluster.
Default: 1.0 (100%)
Backpressure settings
- back_pressure_enabled
-
Enables for the coordinator to apply the specified back pressure strategy to each mutation that is sent to replicas.
Default: false
- back_pressure_strategy
-
To add new strategies, implement org.apache.cassandra.net.BackpressureStrategy and provide a public constructor that accepts a
Map<String, Object>
.Use only strategy implementations bundled with DSE.
- class_name
-
The default class_name uses the ratio between incoming mutation responses and outgoing mutation requests.
Default: org.apache.cassandra.net.RateBasedBackPressur
- high_ratio
-
When outgoing mutations are below this value, they are rate limited according to the incoming rate decreased by the factor (described below). When above this value, the rate limiting is increased by the factor.
Default: 0.90
- factor
-
A number between 1 and 10. When backpressure is below high ratio, outgoing mutations are rate limited according to the incoming rate decreased by the given factor; if above high ratio, the rate limiting is increased by the given factor.
Default: 5
- flow
-
The flow speed to apply rate limiting:
-
FAST - Rate limit to the speed of the fastest replica.
-
SLOW - Rate limit to the speed of the slowest replica.
Default: FAST
-
- dynamic_snitch_badness_threshold
-
The performance threshold for dynamically routing client requests away from a poorly performing node. Specifically, it controls how much worse a poorly performing node has to be before the dynamic snitch prefers other replicas. A value of 0.2 means the database continues to prefer the static snitch values until the node response time is 20% worse than the best performing node. Until the threshold is reached, incoming requests are statically routed to the closest replica as determined by the snitch.
Default: 0.1
- dynamic_snitch_reset_interval_in_ms
-
Time interval after which the database resets all node scores to allow a bad node to recover.
Default: 600000
- dynamic_snitch_update_interval_in_ms
-
The time interval, in milliseconds, between the calculation of node scores. Because score calculation is CPU intensive, be careful when reducing this interval.
Default: 100
Hinted handoff options
- hinted_handoff_enabled
-
Enables hinted handoff. A hint indicates that the write needs to be replayed to an unavailable node. The database writes the hint to a hints file on the coordinator node.
-
true - Globally enable hinted handoff, except for datacenters specified for hinted_handoff_disabled_datacenters.
-
false - Do not enable hinted handoff.
Default: true
-
- hinted_handoff_disabled_datacenters
-
A blacklist of datacenters that will not perform hinted handoffs. To disable hinted handoff on a certain datacenter, add its name to this list.
Default: no datacenters are blacklisted
- max_hint_window_in_ms
-
Maximum amount of time during which the database generates hints for an unresponsive node. After this interval, the database does not generate any new hints for the node until it is back up and responsive. If the node goes down again, the database starts a new interval. This setting can prevent a sudden demand for resources when a node is brought back online and the rest of the cluster attempts to replay a large volume of hinted writes.
Default: 10800000 (3 hours)
- hinted_handoff_throttle_in_kb
-
Maximum amount of traffic per delivery thread in kilobytes per second. This rate reduces proportionally to the number of nodes in the cluster. For example, if there are two nodes in the cluster, each delivery thread uses. the maximum rate. If there are three, each node throttles to half of the maximum, since the two nodes are expected to deliver hints simultaneously.
When applying this limit, the calculated hint transmission rate is based on the uncompressed hint size, even if internode_compression or hints_compression is enabled.
Default: 1024
- hints_flush_period_in_ms
-
The time, in milliseconds, to wait before flushing hints from internal buffers to disk.
Default: 10000
- max_hints_delivery_threads
-
Number of threads the database uses to deliver hints. In multiple datacenter deployments, consider increasing this number because cross datacenter handoff is generally slower.
Default: 2
- max_hints_file_size_in_mb
-
The maximum size for a single hints file, in megabytes.
Default: 128
- hints_compression
-
The compressor for hint files. Supported compressors: LZ, Snappy, and Deflate. When not set, the database does not compress hints files.
Default: LZ4Compressor
- batchlog_replay_throttle_in_kb
-
Total maximum throttle, in KB per second, for replaying hints. Throttling is reduced proportionally to the number of nodes in the cluster
Default: 1024
- batchlog_endpoint_strategy
-
Strategy to choose the batchlog storage endpoints.
-
random_remote - Default, purely random. Prevents the local rack, if possible. Same behavior as earlier releases.
-
dynamic_remote - Uses DynamicEndpointSnitch to select batchlog storage endpoints. Prevents the local rack, if possible. This strategy offers the same availability guarantees as random_remote, but selects the fastest endpoints according to the DynamicEndpointSnitch.DynamicEndpointSnitch tracks reads but not writes. Write-only, or mostly-write, workloads might not benefit from this strategy.
This strategy will fall back to random_remote if dynamic_snitch is not enabled.
-
dynamic - Mostly the same as dynamic_remote, except that local rack is not excluded, which offers lower availability guarantee than random_remote or dynamic_remote.
This strategy will fall back to random_remote if dynamic_snitch is not enabled.
Default: random_remote
-
Security properties
DSE Advanced Security fortifies DataStax Enterprise (DSE) databases against potential harm due to deliberate attack or user error.
Configuration properties include authentication and authorization, permissions, roles, encryption of data in-flight and at-rest, and data auditing.
DSE Unified Authentication provides authentication, authorization, and role management.
Enabling DSE Unified Authentication requires additional configuration in dse.yaml
, see Configuring DSE Unified Authentication.
- authenticator
-
The authentication backend. The only supported authenticator is DseAuthenticator for external authentication with multiple authentication schemes such as Kerberos, LDAP, and internal authentication. Authenticators other than DseAuthenticator are deprecated and not supported. Some security features might not work correctly if other authenticators are used. See authentication_options in
dse.yaml
.Use only authentication implementations bundled with DSE.
Default: com.datastax.bdp.cassandra.auth.DseAuthenticator
- internode_authenticator
-
Internode authentication backend to enable secure connections from peer nodes.
Use only authentication implementations bundled with DSE.
Default: org.apache.cassandra.auth.AllowAllInternodeAuthenticator
- authorizer
-
The authorization backend. Authorizers other than DseAuthorizer are not supported. DseAuthorizer supports enhanced permission management of DSE-specific resources. Authorizers other than DseAuthorizer are deprecated and not supported. Some security features might not work correctly if other authorizers are used. See dse-yaml.adoc#authorizationOptions in
dse.yaml
.Use only authorization implementations bundled with DSE.
Default: com.datastax.bdp.cassandra.auth.DseAuthorizer
- system_keyspaces_filtering
-
Enables system keyspace filtering so that users can access and view only schema information for rows in the
system
andsystem_schema
keyspaces to which they have access.-
true - Enable system keyspace filtering.
-
Data in the
system.local
andsystem.peers
tables are visible. -
Data in the following tables of the
system
keyspace are filtered based on the role’sDESCRIBE
privileges for keyspaces; only rows for appropriate keyspaces will be displayed in:-
size_estimates
-
sstable_activity
-
built_indexes
-
built_views
-
available_ranges
-
view_builds_in_progress
-
-
Data in all tables in the
system_schema
keyspace are filtered based on a role’sDESCRIBE
privilege for keyspaces stored in thesystem_schema
tables. -
Read operations against other tables in the
system
keyspace are denied.
-
-
false - Disable system keyspace filtering.
-
-
Attention: Security requirements and user permissions apply. Enable this feature only after appropriate user permissions are granted. You must grant the
DESCRIBE
permission to the role on any keyspaces stored in the system keyspaces. If you do not grant the permission, you will see an error that states the keyspace is not found.GRANT DESCRIBE ON KEYSPACE <keyspace_name> TO ROLE <role_name>;
Default: false
- role_manager
-
The DSE Role Manager supports LDAP roles and internal roles supported by the CassandraRoleManager. Role options are stored in the dse_security keyspace. When using the DSE Role Manager, increase the replication factor of the dse_security keyspace. Role managers other than DseRoleManager are deprecated and not supported. Some security features might not work correctly if other role managers are used.
Use only role manager implementations bundled with DSE.
Default: com.datastax.bdp.cassandra.auth.DseRoleManager
- roles_validity_in_ms
-
Validity period for roles cache in milliseconds. Determines how long to cache the list of roles assigned to the user; users may have several roles, either through direct assignment or inheritance (a role that has been granted to another role). Adjust this setting based on the complexity of your role hierarchy, tolerance for role changes, the number of nodes in your environment, and activity level of the cluster.
Fetching permissions can be an expensive operation, so this setting allows flexibility. Granted roles are cached for authenticated sessions in
AuthenticatedUser
. After the specified time elapses, role validity is rechecked. Disabled automatically when internal authentication is not enabled when using DseAuthenticator.-
0 - Disable role caching
-
milliseconds - how long to cache the list of roles assigned to the user
Default: 120000 (2 minutes)
-
- roles_update_interval_in_ms
-
Refresh interval for roles cache. After this interval, cache entries become eligible for refresh. On next access, the database schedules an async reload, and returns the old value until the reload completes. If
roles_validity_in_ms
is non-zero, then this value must also be non-zero. When not set, the default is the same value as roles_validity_in_ms.Default: 120000
- permissions_validity_in_ms
-
How long permissions in cache remain valid to manage performance impact of permissions queries. Fetching permissions can be resource intensive. Set the cache validity period to your security tolerances. The cache is used for the standard authentication and the row-level access control (RLAC) cache. The cache is quite effective at small durations.
-
0 - Disable permissions cache.
-
<milliseconds> - Time that permissions in cache remain valid.
REVOKE does not automatically invalidate cached permissions. Permissions are invalidated the next time they are refreshed.
Default: 120000 (2 minutes)
-
- permissions_update_interval_in_ms
-
Refresh interval for the standard authentication cache and the row-level access control (RLAC) cache. After this interval, cache entries become eligible for refresh. On next access, the database schedules an async reload and returns the old value until the reload completes. If permissions_validity_in_ms is non-zero, the value for roles_update_interval_in_ms must also be non-zero. When not set, the default is the same value as permissions_validity_in_ms.
Default: 2000
- permissions_cache_max_entries
-
The maximum number of entries that are held by the standard authentication cache and row-level access control (RLAC) cache. With the default value of 1000, the RLAC permissions cache can have up to 1000 entries in it, and the standard authentication cache can have up to 1000 entries. This single option applies to both caches. To size the permissions cache for use with Setting up Row Level Access Control (RLAC), use this formula:
<numRlacUsers> * <numRlacTables> + 100
If this option is not present in cassandra.yaml, manually enter it to use a value other than 1000. See Enabling DSE Unified Authentication.
Default: 1000
Internode encryption options
Node-to-node (internode) encryption protects data that is transferred between nodes in a cluster using SSL.
- server_encryption_options
-
Internode encryption options. If enabled, you must also generate keys and provide the appropriate key and truststore locations and passwords. No custom encryption options are supported.
-
The passwords used in these options must match the passwords used when generating the keystore and truststore. See Creating a Keystore to Use with JSSE. Also refer to Configuring SSL for node-to-node connections.
-
For
PKCS11
caveats, see keystore_type.
-
- internode_encryption
-
Encryption options for of internode communication using the TLS_RSA_WITH_AES_128_CBC_SHA cipher suite for authentication, key exchange, and encryption of data transfers. Use the DHE/ECDHE ciphers, such as TLS_DHE_RSA_WITH_AES_128_CBC_SHA if running in Federal Information Processing Standard (FIPS) 140 compliant mode.
-
all - Encrypt all internode communications.
-
none - No encryption.
-
dc - Encrypt the traffic between the datacenters.
-
rack - Encrypt the traffic between the racks.
Default: none
-
- keystore
-
Relative path from DSE installation directory or absolute path to the Java keystore (JKS) suitable for use with Java Secure Socket Extension (JSSE), which is the Java version of the Secure Sockets Layer (SSL), and Transport Layer Security (TLS) protocols. The keystore contains the private key used to encrypt outgoing messages.
Default: resources/dse/conf/.keystore
- keystore_password
-
Password for the keystore. This must match the password used when generating the keystore and truststore.
Default: cassandra
- truststore
-
Relative path from DSE installation directory or absolute path to truststore containing the trusted certificate for authenticating remote servers.
Default: resources/dse/conf/.truststore
- truststore_password
-
Password for the truststore.
Default: cassandra
- protocol
-
Default: TLS
- algorithm
-
Default: SunX509
- keystore_type
-
Valid types are JKS, JCEKS, PKCS11, or PKCS12. For file-based keystores, use PKCS12.
DataStax supports PKCS11 as a
keystore_type
on nodes withcassandra
oradvanced
workloads. Theadvanced
workload support was added for DSE 6.8.2 and later. IfPKCS11
is needed, inserver_encryption_options
orclient_encryption_options
, specify thekeystore_type
asPKCS11
and thekeystore
asNONE
.PKCS11 is not supported as a
truststore_type.
Default: JKS
- truststore_type
-
Valid types are JKS, JCEKS, PKCS12. For file-based truststores, use PKCS12.
Due to an OpenSSL issue, you cannot use a PKCS12 truststore that was generated via OpenSSL. For example, a truststore generated via the following command will not work with DSE:
openssl pkcs12 -export -nokeys -out truststore.pfx -in <intermediate.chain.pem>
However, truststores generated via Java’s
keytool
and then converted to PKCS12 work with DSE.Example:
keytool -importcert -alias <rootca> -file <rootca.pem> -keystore <truststore.jks>
keytool -importcert -alias <intermediate> -file <intermediate.pem> -keystore <truststore.jks>
keytool -importkeystore -srckeystore <truststore.jks> -destkeystore <truststore.pfx> -deststoretype pkcs12
Default: JKS
- cipher_suites
-
Supported ciphers:
-
TLS_RSA_WITH_AES_128_CBC_SHA
-
TLS_RSA_WITH_AES_256_CBC_SHA
-
TLS_DHE_RSA_WITH_AES_256_CBC_SHA
-
TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA
-
TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA
Default: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
-
- require_client_auth
-
Enables certificate authentication for node-to-node (internode) encryption.
Default: false
- require_endpoint_verification
-
Whether to verify the connected host and the host IP address in the certificate match. If set to
true
, the endpoint that you specify when generating the certificate key must be an IP address. Do not specify a DNS hostname. Example with a correctly specified IP address:keytool -genkeypair -keyalg RSA \ -alias node0 \ -keystore my_keystore.jks \ -storepass cassandra \ -keypass cassandra \ -validity 730 \ -keysize 2048 \ -dname "CN=node0, OU=lacerda-ssl, O=Datastax, C=CC" \ -ext "san=ip:10.101.35.236"
Default: false
Client-to-node encryption options
Client-to-node encryption protects in-flight data from client machines to a database cluster using SSL (Secure Sockets Layer) and establishes a secure channel between the client and the coordinator node.
- client_encryption_options
-
Enables client-to-node encryption. You must also generate keys and provide the appropriate key and truststore locations and passwords. There are no custom encryption options enabled for DataStax Enterprise.
For
PKCS11
caveats, see keystore_type.
- enabled
-
Enables client-to-node encryption.
Default: false
- optional
-
When
optional
is selected, both encrypted and unencrypted connections over native transport are allowed. That is a necessary transition state to facilitate enabling client to node encryption on live clusters without inducing an outage for existing unencrypted clients. Typically, once existing clients are migrated to encrypted connections,optional
is unselected in order to enforce native transport encryption.Default:
false
- keystore
-
Relative path from DSE installation directory or absolute path to the Java keystore (JKS) suitable for use with Java Secure Socket Extension (JSSE), which is the Java version of the Secure Sockets Layer (SSL), and Transport Layer Security (TLS) protocols. The keystore contains the private key used to encrypt outgoing messages.
Default: resources/dse/conf/.keystore
- keystore_password
-
Password for the keystore.
Default: cassandra
- require_client_auth
-
Enables certificate authentication for client-to-node encryption.
-
true - Require certificate authentication for client-to-node encryption. Client certificates must be present on all nodes in the cluster.
-
false - Do not require certificate authentication for client-to-node encryption.
Default: false
-
- truststore
-
Relative path from DSE installation directory or absolute path to truststore containing the trusted certificate for authenticating remote servers.
Truststore password and path is only required when require_client_auth is set to
true
.Default: resources/dse/conf/.truststore
- truststore_password
-
Password for the truststore. This must match the password used when generating the keystore and truststore.
Truststore password and path is only required when require_client_auth is set to
true
.Default: cassandra
- protocol
-
Default: TLS
- algorithm
-
Default: SunX509
- keystore_type
-
Valid types are JKS, JCEKS, PKCS11, or PKCS12. For file-based keystores, use PKCS12.
DataStax supports PKCS11 as a
keystore_type
on nodes withcassandra
oradvanced
workloads. Theadvanced
workload support was added for DSE 6.8.2 and later. IfPKCS11
is needed, inserver_encryption_options
orclient_encryption_options
, specify thekeystore_type
asPKCS11
and thekeystore
asNONE
.PKCS11 is not supported as a
truststore_type.
Default: JKS
- truststore_type
-
Valid types are JKS, JCEKS, or PKCS12. For file-based truststores, use PKCS12.
Due to an OpenSSL issue, you cannot use a PKCS12 truststore that was generated via OpenSSL. For example, a truststore generated via the following command will not work with DSE:
However, truststores generated via Java’s Example:
|
Default: JKS
- cipher_suites
-
Supported ciphers:
-
TLS_RSA_WITH_AES_128_CBC_SHA
-
TLS_RSA_WITH_AES_256_CBC_SHA
-
TLS_DHE_RSA_WITH_AES_128_CBC_SHA
-
TLS_DHE_RSA_WITH_AES_256_CBC_SHA
-
TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA
-
TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA
Default: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
-
Transparent data encryption options
- transparent_data_encryption_options
-
DataStax Enterprise supports this option only for backward compatibility. When using DSE, configure data encryption options in the
dse.yaml
; see Transparent data encryption.TDE properties:
-
enabled: (Default: false)
-
chunk_length_kb: (Default: 64)
-
cipher: options:
-
AES
-
CBC
-
PKCS5Padding
-
-
key_alias: testing:1
-
iv_length: 16
iv_length
is commented out in the defaultcassandra.yaml
file. Uncomment only if cipher is set to AES. The value must be 16 (bytes). -
key_provider:
-
class_name: org.apache.cassandra.security.JKSKeyProvider
parameters:
-
keystore: conf/.keystore
-
keystore_password: cassandra
-
store_type: JCEKS
-
key_password: cassandra
-
-
-
SSL Ports
- ssl_storage_port
-
The SSL port for encrypted communication. Unused unless enabled in encryption_options. Follow security best practices, do not expose this port to the internet. Apply firewall rules.
Default: 7001
- native_transport_port_ssl
-
Dedicated SSL port where the CQL native transport listens for clients with encrypted communication. For security reasons, do not expose this port to the internet. Firewall it if needed.
-
commented out (disabled) - The native_transport_port encrypts all traffic.
-
port number different than native_transport_port - Use encryption for native_transport_port_ssl, keep native_transport_port unencrypted to use both unencrypted and encrypted traffic.
Default: 9142
-
User-defined functions (UDF) properties
- enable_user_defined_functions
-
Enables user defined functions (UDFs), code that is executed inside Cassandra daemons. UDFs can present a security risk, since they are executed on the server side. UDFs are executed in a sandbox to control what code can be executed. See the DataStax blog post User Defined Functions.
-
true - Enabled. Supports Java as the code language. Detects endless loops and unintended memory leaks.
-
false - Disabled.
Default: false
-
- enable_scripted_user_defined_functions
-
Enables the use of JavaScript language in UDFs. Scripted UDFs are less performant than UDFs and produce more garbage on the heap.
-
true - Enable scripted UDFs. Support JavaScript in addition to Java as a code language.
-
false - Disable scripted UDFs. Allow only Java as a code language.
If enable_user_defined_functions is
false
, this setting has no impact.
Default: false
-
- enable_user_defined_functions_threads
-
Enables sandbox for asynchronous JavaScript UDF execution. Does not apply to Java UDFs.
-
true - Enabled. Only one instance of a function can run at one time. Asynchronous execution prevents UDFs from running too long or forever and destabilizing the cluster.
-
false - Disabled. Allow multiple instances of the same function to run simultaneously.
Disabling asynchronous UDF execution implicitly disables the Java security manager. You must monitor the read timeouts for JavaScipt UDFs that run too long or forever, which can cause the cluster to destabilize.
Default: true
-
- user_defined_function_warn_micros
-
Threshold in microseconds (CPU time). When a UDF runs too long and this threshold is exceeded, a warning is logged and sent to the client. Java UDFs always issue a warning. Scripted UDFs log a warning only if enable_user_defined_functions_threads is set to true.
Default: 500
- user_defined_function_fail_micros
-
Threshold in microseconds (CPU time). When a fatal UDF run-time situation is detected and this threshold is exceeded, the UDF is stopped. Java UDFs always throw an exception and stop. Scripted UDFs throw an exception and stop only if enable_user_defined_functions_threads is set to true.
Default: 10000
- user_defined_function_warn_heap_mb
-
Threshold in MB for heap allocations. When this threshold is exceeded, a warning is logged and sent to the client. Java UDFs always issue a warning. Scripted UDFs log a warning only if enable_user_defined_functions_threads is set to true.
Default: 200
- user_defined_function_fail_heap_mb
-
Threshold in MB for heap allocations. When this threshold is exceeded, the UDF is stopped.
-
Java UDFs fail and are safely stopped. Java UDFs always throw an exception.
-
Scripted UDFs stop and throw an exception only if enable_user_defined_functions_threads is set to true.
Default: 500
-
- user_function_timeout_policy
-
Defines action when a scripted UDF exceeds user_defined_function_fail_micros threshold. Applies only when enable_user_defined_functions_threads is set to true.
-
die - Issue a warning to the client before the Cassandra daemon shuts down.
-
die_immediate - Shut down Cassandra daemon immediately, effectively prevents the client from receiving a warning.
-
ignore - Log the warning, but do not take any action. DataStax does not recommend this option for production environments.
Default: die
-
Continuous paging options
- continuous_paging
-
Tunes continuous paging that pushes pages, when requested, continuously to the client:
-
Maximum memory used:
max_concurrent_sessions ⨉ max_session_pages ⨉ max_page_size_mb
Default: calculated (60 ⨉ 4 ⨉ 8 = 1920 MB)
Guidance
-
Because memtables and SSTables are used by the continuous paging query, you can define the maximum period of time during which memtables cannot be flushed and compacted SSTables cannot be deleted.
-
If fewer threads exist than sessions, a session cannot execute until another one is swapped out.
-
Distributed queries (CL > ONE or non-local data) are swapped out after every page, while local queries at CL = ONE are swapped out after max_local_query_time_ms.
-
- max_concurrent_sessions
-
The maximum number of concurrent sessions. Additional sessions are rejected with an unavailable error.
Default: 60
- max_session_pages
-
The maximum number of pages that can be buffered for each session. If the client is not reading from the socket, the producer thread is blocked after it has prepared max_session_pages.
Default: 4
- max_page_size_mb
-
The maximum size of a page, in MB. If an individual CQL row is larger than this value, the page can be larger than this value.
Default: 8
- max_local_query_time_ms
-
The maximum time for a local continuous query to run. When this threshold is exceeded, the session is swapped out and rescheduled. Swapping and rescheduling ensures the release of resources that prevent the memtables from flushing and ensures fairness when max_threads < max_concurrent_sessions. Adjust when high write workloads exist on tables that have continuous paging requests.
Default: 5000
- client_timeout_sec
-
How long the server will wait, in seconds, for clients to request more pages if the client is not reading and the server queue is full.
Default: 600
- cancel_timeout_sec
-
How long to wait before checking if a paused session can be resumed. Continuous paging sessions are paused because of backpressure or when the client has not request more pages with backpressure updates.
Default: 5
- paused_check_interval_ms
-
How long to wait, in milliseconds, before checking if a continuous paging sessions can be resumed, when that session is paused because of backpressure.
Default: 1
Fault detection setting
- phi_convict_threshold
-
The sensitivity of the failure detector on an exponential scale. Generally, this setting does not need adjusting.
Default: 8
Memory leak detection settings
- sampling_probability
-
The sampling probability to track for the specified resource. For resources tracked, see nodetool leaksdetection.
-
0 - Disable tracking.
-
1 - Enable tracking all the time
-
percentage of time - A number between 0 and 1 to randomly track a resource. For example, 0.5 will track resources 50% of the time.
Tracking incurs a significant stack trace collection cost for every access and consumes heap space. Enable tracking only when directed by DataStax Support.
Default: 0.01
-
- max_stacks_cache_size_mb
-
The size of the cache for call stack traces. Stack traces are used to debug leaked resources, and use heap memory. Set the amount of heap memory dedicated to each resource by setting the max stacks cache size in MB.
Default: 32
- num_access_records
-
The average number of stack traces kept when a resource is accessed. Supported only for chunks in the cache.
Default: 0
- max_stack_depth
-
The depth of the stack traces collected. Changes only the depth of the stack traces that will be collected from the time the parameter is set. Deeper stacks are more unique, so increasing the depth may require increasing
stacks_cache_size_mb
.Default: 30
DataStax Astra emulation
- emulate_dbaas_defaults
-
Defaults for both server side and DataStax drivers are modified to match those used by the DataStax Astra cloud platform. Includes but is not limited to stricter guardrails defaults, or the use of
LOCAL_QUORUM
as the default consistency level.When enabled, the updated defaults reflect those of DataStax Astra at the time of the current DSE release and is only a best-effort emulation of said defaults. All nodes must use the same configuration value.
Default: false
DataStax Enterprise Backup and Restore Service
- enabled
-
Enables the DSE Backup and Restore Service.
Default: false
- staging_directory
-
Directory that holds the backup staging files. The default staging directory for package installations is
/var/lib/cassandra/backups_staging
. If the property is not set, the default staging directory is <$CASSANDRA_HOME>`/data/backups_staging`. - backups_max_retry_attempts
-
Sets the number of times the backup process should retry after each failed attempt.
Default: 3
Guardrails
- guardrails
-
Various settings that can automatically prevent common DSE performance issues. Default values may differ when
emulate_dbaas_defaults
is enabled.Default: disabled
- tombstone_warn_threshold
-
Logs a warning when the number of tombstones in memory exceeds the specified value. See Cassandra Anti-Patterns: Queues and Queue-like Datasets for details.
Because tombstone warning and failure thresholds are configured via |
This threshold may also be adjusted at runtime using the StorageService mbean. |
Default: 1000
- tombstone_failure_threshold
-
Fails when the number of tombstones in memory exceeds the specified value. See Cassandra Anti-Patterns: Queues and Queue-like Datasets for details.
Because tombstone warning and failure thresholds are configured via |
This threshold may also be adjusted at runtime using the StorageService mbean. |
Default: 10000
- partition_size_warn_threshold_in_mb
-
Logs a warning when compacting partitions larger than the specified value.
Default: 100
- batch_size_warn_threshold_in_kb
-
Logs a warning on any multiple-partition batch size that exceeds the specified value. 64kb per batch by default.
Use caution when increasing the size of this threshold as it can lead to node instability.
Default: 64
- batch_size_fail_threshold_in_kb
-
Fails any multiple-partition batch that exceeds the specified value.
Default: 640
- unlogged_batch_across_partitions_warn_threshold
-
Logs a warning for any batches not of type LOGGED that span across more partitions than the specified limit.
Default: 10
- column_value_size_failure_threshold_in_kb
-
Prevents the writing of columns larger than the specified value.
Default: -1 (disabled)
- columns_per_table_failure_threshold
-
Prevents creating more columns per table than the specified value.
Default: -1 (disabled)
- fields_per_udt_failure_threshold
-
Logs a warning when more fields are created in a UDT than the specified value.
Default: -1 (disabled)
- collection_size_warn_threshold_in_kb
-
Logs a warning when the size of a collection exceeds the specified value.
Default: -1 (disabled)
- items_per_collection_warn_threshold
-
Logs a warning when the number of elements in a collection exceeds the specified value.
Default: -1 (disabled)
- read_before_write_list_operations_enabled
-
Enables read-before-write operations such as setting or removing a list element by index.
Lightweight transactions are always allowed.
Default: true
- sai_indexes_per_table_failure_threshold
-
Fails when attempting to create more Storage-Attached Indexing (SAI) indexes per table than the specified value.
Default: 10
- sai_indexes_total_failure_threshold
-
Fails when attempting to create more SAI indexes in the cluster than the specified value.
Default: 100
- secondary_index_per_table_failure_threshold
-
Fails when attempting to create more secondary indexes per table than the specified value. Not applicable to SAI indexes; instead, see sai_indexes_per_table_failure_threshold.
Default: -1 (disabled)
- materialized_view_per_table_failure_threshold
-
Fails when attempting to create more materialized views per table than the specified value.
Default: -1 (disabled)
- tables_warn_threshold
-
Logs a warning when the number of tables created exceeds the specified value.
Default: -1 (disabled)
- tables_failure_threshold
-
Fails when the number of tables created exceeds the specified value.
Default: -1 (disabled)
- table_properties_disallowed
-
Prevent creating tables with the specified properties.
Default: disabled
- user_timestamps_enabled
-
Allows user-provided timestamps in write requests.
Default: true
- write_consistency_levels_disallowed
-
Prevents queries with the listed consistency settings.
Default: disabled
- page_size_failure_threshold_in_kb
-
Fails when paging exceeds the specified threshold. Serves as a hard paging limit.
Default: -1 (disabled)
- in_select_cartesian_product_failure_threshold
-
Fails when an
IN
query creates a cartesian product size exceeding the specified value. For example, a value of 50 would cause the followingIN
query, which results in a cartesian product of 100,a IN (1,2,...10) AND b IN (1,2...10)
, to fail.Default: -1 (disabled)
- partition_keys_in_select_failure_threshold
-
Fails when an IN query tries to queries more partition keys than the specified value.
Default: -1 (disabled)
- disk_usage_percentage_warn_threshold
-
Logs a warning when disk usage exceeds the specified percentage.
Default: -1 (disabled)
- disk_usage_percentage_failure_threshold
-
Fails when a write request would exceed the specified disk usage percentage.
Default: -1 (disabled)
- disk_usage_max_disk_size_in_gb
-
Limits a DSE cluster to a set amount of gigabytes when calculating thresholds for
disk_usage_percentage_warn_threshold
anddisk_usage_percentage_failure_threshold
. Disabled by default, in which case DSE uses the physically available disk size of data directories during disk-usage calculations.Default: -1 (disabled)