Table properties
CREATE KEYSPACE cycling WITH replication = {'class' : 'NetworkTopologyStrategy', 'dc1' : 2, 'dc2' : 2};
USE cycling;
CREATE TABLE cyclist_name ( id UUID PRIMARY KEY, lastname text, firstname text );
DESCRIBE TABLE cyclist_name;
CREATE TABLE cycling.cyclist_name (
id uuid PRIMARY KEY,
firstname text,
lastname text
) WITH bloom_filter_fp_chance = 0.01
AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}
AND comment = ''
AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32', 'min_threshold': '4'}
AND compression = {'chunk_length_in_kb': '64', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}
AND crc_check_chance = 1.0
AND dclocal_read_repair_chance = 0.1
AND default_time_to_live = 0
AND gc_grace_seconds = 864000
AND max_index_interval = 2048
AND memtable_flush_period_in_ms = 0
AND min_index_interval = 128
AND read_repair_chance = 0.0
AND speculative_retry = '99PERCENTILE';
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.
ALTER TABLE cyclist_name WITH bloom_filter_fp_chance = 1.0;
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.
There are three caches built into Cassandra:
- key cache
- row cache
- counter cache
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.
ALTER TABLE cyclist_name 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.
ALTER TABLE cyclist_name WITH comment = 'a simple primary key';
compaction
The compaction property defines the compaction strategy class for this table. Choose the compaction strategy that best fits your data and environment.
Compaction subproperties
Disable background compaction
ALTER TABLE cyclist_name WITH COMPACTION = {'class': 'SizeTieredCompactionStrategy', 'enabled': 'false'};
Enabling extended compaction logging
Set compaction thresholds
ALTER TABLE cyclist_name WITH compaction = {'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 1, 'max_threshold' : 1000000000 };
ConfigurationException: Min compaction threshold cannot be less than 2 (got 1)
ALTER TABLE cyclist_name WITH compaction = {'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 2, 'max_threshold' : 1000000000 };
garbage collection
The default value is 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.
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 (" ").
Compression subproperties
ALTER TABLE cyclist_name WITH COMPRESSION = {'sstable_compression': ''};
CREATE TABLE cycling.cyclist_name (
id uuid PRIMARY KEY,
firstname text,
lastname text
) WITH bloom_filter_fp_chance = 0.1
AND caching = {'keys': 'NONE', 'rows_per_partition': '120'}
AND comment = 'a simple primary key'
AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '1000000000', 'min_threshold': '2'}
AND compression = {'enabled': 'false'}
AND crc_check_chance = 1.0
AND dclocal_read_repair_chance = 0.1
AND default_time_to_live = 0
AND gc_grace_seconds = 864000
AND max_index_interval = 2048
AND memtable_flush_period_in_ms = 0
AND min_index_interval = 128
AND read_repair_chance = 0.0
AND speculative_retry = '99PERCENTILE';
memtable_flush_period_in_ms
The memtable_flush_period_in_ms option sets the interval at which the memtable will be flushed to disk.
The default value of 0 means that periodic flushing is disabled, and flushes will only occur based on the commit log threshold or memtable threshold being reached.
When memtable_flush_period_in_ms=0, the memtable will flush when:
- the flush threshold is met
- on shutdown
- on nodetool flush
- when commitlogs get full
ALTER TABLE cyclist_name WITH memtable_flush_period_in_ms=1000;
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.
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.
ALTER TABLE cyclist_name WITH speculative_retry = '10ms';
ALTER TABLE cyclist_name WITH speculative_retry = '99percentile';