The cassandra.yaml configuration file
The cassandra.yaml file is the main configuration file for Cassandra.
The cassandra.yaml file is the main configuration file for Cassandra.
- Cassandra package installations: /etc/cassandra
- Cassandra tarball installations: install_location/conf
The configuration properties are grouped into the following sections:
- Quick start
The minimal properties needed for configuring a cluster.
- Commonly
used
Properties most frequently used when configuring Cassandra.
- Performance tuning
Tuning performance and system resource utilization, including commit log, compaction, memory, disk I/O, CPU, reads, and writes.
- Advanced
Properties for advanced users or properties that are less commonly used.
- Security
Server and client security settings.
Quick start properties
The minimal properties needed for configuring a cluster.
Related information: Initializing a multiple node cluster (single datacenter) and Initializing a multiple node cluster (multiple datacenters).
- 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 Cassandra binds to for connecting to other
Cassandra nodes. Set this parameter or listen_interface, not both. You must
change the default setting for multiple nodes to
communicate:
- Generally set to empty. If the node is properly configured (host name, name resolution, and so on), Cassandra uses InetAddress.getLocalHost() to get the local address from the system.
- For a single node cluster, you can use the default setting (localhost).
- If Cassandra can't find the correct address, you must specify the IP address or host name.
- Never specify 0.0.0.0; it is always wrong.
- listen_interface
- (Default: eth0)note The interface that Cassandra binds to for connecting to other Cassandra nodes. Interfaces must correspond to a single address, IP aliasing is not supported. See listen_address.
If you have changed any of the default directories during installation, make sure you have root access and set these properties:
- commitlog_directory
- The directory where the commit log is stored. Default
locations:
- Package installations: /var/lib/cassandra/commitlog
- Tarball installations: install_location/data/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 for is acceptable for this purpose.
- data_file_directories
- The directory location where table data (SSTables) is
stored. Cassandra distributes data evenly across the
location, subject to the granularity of the
configured compaction strategy. Default locations:
- Package installations: /var/lib/cassandra/data
- Tarball installations: install_location/data/data
As a production best practice, use RAID 0 and SSDs.
- saved_caches_directory
- The directory location where table key and row caches
are stored. Default location:
- Package installations: /var/lib/cassandra/saved_caches
- Tarball installations: install_location/data/saved_caches
Commonly used properties
Properties most frequently used when configuring Cassandra.
Before starting a node for the first time, you should carefully evaluate your requirements.
- 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, but can be inspected using JMX.
- stop_commit
Shut down the commit log, letting writes collect but continuing to service reads (as in pre-2.0.5 Cassandra).
- ignore
Ignore fatal errors and let the batches fail.
- die
- disk_failure_policy
- (Default: stop) Sets how Cassandra
responds to disk failure. Recommend settings are
stop or
best_effort.
- 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 means you will see obsolete data at consistency level of ONE.
- ignore
Ignores fatal errors and lets the requests fail; all file system errors are logged but otherwise ignored. Cassandra acts as in versions prior to 1.2.
Related information: Handling Disk Failures In Cassandra 1.2 blog and Recovering from a single disk failure using JBOD.
- die
- endpoint_snitch
- (Default:
org.apache.cassandra.locator.SimpleSnitch)
Set to a class that implements the
IEndpointSnitch. Cassandra uses
snitches for locating nodes and routing requests.
- SimpleSnitch
Use for single-datacenter deployments or single-zone in public clouds. Does not recognize datacenter or rack information. It treats strategy order as proximity, which can improve cache locality when disabling read repair.
- GossipingPropertyFileSnitch
Recommended for production. The rack and datacenter for the local node are defined in the cassandra-rackdc.properties file and propagated to other nodes via gossip. To allow migration from the PropertyFileSnitch, it uses the cassandra-topology.properties file if it is present.
- PropertyFileSnitch
Determines proximity by rack and datacenter, which are explicitly configured in the cassandra-topology.properties file.
- Ec2Snitch
For EC2 deployments in a single region. Loads region and availability zone information from the EC2 API. The region is treated as the datacenter and the availability zone as the rack. Uses only private IPs. Subsequently it does not work across multiple regions.
- Ec2MultiRegionSnitch
Uses public IPs 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, Cassandra 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. This snitch is best used as an example for writing a custom snitch class (unless this happens to match your deployment conventions).
Related information: Snitches
- SimpleSnitch
- rpc_address
- (Default: localhost) The listen address
for client connections (Thrift RPC service and
native transport).Valid values are:
- unset:
Resolves the address using the hostname configuration of the node. If left unset, the hostname must resolve to the IP address of this node using /etc/hostname, /etc/hosts, or DNS.
-
0.0.0.0:
Listens on all configured interfaces, but you must set the broadcast_rpc_address to a value other than 0.0.0.0.
- IP address
- hostname
Related information: Network
- unset:
- rpc_interface
- (Default: eth1)note The listen address for client connections. Interfaces must correspond to a single address, IP aliasing is not supported. See rpc_address.
- rpc_interface_prefer_ipv6
- (Default: false) By default, if an interface has an ipv4 and an ipv6 address, the first ipv4 address will be used. If set to true, the first ipv6 address will be used.
- seed_provider
- The addresses of hosts deemed contact points. Cassandra
nodes use the -seeds list to
find each other and learn the topology of the ring.
- class_name (Default:
org.apache.cassandra.locator.SimpleSeedProvider)
The class within Cassandra 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. When running multiple nodes, you must change the list from the default value. In multiple data-center clusters, the seed list should include at least one node from each datacenter (replication group). More than a single seed node per datacenter is recommended for fault tolerance. Otherwise, gossip has to communicate with another datacenter when bootstrapping a node. 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 multiple node cluster (single datacenter) and Initializing a multiple node cluster (multiple datacenters).
- class_name (Default:
org.apache.cassandra.locator.SimpleSeedProvider)
- compaction_throughput_mb_per_sec
- (Default: 16) Throttles compaction to
the specified total throughput across the node. The
faster you insert data, the faster you need to
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) Logs a warning when compaction partitions larger than the set value.
- memtable_heap_space_in_mb
- (Default: 1/4 of heap size)note
- memtable_offheap_space_in_mb
- (Default: 1/4 of heap size)note
- concurrent_reads
- (Default: 32)note For workloads with more data than can fit in memory, the bottleneck is reads fetching data from disk. Setting 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 Cassandra are rarely I/O bound, so the ideal number of concurrent writes depends on the number of CPU cores in your system. 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) .
- incremental_backups
- (Default: false) Backs up data updated
since the last snapshot was taken. When enabled,
Cassandra 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) Enable or disable
taking a snapshot before each compaction. This
option is useful to back up data when there is a
data format change. Be careful using this option
because Cassandra does not clean up older snapshots
automatically.
Related information: Configuring compaction
- phi_convict_threshold
- (Default: 8)note Adjusts the sensitivity of
the failure detector on an exponential scale.
Generally this setting never needs adjusting.
Related information: 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.
- commitlog_sync
- (Default: periodic) The method that
Cassandra uses to acknowledge writes in
milliseconds:
-
periodic: (Default:
10000 milliseconds [10 seconds])
Used with commitlog_sync_period_in_ms to control how often the commit log is synchronized to disk. Periodic syncs are acknowledged immediately.
- batch: (Default:
disabled)note
Used with commitlog_sync_batch_window_in_ms (Default: 2 ms), which is the maximum length of time that queries may be batched together.
Related information: Durability
-
periodic: (Default:
10000 milliseconds [10 seconds])
- commitlog_segment_size_in_mb
- (Default: 32MB) Sets the size of the
individual commitlog file segments. A commitlog
segment may be archived, deleted, or recycled after
all its data has been flushed to SSTables. This
amount of 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.
This property determines the maximum mutation size, defined as half the segment size. If a mutation's size exceeds the maximum mutation size, 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.
Related information: Commit log archive configuration
- commitlog_total_space_in_mb
- (Default: 32MB for 32-bit JVMs,
8192MB for 64-bit JVMs)note Total space used for
commitlogs. If the used space goes above this value,
Cassandra rounds up to the next nearest segment
multiple and flushes memtables to disk for the
oldest commitlog segments, removing those log
segments. This reduces the amount of data to replay
on start-up, and prevents infrequently-updated
tables from indefinitely keeping commitlog segments.
A small total commitlog space tends to cause more
flush activity on less-active tables.
Related information: Configuring memtable throughput
Related information: Configurating Compaction
- concurrent_compactors
- (Default: Smaller of number of disks or number of cores,
with a minimum of 2 and a maximum of 8 per CPU
core)note Sets 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
mitigating the tendency of small SSTables to
accumulate during a single long-running compaction.
If your data directories are backed by SSD, increase
this value to the number of cores. If compaction
running too slowly or too fast, adjust compaction_throughput_mb_per_sec first.
Note: Increasing concurrent compactors impacts the available disk storage because concurrent compactions happen in parallel, especially for STCS. Ensure that adequate disk space is available when increasing this configuration.
- sstable_preemptive_open_interval_in_mb
- (Default: 50MB) When compacting, the replacement opens SSTables before they are completely written and uses 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_allocation_type
- (Default: heap_buffers) Specify the way
Cassandra allocates and manages memtable memory. See
Off-heap memtables
in Cassandra 2.1. Options are:
- 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.
- heap_buffers
- memtable_cleanup_threshold
- (Default: 1/(memtable_flush_writers + 1))note. Ratio used for automatic
memtable flush. Casssandra adds memtable_heap_space_in_mb to memtable_offheap_space_in_mb and
multiplies the total by
memtable_cleanup_threshold to get
a space amount in MB. When the total amount of
memory being used by all non-flushing memtables
exceeds this amount, Casandra flushes the largest
memtable to disk.
For example, consider a system in which the total of memtable_heap_space_in_mb and memtable_offheap_space_in_mb is 1000, and memtable_cleanup_threshold is
0.50
. The "memtable_cleanup" amount is500
MB. This system has two memtables: Memtable A (150MB) and Memtable B (350MB) . When either memtable increases, the total space they use exceeds 500MB. When this happens, Cassandra flushes the Memtable B to disk.A larger value for memtable_cleanup_threshold means larger flushes, less frequent flushes and potentially less compaction activities, but also less concurrent flush activity, which can make it difficult to keep the 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 in cassandra.yaml is
0.11
, which works if the node has many disks or if you set the node's memtable_flush_writers to8
. As another example, if the node uses a single SSD, the value for memttable_cleanup_threshold computes to0.33
, based on the minimum memtable_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.
- memtable_flush_writers
- (Default: Smaller of number of disks or number of cores with a minimum of 2 and a maximum of 8)note Sets 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 SSD, increase this setting to the number of cores.
- 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 make this setting too large because key cache will be overwhelmed. 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, Cassandra may need to use more than this amount of memory.
- index_summary_resize_interval_in_minutes
- (Default: 60 minutes) How frequently index summaries should be re-sampled. This is done periodically to redistribute memory from the fixed-size pool to SSTables proportional their recent read rates. To disable, set to -1. This leaves existing index summaries at their current sampling level.
- stream_throughput_outbound_megabits_per_sec
- (Default: 200 Mbps)note Throttles all outbound streaming file transfers on a node to the specified throughput. Cassandra does mostly sequential I/O when streaming data during bootstrap or repair, which can lead to saturating the network connection and degrading client (RPC) performance.
- inter_dc_stream_throughput_outbound_megabits_per_sec
- (Default: unset)note Throttles all streaming file transfer between the datacenters. This setting allows throttles streaming throughput betweens data centers in addition to throttling all network stream traffic as configured with stream_throughput_outbound_megabits_per_sec.
- trickle_fsync
- (Default: false) When doing sequential writing, enabling this option tells fsync to force the operating system to flush the dirty buffers at a set interval trickle_fsync_interval_in_kb. Enable this parameter to avoid sudden dirty buffer flushing from impacting read latencies. Recommended to use on SSDs, but not on HDDs.
- trickle_fsync_interval_in_kb
- (Default: 10240). Sets the size of the fsync in kilobytes.
Advanced properties
Properties for advanced users or properties that are less commonly used.
- auto_bootstrap
- (Default: true) This setting has been
removed from default configuration. It makes new
(non-seed) nodes automatically migrate the right
data to themselves. When initializing a fresh
cluster without data, add
auto_bootstrap:
false.
Related information: Initializing a multiple node cluster (single datacenter) and Initializing a multiple node cluster (multiple datacenters).
- batch_size_warn_threshold_in_kb
- (Default: 5KB per batch) Log WARN on any batch size exceeding this value in kilobytes. Caution should be taken on increasing the size of this threshold as it can lead to node instability.
- broadcast_address
- (Default: listen_address)note
The IP address a node
tells other nodes in the cluster to contact it by.
It allows public and private address to be
different. For example, use the
broadcast_address parameter
in topologies where not all nodes have access to
other nodes by their private IP
addresses.
If your Cassandra cluster is deployed across multiple Amazon EC2 regions and you use the Ec2MultiRegionSnitch, set the broadcast_address to public IP address of the node and the listen_address to the private IP. See Ec2MultiRegionSnitch.
- initial_token
- (Default: disabled) Used in the single-node-per-token architecture, where a node owns exactly one contiguous range in the ring space. Setting this property overrides num_tokens.
- num_tokens
- (Default: 256) note
Defines the
number of tokens randomly assigned to this node on
the ring when using virtual nodes
(vnodes). The more tokens, relative to other
nodes, the larger the proportion of data that the
node stores. Generally all nodes should have the
same number of tokens assuming equal hardware
capability. The recommended value is
256. If unspecified
(
#num_tokens
), Cassandra uses 1 (equivalent to#num_tokens : 1
) for legacy compatibility and uses the initial_token setting. - partitioner
- (Default:
org.apache.cassandra.dht.Murmur3Partitioner)
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.Cassandra provides the following partitioners for backwards compatibility:
- RandomPartitioner
- ByteOrderedPartitioner
- OrderPreservingPartitioner (deprecated)
Related information: Partitioners
- storage_port
- (Default: 7000) The port for inter-node communication.
- auto_snapshot
- (Default: true) Enable or disable whether a snapshot is taken of the data before keyspace truncation or dropping of tables. To prevent data loss, using the default setting is strongly advised. If you set to false, you will lose data on truncation or drop.
When creating or modifying tables, you 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. Cassandra 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 seconds [4 hours]) Duration in seconds that keys are saved in cache. Caches are saved to saved_caches_directory. Saved caches greatly improve cold-start speeds and has relatively little effect on I/O.
- key_cache_size_in_mb
- (Default: empty) A global cache setting for tables. It
is the maximum size of the key cache in memory. When
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.
- 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- disabled) Maximum size of the row cache in memory. Row cache can save more time than key_cache_size_in_mb, but 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- disabled) Duration in seconds that rows are saved in cache. Caches are saved to saved_caches_directory. This setting has limited use as described in row_cache_size_in_mb.
Counter cache helps to reduce counter locks' contention for hot counter cells. In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before write entirely. With RF > 1 a counter cache hit will still help to reduce the duration of the lock hold, helping with hot counter cell updates, but will 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's relatively cheap.
- counter_cache_size_in_mb
- (Default value: empty)note When no value is specified a minimum of 2.5% of Heap or 50MB. If you perform counter deletes and rely on low gc_grace_seconds, you should disable the counter cache. To disable, set to 0.
- counter_cache_save_period
- (Default: 7200 seconds [2 hours]) Duration after which Cassandra should save the counter cache (keys only). Caches are saved to saved_caches_directory.
- counter_cache_keys_to_save
- (Default value: disabled)note Number of keys from the counter cache to save. When disabled all keys are saved.
When executing a scan, within or across a partition, tombstones must be kept in memory to allow returning them to the coordinator. The coordinator uses them to ensure 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. Additionally, 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 maximum number of tombstones a query can scan before warning.
- tombstone_failure_threshold
- (Default: 100000) The maximum number of tombstones a query can scan before aborting.
- range_request_timeout_in_ms
- (Default: 10000 milliseconds) The time that the coordinator waits for sequential or index scans to complete.
- read_request_timeout_in_ms
- (Default: 5000 milliseconds) The time that the coordinator waits for read operations to complete.
- counter_write_request_timeout_in_ms
- (Default: 5000 milliseconds) The time that the coordinator waits for counter writes to complete.
- cas_contention_timeout_in_ms
- (Default: 1000 milliseconds) The time that the coordinator continues to retry a CAS (compare and set) operation that contends with other proposals for the same row.
- truncate_request_timeout_in_ms
- (Default: 60000 milliseconds) The time that the coordinator waits for truncates (remove all data from a table) to complete. The long default value allows for a snapshot to be taken before removing the data. If auto_snapshot is disabled (not recommended), you can reduce this time.
- write_request_timeout_in_ms
- (Default: 2000 milliseconds) The time
that the coordinator waits for write operations to
complete.
Related information: About hinted handoff writes
- request_timeout_in_ms
- (Default: 10000 milliseconds) The
default time for other, miscellaneous
operations.
Related information: About hinted handoff writes
- cross_node_timeout
- (Default: false) Enable or disable
operation timeout information exchange between nodes
(to accurately measure request timeouts). If
disabled Cassandra assumes the request are forwarded
to the replica instantly by the coordinator, which
means that under overload conditions extra time is
required for processing already-timed-out requests..
CAUTION: Before enabling this property make sure NTP (network time protocol) is installed and the times are synchronized between the nodes.
- internode_send_buff_size_in_bytes
- (Default: N/A)note Sets the sending socket
buffer size in bytes for inter-node calls.When setting this parameter and internode_recv_buff_size_in_bytes, the buffer size is limited by net.core.wmem_max. When unset, buffer size is defined by net.ipv4.tcp_wmem. See man tcp and:
- /proc/sys/net/core/wmem_max
- /proc/sys/net/core/rmem_max
- /proc/sys/net/ipv4/tcp_wmem
- /proc/sys/net/ipv4/tcp_wmem
- internode_recv_buff_size_in_bytes
- (Default: N/A)note Sets the receiving socket buffer size in bytes for inter-node calls.
- internode_compression
- (Default: all) Controls whether traffic
between nodes is compressed. The valid values are:
- all
Compresses all traffic.
- dc
Compresses traffic between data centers.
- none
No compression.
- all
- inter_dc_tcp_nodelay
- (Default: false) Enable or disable tcp_nodelay for inter-datacenter communication. When disabled larger, but fewer, network packets are sent. This reduces overhead from the TCP protocol itself. However, if cross data-center responses are blocked, it will increase latency.
- streaming_socket_timeout_in_ms
- (Default: 3600000 - 1 hour)note Enable or disable socket timeout for streaming operations. When a timeout occurs during streaming, streaming is retried from the start of the current file. Avoid setting this value too low, as it can result in a significant amount of data re-streaming.
- start_native_transport
- (Default: true) Enable or disable the native transport server. Uses the same address as the rpc_address, but the port is different from the rpc_port. See native_transport_port.
- native_transport_port
- (Default: 9042) Port on which 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 and differs as follows:
- Default is different (128 versus unlimited).
- No corresponding native_transport_min_threads.
- Idle threads are stopped after 30 seconds.
- native_transport_max_frame_size_in_mb
- (Default: 256MB) The maximum size of allowed frame. Frame (requests) larger than this are rejected as invalid.
- native_transport_max_concurrent_connections
- (Default: -1) Specifies the maximum number of concurrent client connections. The default value of -1 means unlimited.
- native_transport_max_concurrent_connections_per_ip
- (Default: -1) Specifies the maximum number of concurrent client connections per source IP address. The default value of -1 means unlimited.
Settings for configuring and tuning client connections.
- broadcast_rpc_address
- (Default: unset)note RPC address to broadcast to drivers and other Cassandra nodes. This cannot be set to 0.0.0.0. If blank, it is set to the value of the rpc_address or rpc_interface. If rpc_address or rpc_interfaceis set to 0.0.0.0, this property must be set.
- rpc_port
- (Default: 9160) Thrift port for client connections.
- start_rpc
- (Default: true) Starts the Thrift RPC server.
- rpc_keepalive
- (Default: true) Enable or disable keepalive on client connections (RPC or native).
- rpc_max_threads
- (Default: unlimited)note Regardless of your choice of RPC server (rpc_server_type), the number of maximum requests in the RPC thread pool dictates how many concurrent requests are possible. However, if you are using the parameter sync in the rpc_server_type, it also dictates the number of clients that can be connected. For a large number of client connections, this could cause excessive memory usage for the thread stack. Connection pooling on the client side is highly recommended. Setting a maximum thread pool size acts as a safeguard against misbehaved clients. If the maximum is reached, Cassandra blocks additional connections until a client disconnects.
- rpc_min_threads
- (Default: 16)note Sets the minimum thread pool size for remote procedure calls.
- rpc_recv_buff_size_in_bytes
- (Default: N/A)note Sets the receiving socket buffer size for remote procedure calls.
- rpc_send_buff_size_in_bytes
- (Default: N/A)note Sets the sending socket buffer size in bytes for remote procedure calls.
- rpc_server_type
- (Default: sync) Cassandra provides
three options for the RPC server. On Linux, sync and
hsha performance is about the same, but hsha 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, 180KB 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 and thus scales well to many clients. The RPC requests are synchronous (one thread per active request).Note: When selecting 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.
- sync: (Default One thread per
Thrift connection.)
Settings to handle poorly performing or failing nodes.
- gc_warn_threshold_in_ms
- (Default: 1000) Any GC pause longer
than this interval is logged at the WARN level. (By
default, Cassandra logs any GC pause greater than
200 ms at the INFO level.)
Additional information: Configuring logging.
- dynamic_snitch_badness_threshold
- (Default: 0.1) Sets 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 over it. A value of 0.2 means Cassandra 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). If the value of this parameter is greater than zero and read_repair_chance is less than 1.0, cache capacity is maximized across the nodes.
- dynamic_snitch_reset_interval_in_ms
- (Default: 600000 milliseconds) Time interval to reset all node scores, which allows a bad node to recover.
- dynamic_snitch_update_interval_in_ms
- (Default: 100 milliseconds) The time interval for how often the snitch calculates node scores. Because score calculation is CPU intensive, be careful when reducing this interval.
- hinted_handoff_enabled
- (Default: true) Enable or disable
hinted handoff. To enable per datacenter, add data
center list. For example:
hinted_handoff_enabled: DC1,DC2
. A hint indicates that the write needs to be replayed to an unavailable node. Where Cassandra writes the hint depends on the version:- Prior to 1.0
Writes to a live replica node.
- 1.0 and later
Writes to the coordinator node.
Related information: About hinted handoff writes
- Prior to 1.0
- hinted_handoff_throttle_in_kb
- (Default: 1024) Maximum throttle 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.Note: When applying this limit, Cassandra computes the hint transmission rate based on the uncompressed hint size, even if internode_compression is enabled.
- max_hint_window_in_ms
- (Default: 10800000 milliseconds [3
hours]) Maximum amount of time hints are generated
for an unresponsive node. After this interval, new
hints are no longer generated until the node is back
up and responsive. If the node goes down again, a
new interval begins. 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: Failure detection and recovery
- max_hints_delivery_threads
- (Default: 2) Number of threads with which to deliver hints. In multiple data-center deployments, consider increasing this number because cross data-center handoff is generally slower.
- batchlog_replay_throttle_in_kb
- (Default: 1024KB per second) Total maximum throttle. Throttling is reduced proportionally to the number of nodes in the cluster.
Settings to handle incoming client requests according to a defined policy. If you need to use these properties, your nodes are overloaded and dropping requests. It is recommended that you add more nodes and not try to prioritize requests.
- request_scheduler
- (Default:
org.apache.cassandra.scheduler.NoScheduler)
Defines a 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 are:
- org.apache.cassandra.scheduler.NoScheduler
No scheduling takes place.
- org.apache.cassandra.scheduler.RoundRobinScheduler
Round robin of client requests to a node with a separate queue for each request_scheduler_id property.
- A Java class that implements the RequestScheduler interface.
- org.apache.cassandra.scheduler.NoScheduler
- request_scheduler_id
- (Default: keyspace)note An identifier on which to perform request scheduling. Currently the only valid value is keyspace. See weights.
- request_scheduler_options
- (Default: disabled) Contains a list of
properties that define configuration options for
request_scheduler:
- throttle_limit
The number of in-flight requests per client. Requests beyond 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 are handled during each turn of the RoundRobin.
- weights: (Default:
Keyspace:
1)
Takes a list of keyspaces. It sets how many requests are handled during each turn of the RoundRobin, based on the request_scheduler_id.
- throttle_limit
Legacy API for older clients. CQL is a simpler and better API for Cassandra.
- 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.
- thrift_max_message_length_in_mb
- (Default: 16) The maximum length of a Thrift message in megabytes, including all fields and internal Thrift overhead (1 byte of overhead for each frame). Message length is usually used in conjunction with batches. A frame length greater than or equal to 24 accommodates a batch with four inserts, each of which is 24 bytes. The required message length is greater than or equal to 24+24+24+24+4 (number of frames).
Security properties
Server and client security settings.
- authenticator
- (Default: AllowAllAuthenticator) The
authentication backend. It implements IAuthenticator
for identifying users. The available authenticators
are:
- AllowAllAuthenticator:
Disables authentication; no checks are performed.
- PasswordAuthenticator
Authenticates users with user names and hashed passwords stored in the system_auth.credentials table. If you use the default, 1, and the node with the lone replica goes down, you will not be able to log into the cluster because the system_auth keyspace was not replicated.
Related information: Internal authentication
- AllowAllAuthenticator:
- internode_authenticator
- (Default: enabled)note Internode authentication backend. It implements org.apache.cassandra.auth.AllowAllInternodeAuthenticator to allows or disallow connections from peer nodes.
- authorizer
- (Default: AllowAllAuthorizer) The
authorization backend. It implements IAuthenticator
to limit access and provide permissions. The
available authorizers are:
- AllowAllAuthorizer
Disables authorization; allows any action to any user.
- CassandraAuthorizer
Stores permissions in system_auth.permissions table. If you use the default, 1, and the node with the lone replica goes down, you will not be able to log into the cluster because the system_auth keyspace was not replicated.
Related information: Object permissions
- AllowAllAuthorizer
- permissions_validity_in_ms
- (Default: 2000) How long permissions in
cache remain valid. Depending on the authorizer,
such as CassandraAuthorizer,
fetching permissions can be resource intensive. This
setting disabled when set to 0 or
when AllowAllAuthorizer is set.
Related information: Object permissions
- permissions_update_interval_in_ms
- (Default: same value as permissions_validity_in_ms)note Refresh interval for permissions cache (if enabled). After this interval, cache entries become eligible for refresh. On next access, an async reload is scheduled and the old value is returned until it completes. If permissions_validity_in_ms , then this property must benon-zero.
- server_encryption_options
- Enable or disable inter-node encryption. You must also
generate keys and provide the appropriate key and
trust store locations and passwords. No custom
encryption options are currently enabled. The
available options are:
- internode_encryption:
(Default: none) Enable or disable
encryption of inter-node 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 if
running in (Federal Information Processing
Standard) FIPS 140 compliant mode. The available
inter-node options are:
- 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).
- all
- 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.
The advanced settings are:
- protocol: (Default: TLS)
- algorithm: (Default: SunX509)
- store_type: (Default: JKS)
- cipher_suites: (Default: TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA)
- require_client_auth:
(Default: false)
Enables or disables certificate authentication.
Related information: Node-to-node encryption
- internode_encryption:
(Default: none) Enable or disable
encryption of inter-node 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 if
running in (Federal Information Processing
Standard) FIPS 140 compliant mode. The available
inter-node options are:
- client_encryption_options
- Enable or disable client-to-node encryption. You must
also generate keys and provide the appropriate key
and trust store locations and passwords. No custom
encryption options are currently enabled. The
available options are:
- enabled: (Default:
false)
To enable, set to true.
- 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. (Available starting with Cassandra 1.2.3.)
- truststore: (Default:
conf/.truststore)
Set if require_client_auth is true.
- truststore_password:
<truststore_password>
Set if require_client_auth is true.
The advanced settings are:
- protocol: (Default: TLS)
- algorithm: (Default: SunX509)
- store_type: (Default: JKS)
- cipher_suites: (Default: TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA)
Related information: Client-to-node encryption
- enabled: (Default:
false)
- ssl_storage_port
- (Default: 7001) The SSL port for encrypted communication. Unused unless enabled in encryption_options.