Table properties

A list of CQL table properties and their syntax.

CQL supports Cassandra table properties, such as comments and compaction options, listed in the following table.

In CQL commands, such as CREATE TABLE, add properties and their values in either the name-value pair or collection map format:

name = value [ AND name = value ] [ ... ]
{ name : value, name : value [, name : value ] [ ... ] }

Enclose strings in single quotation marks.

See CREATE TABLE for examples.

Table 1. CQL properties
CQL property Default Description
bloom_filter_fp_chance For SizeTieredCompactionStrategy : 0.01, for DateTieredCompactionStrategy : 0.1, for LeveledCompactionStrategy : 0.1 Desired false-positive probability for SSTable Bloom filters. See Bloom filter below.
caching Apache Cassandra 2.1:

ALL for keys

NONE for rows_per_partition

Optimizes the use of cache memory without manual tuning. See caching below.
comment N/A A human readable comment describing the table. See comments below.
compaction SizeTieredCompactionStrategy Sets the compaction strategy for the table. See compaction below.
compression LZ4Compressor The compression algorithm. Valid values are LZ4Compressor), SnappyCompressor, and DeflateCompressor. See compression below.
dclocal_read_repair_chance 0.1, The probability that a successful read operation triggers a read repair. Unlike the repair controlled by read_repair_chance, this repair is limited to replicas in the same DC as the coordinator. The value must be between 0 and 1. For details see read repairs below.
default_time_to_live 0

Set this property in MapReduce scenarios when you have no control of TTL. The value of this property is a number of seconds. If it is set, Cassandra applies a default TTL marker to each column in the table, set to this value. When the table TTL is exceeded, Cassandra tombstones the table.

Note: You can effectively delete any column TTLs in a table by setting the default_time_to_live to zero.
gc_grace_seconds 864000 [10 days] The number of seconds after data is marked with a tombstone (deletion marker) before it is eligible for garbage-collection. Cassandra will not execute hints or batched mutations on a tombstoned record within its gc_grace_period. The default value allows a great deal of time for Cassandra to maximize consistency prior to deletion. For details about decreasing this value, see garbage collection below.
min_index_interval, max_index_interval (Cassandra 2.1.x) min_index_interval 128 and max_index_interval 2048, or index_interval 128 To control the sampling of entries from the partition index, configure the sample frequency of the partition summary by changing these properties.
memtable_flush_period_in_ms 0 The number of milliseconds before Cassandra flushes memtables associated with this table.
min_index_interval The minimum gap between index entries in the index summary. A lower min_index_interval means the index summary contains more entries from the index, which allows Cassandra to search fewer index entries to execute a read. A larger index summary may also use more memory. The value for min_index_interval is the densest possible sampling of the index.
max_index_interval If the total memory usage of all index summaries reaches this value, Cassandra decreases the index summaries for the coldest SSTables to the maximum set by max_index_interval. The max_index_interval is the sparsest possible sampling in relation to memory pressure.
read_repair_chance 0.0 The probability that a successful read operation will trigger a read repair.of read repairs being invoked. Unlike the repair controlled by dc_local_read_repair_chance, this repair is not limited to replicas in the same DC as the coordinator. The value must be between 0 and 1. For details see read repairs below.
speculative_retry 99percentile Overrides normal read timeout when read_repair_chance is not 1.0, sending another request to read. See speculative retry below.

Bloom filter

The Bloom filter sets the false-positive probability for SSTable Bloom filters. When a client requests data, Cassandra uses the Bloom filter to check if the row exists before doing disk I/O. Bloom filter property value ranges from 0 to 1.0. Lower Bloom filter property probabilities result in larger Bloom filters that use more memory. The effects of the minimum and maximum values:
  • 0: Enables the unmodified, effectively the largest possible, Bloom filter.
  • 1.0: Disables the Bloom filter.
Recommended setting: 0.1. A higher value yields diminishing returns.

caching

Caching optimizes the use of cache memory by a table without manual tuning. Cassandra weighs the cached data by size and access frequency. Coordinate this setting with the global caching properties in the cassandra.yaml file. See Cassandra 3.0 documentation.

Configure the cache by creating a property map of values for the caching property. Options:
  • keys: ALL or NONE
  • rows_per_partition: number of CQL rows (N), NONE, or ALL
According to the rows_per_partition value, Cassandra caches only the first N rows in a partition, as determined by the clustering order.
For example:
CREATE TABLE DogTypes (
  block_id uuid,
  species text,
  alias text,
  population varint,
  PRIMARY KEY (species, block_id)
) WITH caching = { 'keys' : 'NONE', 'rows_per_partition' : '120' };

comments

Use comments to document CQL statements in your application code. Single line comments can begin with a double dash (--) or a double slash (//) and extend to the end of the line. Enclose multi-line comments in /* and */ characters.

compaction

The compaction property defines the compaction strategy class for this table. Choose the compaction strategy that best fits your data and environment.
  • SizeTieredCompactionStrategy (STCS): The default compaction strategy. This strategy triggers a minor compaction when there are a number of similar sized SSTables on disk as configured by the table subproperty min_threshold. A minor compaction does not involve all the tables in a keyspace. See STCS compaction subproperties.
  • DateTieredCompactionStrategy (DTCS): This strategy is particularly useful for time series data. DateTieredCompactionStrategy stores data written within a certain period of time in the same SSTable. For example, Cassandra stores your last hour of data in one SSTable time window, and the next 4 hours of data in another time window, and so on. Cassandra performs compaction when the number of SSTables in those windows reaches min_threshold (4 by default). The most common queries for time series workloads retrieve the last hour/day/month of data. Cassandra can limit SSTables returned to those having the relevant data. Also, Cassandra can store data set to expire using TTL in an SSTable with other data scheduled to expire at approximately the same time. Cassandra can then drop the SSTable without doing any compaction. Also see DTCS compaction subproperties.
  • LeveledCompactionStrategy (LCS): Creates SSTables of a fixed, relatively small size (160 MB by default) that are grouped into levels. Within each level, SSTables are guaranteed to be non-overlapping. LCS uses STCS for the first level, called level 0 (L0). Each level beyond L0 (L1, L2 and so on) is 10 times as large as the previous level. Disk I/O is more uniform and predictable on higher than on lower levels as SSTables are continuously being compacted into progressively larger levels. At each level, row keys are merged into non-overlapping SSTables. This can improve performance for reads, because Cassandra can determine which SSTables in each level to check for the existence of row key data. This compaction strategy is modeled after Google's LevelDB implementation. See LCS compaction subproperties.

Hybrid (leveled and size-tiered) compaction improvements to the leveled compaction strategy reduce the performance overhead on read operations when compaction cannot keep pace with write-heavy workload. If Cassandra cannot keep pace with the workload when using the LCS, the compaction strategy switches to STCS until Cassandra catches up. For this reason, it is a best practice to configure the max_threshold subproperty for a table to use when the switch occurs.

You can specify a custom strategy. Use the full class name as a string constant.

garbage collection

The default value or gc_grace_seconds is 864000 seconds (10 days). In a single-node cluster, this property can safely be set to zero. This value can also be reduced for tables whose data will not be explicitly deleted — for example, tables containing only data with TTL set, or tables with default_time_to_live set. However, if you lower the gc_grace_seconds value, consider its interaction with these operations:
  • hint replays — When a node goes down and then comes back up, other nodes replay the write operations (called hints) that are queued for that node while it was unresponsive. Cassandra does not replay hints older than gc_grace_seconds after creation. The max_hint_window_in_ms setting in the cassandra.yaml file sets the time limit (3 hours by default) for collecting hints for the unresponsive node.
  • batch replays — Like hint queues, batch operations store database mutations that are replayed in sequence. As with hints, Cassandra does not replay a batched mutation until gc_grace_seconds after it was created. If your application uses batch operations, consider the possibility that decreasing gc_grace_seconds increases the chance that a batched write operation may restore deleted data. The batchlog_replay_throttle_in_kb property in the cassandra.yaml file gives some control of the batch replay process. The most important factors, however, are the size and scope of the batches you use.

For more details about deletes, see About deletes or How is data deleted? in the Apache Cassandra 3.0 documentation.

compression

Configure compression when creating or altering a table by setting compaction property to LZ4Compressor, SnappyCompressor, or DeflateCompressor. To disable compression, use an empty string (" "), as shown in the example of how to use subproperties. Choosing the right compressor depends on your requirements for space savings over read performance. LZ4 is fastest to decompress, followed by Snappy, then by Deflate. Compression effectiveness is inversely correlated with decompression speed. The extra compression from Deflate or Snappy does not offset the decreased performance for general-purpose workloads, but these compresssors may work for archival data. You can also implement a custom compression class using the org.apache.cassandra.io.compress.ICompressor interface. To specify a custom compressor class, add the full class name enclosed in single quotation marks as the value for the compaction property. You can configure compression further using Cassandra's compression subproperties.

read repairs

Cassandra performs read repair whenever a read reveals inconsistencies among replicas. You can also configure Cassandra to perform read repair after a completely consistent read. Cassandra compares and coordinates all replicas, even those that were not accessed in the successful read. The probability that a consistent read of a table triggers a read repair is set by dclocal_read_repair_chance and read_repair_chance. The first of these properties sets the probability for a read repair that is confined to the same datacenter as the coordinator node. The second property sets the probability for a read repair across all datacenters that contain matching replicas. This cross-datacenter operation is much more resource-intensive than the local operation.

Recommendations: if the table is for time series data, both properties can be set to 0 (zero). For other tables, the more performant strategy is to set dc_local_read_repair_chance to 0.1 and read_repair_chance to 0. If you want to use read_repair_chance, set this property to 0.1.

speculative retry

Use the speculative retry property to configure rapid read protection. In a normal read, Cassandra sends data requests to just enough replica nodes to satisfy the consistency level. In rapid read protection, Cassandra sends out extra read requests to other replicas, even after the consistency level has been met. The speculative retry property specifies the trigger for these extra read requests.

  • ALWAYS: Send extra read requests to all other replicas after every read.
  • Xpercentile: Cassandra constantly tracks each table's typical read latency (in milliseconds). If you set speculative retry to Xpercentile, Cassandra sends redundant read requests if the coordinator has not received a response after X percent of the table's typical latency time.
  • Nms: Send extra read requests to all other replicas if the coordinator node has not received any responses within N milliseconds.
  • NONE: Do not send extra read requests after any read.
For example:
ALTER TABLE users WITH speculative_retry = '10ms';
Or:
ALTER TABLE users WITH speculative_retry = '99percentile';