The cassandra.yaml
configuration file
The cassandra.yaml
file is the main configuration file for DataStax Enterprise (DSE).
Where is the cassandra.yaml
file?
The location of the cassandra.yaml
file depends on the type of installation:
Installation Type | Location |
---|---|
Package installations + Installer-Services installations |
|
Tarball installations + Installer-No Services installations |
|
After changing properties in the
|
Syntax
For the properties in each section, the main setting has zero spaces, and at least two spaces are required before each entry in that section. Adhere to the YAML syntax.
Organization
The configuration properties are grouped into the following sections:
-
The minimal properties needed for configuring a cluster.
-
Properties most frequently used when configuring DSE.
-
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.
-
DSE Unified Authentication provides authentication, authorization, and role management.
The dse.yaml
file is the primary configuration file for security, DSE Search, DSE Graph, and DSE Analytics.
Where is the dse.yaml
file?
The location of the dse.yaml
file depends on the type of installation:
Installation Type | Location |
---|---|
Package installations + Installer-Services installations |
|
Tarball installations + Installer-No Services installations |
|
Enabling DSE Unified Authentication requires additional configuration in dse.yaml
, see Configuring DSE Unified Authentication.
-
Continuous paging Properties configure memory, threads, and duration when pushing pages continuously to the client.
Values with note mark default values that are defined internally, missing, or commented out, or whose implementation depends on other properties in the |
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.
Related information: Initializing a single datacenter per workload type and Initializing multiple datacenters per workload type.
cluster_name
-
(Default:
Test Cluster
) 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.
listen_address
-
(Default:
localhost
) 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 this parameter to
listen_address
orlisten_interface
, not both.
Correct settings for various use cases:
-
Single-node installations: do one of the following:
-
Comment this property out.
If the node is properly configured (host name, name resolution, and so on), the database uses
InetAddress.getLocalHost()
to get the local address from the system. -
Leave set to the default,
localhost
.
-
-
Node in multi-node installations: set this property to the node’s IP address or hostname, or set
listen_interface
. -
Node in a multi-network or multi-Datacenter installation, within an EC2 environment that supports automatic switching between public and private interfaces: set
listen_address
to the node’s IP address or hostname, or setlisten_interface
. -
Node with two physical network interfaces in a multi-datacenter installation or cluster deployed across multiple Amazon EC2 regions using the
Ec2MultiRegionSnitch
:-
Set
listen_address
to this node’s private IP or hostname, or setlisten_interface
(for communication within the local datacenter). -
Set
broadcast_address
to the second IP or hostname (for communication between datacenters). -
Set
listen_on_broadcast_address
totrue
. -
If this node is a seed node, add the node’s public IP address or hostname to the seeds list.
-
-
Open the
storage_port
orssl_storage_port
on the public IP firewall.
-
listen_interface
-
The interface that the database uses to connect to other nodes. Interfaces must correspond to a single address. IP aliasing is not supported.
Set
listen_address
orlisten_interface
, do not set both.
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.
cdc_raw_directory
-
The directory where the CDC log is stored. Default locations:
-
Package installations and Installer-Services:
/var/lib/cassandra/cdc_raw
-
Tarball installations and Installer-No Services:
/var/lib/cassandra/cdc_raw
The directory where Change Data Capture logs are stored.
-
commitlog_directory
-
The directory where the commit log is stored. Default location:
/var/lib/cassandra/commitlog
For optimal write performance, place the commit log be on a separate disk partition, or (ideally) a separate physical device from the data file directories. Because the commit log is append only, an HDD is acceptable for this purpose.
data_file_directories
-
The directory location where table data is stored (in SSTables). The database distributes data evenly across the location, subject to the granularity of the configured compaction strategy. Default locations:
/var/lib/cassandra/data
.For production, DataStax recommends RAID 0 and SSDs.
saved_caches_directory
-
The directory location where table key and row caches are stored. For all installations, the default location of the
saved_caches
directory is/var/lib/cassandra/saved_caches
.
Commonly used properties
Properties most frequently used when configuring DSE.
Before starting a node for the first time, you should carefully evaluate your requirements.
Common initialization properties
Be sure to set the properties in the Quick start section as well. |
disk_access_mode
-
Default: auto. Disk access mode.
Enabling
mmap
can cause excessive paging if all of the actively read SSTables do not fit into RAM.-
auto
Enable
mmap
on both data and index files on a 64-bit JVM. -
mmap
Map index and data files.
-
mmap_index_only
Map only index files. Use this setting if you observe high number of page faults or steals along with increased latencies.
-
standard
Disable
mmap
entirely.
-
commit_failure_policy
-
Default: stop. Policy for commit disk failures:
-
die
Shut down gossip and Thrift and kill the JVM, so the node can be replaced.
-
stop
Shut down gossip and Thrift, 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.
-
prepared_statements_cache_size_mb
-
Default: auto, which is 1/256th of the heap or 10 MB, whichever is greater. Maximum size of the native protocol prepared statement cache.
Specifying a value that is too large results in long running GCs and possibly out-of-memory errors. Keep the value at a small fraction of the heap.
Constantly re-preparing statements is a performance penalty.
thrift_prepared_statements_cache_size_mb
-
Default: auto, which is 1/256th of the heap or 10 MB, whichever is greater. Maximum size of the Thrift prepared statement cache. Leave empty if you do not use Thrift.
Specifying a value that is too large results in long running GCs and possibly out-of-memory errors. Keep the value at a small fraction of the heap.
Constantly re-preparing statements is a performance penalty.
disk_optimization_strategy
-
Default:
disabled
. note The strategy for optimizing disk reads. Possible values: ssd or spinning. disk_failure_policy
-
Default:
stop
. Sets how the database responds to disk failure. Recommend settings:stop
orbest_effort
. Valid values:-
die
Shut down gossip and Thrift and kill the JVM for any file system errors or single SSTable errors, so the node can be replaced.
-
stop_paranoid
Shut down gossip and Thrift even for single SSTable errors.
-
stop
Shut down gossip and Thrift, 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 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.
Related information: Handling Disk Failures In Cassandra 1.2 blog and Recovering from a single disk failure using JBOD.
-
endpoint_snitch
-
Default:
DseSimpleSnitch
. Set to a class that implements theIEndpointSnitch
interface. The database uses the snitch to locate nodes and route requests.Use only snitch implementations bundled with DSE.
-
DseSimpleSnitch
Only appropriate 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.Where is the
cassandra-topology.properties
file?The location of the
cassandra-topology.properties
file depends on the type of installation:Installation Type Location Package installations + Installer-Services installations
/etc/dse/cassandra/cassandra-topology.properties
Tarball installations + Installer-No Services installations
<installation_location>/resources/cassandra/conf/cassandra-topology.properties
-
PropertyFileSnitch
Determines proximity by rack and datacenter, which 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 and the availability zone as the rack and uses only private IP addresses. For this reason, it 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 thestorage_port
orssl_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.
Related information: Snitches
-
rpc_address
-
Default:
localhost
. The listen address for client connections (Thrift RPC service and native transport). Valid values:-
unset
:Resolves the address using the configured hostname configuration of the node. If left unset, the hostname resolves to the IP address of this node using
/etc/hostname
,/etc/hosts
, or DNS. -
0.0.0.0
:Listens on all configured interfaces. You must set the
broadcast_rpc_address
to a value other than0.0.0.0
. -
IP address
-
hostname
Related information: Network
-
-
rpc_interface
-
Default:
eth1
. note The listen address for client connections. Interface must correspond to a single address, IP aliasing is not supported. Seerpc_address
. rpc_interface_prefer_ipv6
-
Default:
false
. 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.
-
seed_provider
-
The addresses of hosts 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
(Default:org.apache.cassandra.locator.SimpleSeedProvider
)The class that handles the seed logic. It can be customized, but this is typically not required.
-
seeds
(Default: 127.0.0.1)A comma-delimited list of IP addresses 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.
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).
Related information: Initializing a single datacenter per workload type and Initializing multiple datacenters per workload type.
-
enable_user_defined_functions
-
Default:
false
. User defined functions (UDFs) present a security risk, since they are executed on the server side. UDFs are executed in a sandbox to contain the execution of malicious code. They are disabled by default. enable_scripted_user_defined_functions
-
Default:
false
. Java UDFs are always enabled, ifenable_user_defined_functions
is true. Enable this option to use UDFs withlanguage javascript
or any custom JSR-223 provider. This option has no effect ifenable_user_defined_functions
is false.
Common compaction settings
compaction_throughput_mb_per_sec
-
Default: 16. Throttles compaction to the specified Mb/second across the instance. The faster the database inserts data, the faster the system must compact in order to keep the SSTable count down. The recommended value is 16 to 32 times the rate of write throughput (in Mb/second). Setting the value to 0 disables compaction throttling.
Related information: Configuring compaction
compaction_large_partition_warning_threshold_mb
-
Default: 100. The database logs a warning when compacting partitions larger than the set value.
Common memtable settings
memtable_heap_space_in_mb
-
Default: 1/4 of heap size. note
The amount of on-heap memory allocated for memtables. The database uses the total of this amount and the value of
memtable_offheap_space_in_mb
to set a threshold for automatic memtable flush. For details, seememtable_cleanup_threshold
.Related information: Tuning the Java heap
memtable_offheap_space_in_mb
-
Default: 1/4 of heap size. note
Sets the total amount of off-heap memory allocated for memtables. The database uses the total of this amount and the value of
memtable_heap_space_in_mb
to set a threshold for automatic memtable flush. For details, seememtable_cleanup_threshold
.Related information: Tuning the Java heap
Common disk settings
concurrent_reads
-
Default: 32. note Workloads with more data than can fit in memory encounter a bottleneck in fetching data from disk during reads. Setting
concurrent_reads
to (16 ×number_of_drives
) allows operations to queue low enough in the stack so that the OS and drives can reorder them. The default setting applies to both logical volume managed (LVM) and RAID drives.
concurrent_writes
-
Default: 32. note Writes in DSE are rarely I/O bound, so the ideal number of concurrent writes depends on the number of CPU cores on the node. The recommended value is 8 ×
number_of_cpu_cores
. concurrent_counter_writes
-
Default: 32. note Counter writes read the current values before incrementing and writing them back. The recommended value is (16 ×
number_of_drives
) . concurrent_materialized_view_writes
-
Default: 32. Limit on the number of concurrent materialized view writes. Set this to the lesser of concurrent reads or concurrent writes, because there is a read involved in each materialized view write.
Common automatic backup settings
incremental_backups
-
Default: false. Backs up data updated since the last snapshot was taken. When enabled, the database creates a hard link to each SSTable flushed or streamed locally in a backups subdirectory of the keyspace data. Removing these links is the operator’s responsibility.
Related information: Enabling incremental backups
snapshot_before_compaction
-
Default: false. Enables or disables taking 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.
Related information: Configuring compaction
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
Common fault detection setting
phi_convict_threshold
-
Default:
8
. note Adjusts the sensitivity of the failure detector on an exponential scale. Generally this setting does not need adjusting.Related information: About failure detection and recovery
Performance tuning properties
Tuning performance and system resource utilization, including commit log, compaction, memory, disk I/O, CPU, reads, and writes.
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
. -
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
-
Default:
10000
. Use withcommitlog_sync: periodic
. Controls how often the commit log is synchronized to disk. Periodic syncs are acknowledged immediately. commitlog_sync_batch_window_in_ms
-
Default:
disabled
. Use withcommitlog_sync:batch
. The maximum length of time that queries may be batched together.
commitlog_segment_size_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.
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 for most commitlog archiving, but if you want a finer granularity, 8 or 16 MB is reasonable.
Default:
32
max_mutation_size_in_kb
-
Default: ½ of
commitlog_segment_size_in_mb
.If a mutation’s size exceeds this value, 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.
Restriction: If you set
max_mutation_size_in_kb
explicitly, then you must setcommitlog_segment_size_in_mb
to at least twice the size ofmax_mutation_size_in_kb / 1024
.See
commitlog_segment_size_in_mb
above. 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: 128
# max_pending_lw_transactions: 10000
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 this uses 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.
-
Default: calculated 8x the number of TPC cores
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
cdc_enabled: false
cdc_total_space_in_mb: 4096
cdc_free_space_check_interval_ms: 250
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
-
Default: 4096 and 1/8th of the total space of the drive where the
cdc_raw_directory
resides.)note If space gets above this value, the database throwsWriteTimeoutException
on Mutations including tables with CDC enabled. ACDCCompactor
(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
-
Default: 250.note When the
cdc_raw
limit is hit and the CDCCompactor is either running behind or experiencing backpressure, this interval is checked to see if any new space for cdc-tracked tables has been made available. commitlog_total_space_in_mb
-
Disk usage threshold for commit logs before triggering the database to flush 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 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:
32
for 32-bit JVMs,8192
for 64-bit JVMs. gc_log_threshold_in_ms
-
Default: 200. The threshold for log messages at the INFO level. Adjust to minimize logging.
Compaction settings
Related information: Configuring compaction
concurrent_compactors
-
Defaults to the smaller of number of disks or number of cores, with a minimum of 2 and a maximum of 8. note 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.
sstable_preemptive_open_interval_in_mb
-
Default: 50. 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 setting helps to smoothly transfer reads between the SSTables by reducing page cache churn and keeps hot rows hot.
Memtable settings
memtable_allocation_type
-
Default:
heap_buffers
. The method the database uses to allocate and manage memtable memory.-
heap_buffers
On heap NIO (non-blocking I/O) buffers.
-
offheap_buffers
Off heap (direct) NIO buffers.
-
offheap_objects
Native memory, eliminating NIO buffer heap overhead.
-
memtable_cleanup_threshold
-
Default: 1/(
memtable_flush_writers
+ 1). note Ratio used for automatic memtable flush. The database addsmemtable_heap_space_in_mb
tomemtable_offheap_space_in_mb
and multiplies the total bymemtable_cleanup_threshold
to get a space amount in MB. When the total amount of memory used by all non-flushing memtables exceeds this amount, the database flushes the largest memtable to disk.For example, consider a node where the total of
memtable_heap_space_in_mb
andmemtable_offheap_space_in_mb
is 1000, andmemtable_cleanup_threshold
is0.50
. Thememtable_cleanup
amount is 500MB. This node has two memtables: Memtable A (150MB) and Memtable B (350MB). When either memtable increases, the total space they use exceeds 500MB and the database flushes the Memtable B to disk.A larger value for
memtable_cleanup_threshold
means larger flushes, less frequent flushes and potentially less compaction activity, but also less concurrent flush activity, which can make it difficult to keep your disks saturated under heavy write load.This section documents the formula used to calculate the ratio based on the number of
memtable_flush_writers
. The default value incassandra.yaml
is0.11
, which works if the node has many disks or if you set the node’smemtable_flush_writers
to8
. As another example, if the node uses a single SSD, the value formemtable_cleanup_threshold
computes to0.33
, based on the minimummemtable_flush_writers
value of2
.
file_cache_size_in_mb
-
Default: Smaller of 1/4 heap or 512. Total memory to use for SSTable-reading buffers.
buffer_pool_use_heap_if_exhausted
-
Default: disabled. note Indicates whether the database allocates on-heap or off-heap memory when the SSTable buffer pool is exhausted (when the buffer pool has exceeded the maximum memory
file_cache_size_in_mb
), beyond this amount, the database stops caching buffers, but allocates on request.
memtable_flush_writers
-
Default: Smaller of number of disks or number of cores with a minimum of 2 and a maximum of 8. note The number of memtable flush writer threads. These threads are blocked by disk I/O, and each one holds a memtable in memory while blocked. If your data directories are backed by SSDs, increase this setting to the number of cores.
Cache and index settings
column_index_size_in_kb
-
Default: 64. Granularity of the index of rows within a partition. For huge rows, decrease this setting to improve seek time. If you use key cache, be careful not to overwhelm it by making this setting too large. If you’re unsure of the size of the rows, it’s best to use the default setting.
index_summary_capacity_in_mb
-
Default: 5% of the heap size [empty]. note Fixed memory pool size in MB for SSTable index summaries. If the memory usage of all index summaries exceeds this limit, any SSTables with low read rates shrink their index summaries to meet this limit. This is a best-effort process. In extreme conditions, the database may use more than this amount of memory.
index_summary_resize_interval_in_minutes
-
Default: 60. How frequently index summaries should be re-sampled. Re-sampling is done periodically to redistribute memory from the fixed-size pool to SSTables proportional their recent read rates. To disable, set to -1. This setting leaves existing index summaries at their current sampling level.
Streaming settings
stream_throughput_outbound_megabits_per_sec
-
Default: 200. note Throttle for the throughput of all outbound streaming file transfers on a node. The database does mostly sequential I/O when streaming data during bootstrap or repair. This can saturate the network connection and degrade client (RPC) performance.
inter_dc_stream_throughput_outbound_megabits_per_sec
-
Default: 200. note Throttle for all streaming file transfers between datacenters, and for network stream traffic as configured with
stream_throughput_outbound_megabits_per_sec
.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.
streaming_keep_alive_period_in_secs
-
Default: 300. note Specifies for node to send keep-alive message at this interval to prevent reset connections during streaming. The stream session fails when a keep-alive message is not received for 2 keep-alive cycles.
Disks settings
trickle_fsync
-
Default: true. 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. trickle_fsync_interval_in_kb
-
Default: 10240. The size of the fsync in kilobytes.
Advanced properties
Properties for advanced users or properties that are less commonly used.
Advanced initialization properties
auto_bootstrap
-
Default: true. This setting has been removed from default configuration. It causes new (non-seed) nodes migrate the right data to themselves automatically. When initializing a fresh cluster without data, add
auto_bootstrap
:false
.Related information: Initializing a single datacenter per workload type and Initializing multiple datacenters per workload type.
batch_size_warn_threshold_in_kb
-
Default: 64. Log a warning message when any multiple-partition batch size exceeds this value.
Increasing this threshold can lead to node instability.
batch_size_fail_threshold_in_kb
-
Default: 640. Fails any batch whose size exceeds this setting. The default value is 10X the value of
batch_size_warn_threshold_in_kb
. unlogged_batch_across_partitions_warn_threshold
-
Default: 10. Causes the database to log a
WARN
message on any batches not of typeLOGGED
that span across more partitions than this limit. The default value is 10 partitions. cdc_enabled
-
Default: false. Enable or disable change data capture (CDC) functionality on a per-node basis. This modifies the logic used for write path allocation rejection (standard: never reject. cdc: reject Mutation containing a CDC-enabled table if at space limit in
cdc_raw_directory
).Do not enable CDC on a mixed-version cluster. Upgrade all nodes to DSE 5.1 before enabling and restarting the cluster.
broadcast_address
-
Default:
listen_address
. note 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 aslisten_address
. A node does not need a separatebroadcast_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 separatelisten_address
andbroadcast_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 forlisten_address
.
listen_on_broadcast_address
-
Default: false. If this node uses multiple physical network interfaces, set a unique IP address for
broadcast_address
and setlisten_on_broadcast_address
to true. This enables the node to communicate on both interfaces.Set this property to false if the node is on a network that automatically routes between public and private networks, as Amazon EC2 does.
For configuration details, see the instructions for
listen_address
.
initial_token
-
Default: 1 (disabled). Set this property for single-node-per-token architecture, in which a node owns exactly one contiguous range in the ring space. Setting this property overrides
num_tokens
.If your installation is not using vnodes or this node’s
num_tokens
is set it to 1 or is commented out, you should always set aninitial_token
value when setting up a production cluster for the first time, and when adding capacity. See Generating tokens.Use this parameter only with
num_tokens
(vnodes) in special cases such as Restoring from a snapshot.
num_tokens
-
Default: 1 (disabled)
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.
-
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.
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 incassandra.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.
-
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. TheMurmur3Partitioner
is the default partitioning strategy for new DSE 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.
See Virtual node (vnode) configuration, and for set up instructions see Adding vnodes to an existing cluster or Adding a datacenter to a cluster.
-
- partitioner
-
Default:
org.apache.cassandra.dht.Murmur3Partitioner
. Sets the class that distributes rows (by partition key) across all nodes in the cluster. AnyIPartitioner
may be used, including your own as long as it is in the class path. For new clusters use the default partitioner.DSE provides the following partitioners for backwards compatibility:
-
RandomPartitioner
-
ByteOrderedPartitioner
(deprecated) -
OrderPreservingPartitioner
(deprecated)
Use only partitioner implementations bundled with DSE.
Related information: Partitioners
-
storage_port
-
Default: 7000. The port for inter-node communication.
tracetype_query_ttl
-
Default: 86400. TTL for different trace types used during logging of the query process.
tracetype_repair_ttl
-
Default: 604800. TTL for different trace types used during logging of the repair process.
Advanced automatic backup setting
auto_snapshot
-
Default: true. Enables or disables whether the database takes a snapshot 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.
Key caches and global row properties
When creating or modifying tables, you can enable or disable the key cache (partition key cache) or row cache for that table by setting the caching parameter. Other row and key 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.
Related information: Configuring caches
key_cache_keys_to_save
-
Default: disabled. All keys are saved. note Number of keys from the key cache to save.
key_cache_save_period
-
Default: 14400. (4 hours) Duration in seconds that keys are kept in cache. Caches are saved to
saved_caches_directory
. Saved caches greatly improve cold-start speeds and have relatively little effect on I/O.
key_cache_size_in_mb
-
Default: empty. A global cache setting for the maximum size of the key cache in memory (for all tables). If no value is set, the cache is set to the smaller of 5% of the available heap, or 100MB. To disable set to 0.
Related information:
nodetool setcachecapacity
, Enabling and configuring caching. column_index_cache_size_in_kb
-
Default: 2. A 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. This limitation prevents index entries from large partitions from taking up all the space in the partition key cache (which is controlled by
key_cache_size_in_mb
). row_cache_class_name
-
Default: disabled. note The classname of the row cache provider to use. Valid values:
OHCProvider
(fully off-heap) orSerializingCacheProvider
(partially off-heap).Use only row cache provider implementations bundled with DSE.
row_cache_keys_to_save
-
Default: disabled. All keys are saved. note Number of keys from the row cache to save.
row_cache_size_in_mb
-
Default: 0. To disable, set to 0. Maximum size of the row cache in memory. The row cache can save more time than
key_cache_size_in_mb
,, 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. row_cache_save_period
-
Default: 0. To disable, set to 0. The number of seconds that rows are kept in cache. Caches are saved to
saved_caches_directory
. This setting has limited use as described inrow_cache_size_in_mb
.
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
-
Default value: empty. note When no value is set, the database uses the smaller of minimum of 2.5% of Heap or 50 megabytes (MB). If your system performs counter deletes and relies on low
gc_grace_seconds
, you should disable the counter cache. To disable, set to 0. counter_cache_save_period
-
Default: 7200. (2 hours) the amount of time after which the database saves the counter cache (keys only). The database saves caches to
saved_caches_directory
. counter_cache_keys_to_save
-
Default value: disabled. note Number of keys from the counter cache to save. When this property is disabled, the database saves 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.
See Cassandra anti-patterns: Queues and queue-like datasets.
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
.
Related information: Cassandra anti-patterns: Queues and queue-like datasets
tombstone_warn_threshold
-
Default: 1000. The database issues a warning if a query scans more than this number of tombstones.
tombstone_failure_threshold
-
Default: 100000. The database aborts a query if it scans more than this number of tombstones.
Network timeout settings
aggregated_request_timeout_in_ms
-
Number of milliseconds that the coordinator waits for aggregated read operations to complete. For example,
SELECT COUNT(*), MIN(x)
. Default: 120000. cas_contention_timeout_in_ms
-
Default: 1000. The number of milliseconds during which 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.
counter_write_request_timeout_in_ms
-
Default: 5000. The number of milliseconds that the coordinator waits for counter writes to complete before timing it out.
range_request_timeout_in_ms
-
Default: 10000. The number of milliseconds that the coordinator waits for sequential or index scans to complete before timing it out.
read_request_timeout_in_ms
-
Default: 5000. The number of milliseconds that the coordinator waits for read operations to complete before timing it out.
request_timeout_in_ms
-
Default: 10000. The default timeout value for other miscellaneous operations.
Related information: Hinted handoff: repair during write path.
slow_query_log_timeout_in_ms
-
Default: 500. How long before a node logs slow queries. Select queries that exceed this value generate an aggregated log message to identify slow queries. To disable, set to 0.
truncate_request_timeout_in_ms
-
Default: 60000. The number of milliseconds that 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. write_request_timeout_in_ms
-
Default: 2000. The number of milliseconds that the coordinator waits for a write operations to complete before timing it out for requests with at least one node in the local datacenter.
Related information: Hinted handoff: repair during write path.
cross_dc_rtt_in_ms
-
Default: 0. Increases the cross-datacenter timeout (
write_request_timeout_in_ms + cross_dc_rtt_in_ms
) for requests that only involve 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 asQUORUM
.
Inter-node settings
cross_node_timeout
-
Default: false. Enables or disables operation timeout information exchange between nodes (to accurately measure request timeouts). If this property is disabled, 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.
internode_send_buff_size_in_bytes
*-
Default: empty. note The sending socket buffer size in bytes for inter-node calls.
The buffer size set by this parameter and
internode_recv_buff_size_in_bytes
is limited bynet.core.wmem_max
. If this property is not set,net.ipv4.tcp_wmem
determines the buffer size. 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
Related information: TCP settings
-
internode_recv_buff_size_in_bytes
-
Default: empty. note The receiving socket buffer size in bytes for inter-node calls.
internode_compression
-
Default: dc. Controls whether traffic between nodes is compressed. Valid values:
-
all
Compresses all traffic.
-
dc
Compresses traffic between datacenters only.
-
none
No compression.
-
inter_dc_tcp_nodelay
-
Default: false. Enable this property or disable tcp_nodelay for inter-datacenter communication. If this property is disabled, the network sends larger, but fewer, network packets. This reduces overhead from the TCP protocol itself. However, disabling
inter_dc_tcp_nodelay
may increase latency by blocking cross datacenter responses.
Native transport (CQL Binary Protocol)
start_native_transport
-
Default: true. Enables or disables the native transport server. This server uses the same address as the
rpc_address
, but the port it uses is different fromrpc_port
. Seenative_transport_port
.
native_transport_port
-
Default: 9042. The port where the CQL native transport listens for clients.
native_transport_max_threads
-
Default: 128. note The maximum number of thread handling requests. Similar to
rpc_max_threads
, but this property differs as follows:-
The default for
native_transport_max_threads
is 128; the default forrpc_max_threads
is unlimited. -
There is no corresponding
native_transport_min_threads
. -
The database stops idle native transport threads after 30 seconds.
-
native_transport_max_frame_size_in_mb
-
Default: 256. The maximum allowed size of a frame. Frame (requests) larger than this are rejected as invalid.
native_transport_max_concurrent_connections
-
Default: -1. The maximum number of concurrent client connections. The default value of -1 means unlimited.
native_transport_max_concurrent_connections_per_ip
-
Default: -1. The maximum number of concurrent client connections per source IP address. The default value of -1 means unlimited.
RPC (remote procedure call) settings
Settings for configuring and tuning client connections.
broadcast_rpc_address
-
Default: empty. note The RPC address for broadcast to drivers and other nodes. This cannot be set to 0.0.0.0. If left blank, the database uses the
rpc_address
orrpc_interface
. Ifrpc_address
orrpc_interface
is set to 0.0.0.0, this property must be set.
rpc_port
-
Default: 9160. Thrift port for client connections.
start_rpc
-
Default: true. Enables or disables the Thrift RPC server.
rpc_keepalive
-
Default: true. Enables or disables keepalive on client connections (RPC or native).
rpc_max_threads
-
Default: unlimited. note Regardless of your choice of RPC server (
rpc_server_type
),rpc_max_threads
dictates the maximum number of concurrent requests in the RPC thread pool. If you are using the parameter sync (seerpc_server_type
) it also dictates the number of clients that can be connected. A high number of client connections could cause excessive memory usage for the thread stack. Connection pooling on the client side is highly recommended. Setting arpc_max_threads
acts as a safeguard against misbehaving clients. If the number of threads reaches the maximum, the database blocks additional connections until a client disconnects. rpc_min_threads
-
Default: unlimited. note The minimum thread pool size for remote procedure calls.
rpc_recv_buff_size_in_bytes
-
Default: empty. note The receiving socket buffer size for remote procedure calls.
rpc_send_buff_size_in_bytes
-
Default: empty. note The sending socket buffer size in bytes for remote procedure calls.
rpc_server_type
-
Default: sync. The database provides three options for the RPC server.
sync
andhsha
performance is about the same, buthsha
uses less memory.-
sync: (Default: one thread per Thrift connection.)
For a very large number of clients, memory is the limiting factor. On a 64-bit JVM, 180 KB is the minimum stack size per thread and corresponds to your use of virtual memory. Physical memory may be limited depending on use of stack space.
-
hsha
:Half synchronous, half asynchronous. All Thrift clients are handled asynchronously using a small number of threads that does not vary with the number of clients. This mechanism scales well to many clients. The RPC requests are synchronous (one thread per active request).
If you select this option, you must change the default value (unlimited) of
rpc_max_threads
. -
Your own RPC server
You must provide a fully-qualified class name of an
o.a.c.t.TServerFactory
that can create a server instance.
-
Advanced fault detection settings
Settings to handle poorly performing or failing components.
gc_warn_threshold_in_ms
-
Default: 1000. 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.)
Additional information: Configuring logging.
otc_coalescing_strategy
-
Default:
DISABLED
. Supported strategies are:FIXED
,MOVINGAVERAGE
,TIMEHORIZON
, andDISABLED
. Suitable for VMs, but not noticeably performant in other environments. TheOutboundTcpConnection
(otc) strategy to:-
Increase message throughput (doubling or more).
-
Process multiple messages with one trip to read from a socket.
-
Perform all the task submission work at the same time.
-
Reduce context switching.
-
Increase cache friendliness of network message processing.
Use only strategy implementations bundled with DSE.
-
otc_coalescing_window_us
-
Default: disabled. note How many microseconds to wait for coalescing. For fixed strategy, the amount of time after the first message is received before it is sent with any accompanying messages. For moving average, this is the maximum wait time and the interval that messages must arrive on average to enable coalescing.
otc_coalescing_enough_coalesced_messages
-
Default: disabled. note The threshold for the number of messages. Do not coalesce messages when this value is exceeded. Should be more than 2 and less than 128.
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%)
back_pressure_enabled
-
Default: false. Enable for the coordinator to apply the specified back pressure strategy to each mutation that is sent to replicas.
back_pressure_strategy
back_pressure_strategy:
- class_name: org.apache.cassandra.net.RateBasedBackPressure
parameters:
- high_ratio: 0.90
factor: 5
flow: FAST
+
Default: RateBasedBackPressure
.
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. |
+
* RateBasedBackPressure
+ Ratio between incoming mutation responses and outgoing mutation requests.
-
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.
-
factor
A number between 1 and 10. Increases or decreases rate limiting.
-
flow
Default:
FAST
. The flow speed to apply rate limiting:-
FAST
- rate limited to the speed of the fastest replica. -
SLOW
- rate limit to the speed of the slowest replica.max_value_size_in_mb
-
Default: 256. note The maximum size of any value in SSTables. It detects SSTable corruption and marks the SSTables as corrupted when the threshold is exceeded.
dynamic_snitch_badness_threshold
-
Default: 0.1. 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. A value of zero to 1.0 for the
read_repair_chance
table property maximizes cache capacity across the nodes. dynamic_snitch_reset_interval_in_ms
-
Default: 600000. Time interval after which the database resets all node scores. This allows a bad node to recover.
dynamic_snitch_update_interval_in_ms
-
Default: 100. The number of milliseconds between the database’s calculation of node scores. Because score calculation is CPU intensive, be careful when reducing this interval.
hints_flush_period_in_ms
-
Default: 10000. The number of milliseconds the database waits before flushing hints from internal buffers to disk.
-
hints_directory
-
The directory in which hints are stored. If not set, the directory is
$CASSANDRA_HOME/data/hints
.Default:
/var/lib/cassandra/hints
hinted_handoff_enabled
-
Enables or disables 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.
-
false - do not enable hinted handoff
-
true - globally enable hinted handoff, except for datacenters specified for
hinted_handoff_disabled_datacenters
Default:true
-
hinted_handoff_disabled_datacenters
-
A blacklist of datacenters that do not perform hinted handoffs. To disable hinted handoff on a certain datacenter, add its name to this list.
Default: commented out
hinted_handoff_throttle_in_kb
-
Default: 1024. 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 database computes the hint transmission rate based on the uncompressed hint size, even if
internode_compression
orhints_compression
is enabled. max_hint_window_in_ms
-
Default: 10800000. (3 hours) 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.
Related information: About failure detection and recovery
max_hints_delivery_threads
-
Default: 2. Number of threads the database uses to deliver hints. In multiple datacenter deployments, consider increasing this number because cross datacenter handoff is generally slower.
max_hints_file_size_in_mb
-
Default: 128. The maximum size for a single hints file, in megabytes.
hints_compression
-
Default:
LZ4Compressor
. The compressor for hint files. Supported compressors: LZ, Snappy, and Deflate. If you do not specify a compressor, the database does not compress hints files. batchlog_replay_throttle_in_kb
-
(Default: 1024 kilobytes per second) Total maximum throttle for replaying failed logged batches. Throttling is reduced proportionally to the number of nodes in the cluster.
Request scheduler properties
Settings to handle incoming client requests according to a defined policy. If your nodes are overloaded and dropping requests, DataStax recommends that you add more nodes rather than use these properties to prioritize requests.
The properties in this section apply only to the Thrift transport. They have no effect on the use of CQL over the native protocol. |
request_scheduler
-
Default:
org.apache.cassandra.scheduler.NoScheduler
. The scheduler to handle incoming client requests according to a defined policy. This scheduler is useful for throttling client requests in single clusters containing multiple keyspaces. This parameter is specifically for requests from the client and does not affect inter-node communication. Valid values:-
org.apache.cassandra.scheduler.NoScheduler
The database does no scheduling.
-
org.apache.cassandra.scheduler.RoundRobinScheduler
The database uses a round robin of client requests to a node with a separate queue for each
request_scheduler_id
property. -
The database uses a Java class that implements the
RequestScheduler
interface.
-
request_scheduler_id
-
Default:
keyspace
. note The scope of the scheduler’s activity. Currently the only valid value iskeyspace
. request_scheduler_options
-
Default:
NoScheduler
. A list of properties that define configuration options forrequest_scheduler
.RoundRobin
:A round robin of client requests to a node with a separate queue for each
request_scheduler_id
.-
throttle_limit
- The number of in-flight requests per client. Requests that exceed this limit are queued up until running requests complete. Recommended value is ((concurrent_reads + concurrent_writes) × 2
). -
default_weight
- Default: 1. note How many requests the scheduler handles during each turn of the round robin. -
weights
- A list of keyspaces with assigned weights.
-
Thrift interface properties
Legacy API for older clients. CQL is a simpler and better API for the database.
thrift_framed_transport_size_in_mb
-
Default:
15
. Frame size (maximum field length) for Thrift. The frame is the row or part of the row that the application is inserting.
Security properties
DSE Advanced Security fortifies 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.
These properties protect Cassandra databases against potential harm due to deliberate attack or user error.
authenticator
-
Default:
com.datastax.bdp.cassandra.auth.DseAuthenticator
. 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. Seeauthentication_options
indse.yaml
.Use only authentication implementations bundled with DSE.
Default:
com.datastax.bdp.cassandra.auth.DseAuthenticator
internode_authenticator
-
Default:
enabled
. note Internode authentication backend. It implementsorg.apache.cassandra.auth.AllowAllInternodeAuthenticator
to allows or disallow connections from peer nodes.Use only authentication implementations bundled with DSE.
authorizer
-
Default:
com.datastax.bdp.cassandra.auth.DseAuthorizer
. The authorization backend. Authorizers other thanDseAuthorizer
are not supported.DseAuthorizer
supports enhanced permission management of DSE-specific resources. Authorizers other thanDseAuthorizer
are deprecated and not supported. Some security features might not work correctly if other authorizers are used. See Authorization options indse.yaml
.Use only authorization implementations bundled with DSE.
Default:
com.datastax.bdp.cassandra.auth.DseAuthorizer
role_manager
-
Default:
com.datastax.bdp.cassandra.auth.DseRoleManager
. The DSE Role Manager supports LDAP roles and internal roles supported by theCassandraRoleManager
. Role options are stored in thedse_security
keyspace. When using the DSE Role Manager, increase the replication factor of thedse_security
keyspace. Role managers other thanDseRoleManager
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.
roles_validity_in_ms
-
Default:
2000
. Validity period for roles cache; set to0
to disable. 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. roles_update_interval_in_ms
-
Default:
2000
. Enable to refresh interval for roles cache. Defaults to the same value asroles_validity_in_ms
. 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. Ifroles_validity_in_ms
is non-zero, then this must be also. credentials_validity_in_ms
-
Default:
2000
. How many milliseconds credentials in the cache remain valid. This cache is tightly coupled to the providedPasswordAuthenticator
implementation ofIAuthenticator
. If anotherIAuthenticator
implementation is configured, the database does not use this cache, and these settings have no effect.Credentials are cached in encrypted form. This may cause a performance penalty that offsets the reduction in latency gained by caching.
Cache credentials and permissions are not automatically invalidated after issuing a
REVOKE
statement.This setting is disabled when set to 0.
credentials_update_interval_in_ms
-
Default: same value as
credentials_validity_in_ms
. After this interval, cache entries become eligible for refresh. The next time the cache is accessed, the system schedules an asynchronous reload of the cache. Until this cache reload is complete, the cache returns the old values.If
credentials_validity_in_ms
is nonzero, this property must also be nonzero.
permissions_validity_in_ms
-
Default:
2000
. Fetching permissions can be resource intensive. Define how many milliseconds permissions in cache remain valid to manage performance impact of permissions queries. 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.Cache credentials and permissions are not automatically invalidated after issuing a
REVOKE
statement.This setting is disabled when set to
0
. permissions_update_interval_in_ms
-
Default: same value as
permissions_validity_in_ms
. Sets 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. Ifpermissions_validity_in_ms
is nonzero,roles_update_interval_in_ms
must also be non-zero. permissions_cache_max_entries
-
Default:
1000
. 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 row-level permissions, use this formula:numRlacUsers * numRlacTables + 100
If this option is not present in
cassandra.yaml
, manually enter it. See Enabling DSE Unified Authentication. server_encryption_options
-
Configure inter-node encryption. If enabled, you must also generate keys and provide the appropriate key and truststore locations and passwords. No custom encryption options are supported. Available options:
-
internode_encryption
: Default: none. Enables or disables encryption of inter-node communication using theTLS_RSA_WITH_AES_128_CBC_SHA
cipher suite for authentication, key exchange, and encryption of data transfers. Use the DHE/ECDHE ciphers, such asTLS_DHE_RSA_WITH_AES_128_CBC_SHA
if running in (Federal Information Processing Standard) FIPS 140 compliant mode. Available inter-node options:-
all
Encrypt all inter-node communications.
-
none
No encryption.
-
dc
Encrypt the traffic between the datacenters (server only).
-
rack
Encrypt the traffic between the racks (server only).
-
-
keystore
: Default:conf/.keystore
.The location of a 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.
-
keystore_password
: Default:cassandra
.Password for the keystore.
-
truststore
: Default:conf/.truststore
.Location of the truststore containing the trusted certificate for authenticating remote servers.
-
truststore_password
: Default:cassandra
.Password for the truststore.
The passwords used in these options must match the passwords used when generating the keystore and truststore. For instructions on generating these files, see Creating a Keystore to Use with JSSE.
Advanced settings:
-
protocol
: Default:TLS
. -
algorithm
: Default:SunX509
. -
store_type
: 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
-
-
require_client_auth
: Default:false
.Enables or disables certificate authentication.
-
require_endpoint_verification
: Default:false
.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"
Related information: Securing internal transactional node connections
-
client_encryption_options
-
Enables or disables 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 currently enabled for DSE. Available options:
-
enabled
: Default:false
.To enable client encryption, set to true.
-
optional
: Default:false
.When
optional
is selected, both encrypted and unencrypted connections over native transport are allowed. This 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. -
keystore
: Default:conf/.keystore
.The location of a 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.
-
keystore_password
: Default:cassandra
.Password for the keystore. This must match the password used when generating the keystore and truststore.
-
require_client_auth
: Default:false
.Enables or disables certificate authentication.
-
truststore
: Default:conf/.truststore
.Set this property if
require_client_auth
is true. -
truststore_password
: Default:cassandra
Set if
require_client_auth
is true.
Advanced settings:
-
protocol
: Default:TLS
. -
algorithm
: Default:SunX509
. -
store_type
: 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
Related information: Securing client to cluster connections
-
-
transparent_data_encryption_options
-
DSE only supports this option for backwards 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 be16
(bytes). -
key_provider
:-
class_name
:org.apache.cassandra.security.JKSKeyProvider
parameters:
-
keystore
:conf/.keystore
-
keystore_password
:cassandra
-
store_type
:JCEKS
-
key_password
:cassandra
-
-
-
ssl_storage_port
-
Default:
7001
. The SSL port for encrypted communication. Unused unless enabled inencryption_options
. native_transport_port_ssl
-
Default:
9142
. If client encryption is enabled andnative_transport_port_ssl
is disabled, thenative_transport_port
(default:9042
) encrypts all traffic. To use both unencrypted and encrypted traffic, enablenative_transport_port_ssl
.
Continuous paging
continuous_paging
-
Pushes pages continuously to the client when requested by the client, parameters control:
-
Maximum memory used. Default:
60 ⨉ 4 ⨉ 8 = 1920 MB
(max_concurrent_sessions
⨉max_session_pages
⨉max_page_size_mb
). -
Maximum number of threads.
-
Maximum duration for local queries.
Guidance
: -
If the client is not reading from the socket, the producer thread is blocked after it has prepared
max_session_pages
, up tomax_client_wait_time_ms
. -
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.
Maximum period of time =
max_client_wait_time_ms
+max_local_query_time_ms
. -
Consider adjusting
max_local_query_time_ms
andmax_client_wait_time_ms
when high write workloads exist on tables that have continuous paging requests. -
If fewer threads exist than sessions (
max_threads
<max_concurrent_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
. -
If the client is slow in reading pages, try increasing the delay by adjusting
max_client_wait_time_ms
. Parameters: -
max_concurrent_sessions
Default:
60
. The maximum number of concurrent sessions. Additional sessions are rejected with an unavailable error. -
max_session_pages
Default:
4
. The maximum number of pages that can be buffered for each session. -
max_page_size_mb
Default:
8
. The maximum size of a page, in MB. If an individual CQL row is larger than this value, the page can be bigger than this value. -
max_client_wait_time_ms
Default:
20000
. The maximum time for the server to wait for the client to read from the socket. If exceeded, the session is aborted and the client receives an error. Settingmax_client_wait_time_ms
to a value too low may result in client side errors. -
max_local_query_time_ms
Default:
5000
. The maximum time for a local continuous query to run. When 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 whenmax_threads
<max_concurrent_sessions
. -
max_threads
Default:
24
. The number of threads dedicated to continuous paging sessions.
-