[ 
https://issues.apache.org/jira/browse/CASSANDRA-17021?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18024149#comment-18024149
 ] 

Jon Haddad edited comment on CASSANDRA-17021 at 10/1/25 9:16 PM:
-----------------------------------------------------------------

h3. New nodeool command: {{traincompressiondictionary}}

*Implementation:* 
{{src/java/org/apache/cassandra/tools/nodetool/TrainCompressionDictionary.java}}
{code:bash}
  # Basic usage
  nodetool traincompressiondictionary <keyspace> <table>

  # With options
  nodetool traincompressiondictionary <keyspace> <table> \
    --max-sampling-duration 600 \
    --sampling-rate 0.5 \
    --async

  # Check status
  nodetool traincompressiondictionary --status <keyspace> <table>
  {code}
*Options:*
 * {{{}-d, --max-sampling-duration{}}}: Max time to collect samples before 
training (default: 600 seconds)
 * {{{}-r, --sampling-rate{}}}: Sampling rate as double (0, 1]. 1.0 = sample 
all, 0.5 = 50% (optional)
 * {{{}-a, --async{}}}: Run training asynchronously without waiting
 * {{{}-s, --status{}}}: Show current training status (NOT_STARTED, SAMPLING, 
TRAINING, COMPLETED, FAILED)

h2. New Configs in cassandra.yaml

*Configuration files:*
 * User config: {{conf/cassandra.yaml}} (lines with 
{{{}compression_dictionary_*{}}})
 * Java config: {{src/java/org/apache/cassandra/config/Config.java}}
 * Descriptor: {{src/java/org/apache/cassandra/config/DatabaseDescriptor.java}}

h3. Dictionary Refresh & Caching
 * {{{}compression_dictionary_refresh_interval{}}}: How often to refresh 
dictionaries cluster-wide (default: 3600s)
 * {{{}compression_dictionary_refresh_initial_delay{}}}: Initial delay before 
first refresh (default: 10s)
 * {{{}compression_dictionary_cache_size{}}}: Max dictionaries to cache per 
table (default: 10)
 * {{{}compression_dictionary_cache_expire{}}}: Dictionary cache TTL (default: 
3600s)

h3. Training Configuration
 * {{{}compression_dictionary_training_max_dictionary_size{}}}: Max dictionary 
size in bytes (default: 65536)
 * {{{}compression_dictionary_training_max_total_sample_size{}}}: Max sample 
data size (default: 10485760, ~10MB)
 * {{{}compression_dictionary_training_auto_train_enabled{}}}: Enable automatic 
training (default: false)
 * {{{}compression_dictionary_training_sampling_rate{}}}: Sampling rate 1-10000 
(100 = 1% of writes, default: 100)

h2. Parameters to Use ZSTD Dictionaries

*Implementation:*
 * Compressor: 
{{src/java/org/apache/cassandra/io/compress/ZstdDictionaryCompressor.java}}
 * Base class: 
{{src/java/org/apache/cassandra/io/compress/ZstdCompressorBase.java}}
 * Interface: 
{{src/java/org/apache/cassandra/io/compress/IDictionaryCompressor.java}}
 * Compression params: 
{{src/java/org/apache/cassandra/schema/CompressionParams.java:239}} 
(isDictionaryCompressionEnabled)

h3. Enable dictionary compression on a table
{code:sql}
  -- Create table with dictionary compression
  CREATE TABLE mytable (
      id int PRIMARY KEY,
      data text
  ) WITH compression = {
      'class': 'ZstdDictionaryCompressor'
  };

  -- With compression level
  CREATE TABLE mytable (
      id int PRIMARY KEY,
      data text
  ) WITH compression = {
      'class': 'ZstdDictionaryCompressor',
      'compression_level': '3'
  };

  -- Alter existing table
  ALTER TABLE mytable
  WITH compression = {
      'class': 'ZstdDictionaryCompressor'
  };
  {code}
*Key difference:* Use {{ZstdDictionaryCompressor}} class instead of 
{{ZstdCompressor}}
h2. How to Train a Dictionary

*Core components:*
 * Manager: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryManager.java}}
 * Trainer: 
{{src/java/org/apache/cassandra/db/compression/ZstdDictionaryTrainer.java}}
 * Scheduler: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryScheduler.java}}
 * Training options: 
{{src/java/org/apache/cassandra/db/compression/ManualTrainingOptions.java}}

h3. Manual training via nodetool
{code:bash}
  # Start training
  nodetool traincompressiondictionary mykeyspace mytable \
    --max-sampling-duration 600 \
    --sampling-rate 1.0

  # The process:
  # 1. Collects samples from chunk data during SSTable writes
  # 2. After sampling period, trains dictionary using collected samples
  # 3. Persists dictionary to system_distributed.compression_dictionaries
  # 4. Dictionary becomes available for compression of new SSTables
  {code}
*Note:* Consider running {{nodetool flush}} alongside training command to make 
chunk data available for sampling.
h3. Automatic training
 * Set {{compression_dictionary_training_auto_train_enabled: true}} in 
cassandra.yaml
 * System automatically samples writes based on 
{{compression_dictionary_training_sampling_rate}}
 * Trains dictionaries in background

h2. Dictionary Storage

*Storage location:* {{system_distributed.compression_dictionaries}} table

*Implementation:*
 * System table: 
{{src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java}} 
(COMPRESSION_DICTIONARIES_CQL)
 * Dictionary class: 
{{src/java/org/apache/cassandra/db/compression/ZstdCompressionDictionary.java}}
 * Cache: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryCache.java}}
 * Event handler: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryEventHandler.java}}

*Schema:*
{code:sql}
  CREATE TABLE system_distributed.compression_dictionaries (
      keyspace_name text,
      table_name text,
      kind text,           -- e.g., 'ZSTD'
      dict_id bigint,      -- Higher ID = newer dictionary
      dict blob,           -- Raw dictionary bytes
      PRIMARY KEY ((keyspace_name, table_name), dict_id)
  ) WITH CLUSTERING ORDER BY (dict_id DESC);
  {code}
*Key points:*
 * Dictionaries are cluster-wide (stored in distributed system keyspace)
 * Each table can have multiple dictionary versions (current + historical for 
reading old SSTables)
 * Newest dictionary has highest {{dict_id}}
 * Dictionaries are cached locally per node according to 
{{compression_dictionary_cache_size}} and 
{{compression_dictionary_cache_expire}} settings


was (Author: rustyrazorblade):
Summary of changes to ZSTD dictionary compression support in Cassandra.
h2. Nodetool Commands
h3. New command: {{traincompressiondictionary}}

*Implementation:* 
{{src/java/org/apache/cassandra/tools/nodetool/TrainCompressionDictionary.java}}
{code:bash}
  # Basic usage
  nodetool traincompressiondictionary <keyspace> <table>

  # With options
  nodetool traincompressiondictionary <keyspace> <table> \
    --max-sampling-duration 600 \
    --sampling-rate 0.5 \
    --async

  # Check status
  nodetool traincompressiondictionary --status <keyspace> <table>
  {code}
*Options:*
 * {{{}-d, --max-sampling-duration{}}}: Max time to collect samples before 
training (default: 600 seconds)
 * {{{}-r, --sampling-rate{}}}: Sampling rate as double (0, 1]. 1.0 = sample 
all, 0.5 = 50% (optional)
 * {{{}-a, --async{}}}: Run training asynchronously without waiting
 * {{{}-s, --status{}}}: Show current training status (NOT_STARTED, SAMPLING, 
TRAINING, COMPLETED, FAILED)

h2. New Configs in cassandra.yaml

*Configuration files:*
 * User config: {{conf/cassandra.yaml}} (lines with 
{{{}compression_dictionary_*{}}})
 * Java config: {{src/java/org/apache/cassandra/config/Config.java}}
 * Descriptor: {{src/java/org/apache/cassandra/config/DatabaseDescriptor.java}}

h3. Dictionary Refresh & Caching
 * {{{}compression_dictionary_refresh_interval{}}}: How often to refresh 
dictionaries cluster-wide (default: 3600s)
 * {{{}compression_dictionary_refresh_initial_delay{}}}: Initial delay before 
first refresh (default: 10s)
 * {{{}compression_dictionary_cache_size{}}}: Max dictionaries to cache per 
table (default: 10)
 * {{{}compression_dictionary_cache_expire{}}}: Dictionary cache TTL (default: 
3600s)

h3. Training Configuration
 * {{{}compression_dictionary_training_max_dictionary_size{}}}: Max dictionary 
size in bytes (default: 65536)
 * {{{}compression_dictionary_training_max_total_sample_size{}}}: Max sample 
data size (default: 10485760, ~10MB)
 * {{{}compression_dictionary_training_auto_train_enabled{}}}: Enable automatic 
training (default: false)
 * {{{}compression_dictionary_training_sampling_rate{}}}: Sampling rate 1-10000 
(100 = 1% of writes, default: 100)

h2. Parameters to Use ZSTD Dictionaries

*Implementation:*
 * Compressor: 
{{src/java/org/apache/cassandra/io/compress/ZstdDictionaryCompressor.java}}
 * Base class: 
{{src/java/org/apache/cassandra/io/compress/ZstdCompressorBase.java}}
 * Interface: 
{{src/java/org/apache/cassandra/io/compress/IDictionaryCompressor.java}}
 * Compression params: 
{{src/java/org/apache/cassandra/schema/CompressionParams.java:239}} 
(isDictionaryCompressionEnabled)

h3. Enable dictionary compression on a table
{code:sql}
  -- Create table with dictionary compression
  CREATE TABLE mytable (
      id int PRIMARY KEY,
      data text
  ) WITH compression = {
      'class': 'ZstdDictionaryCompressor'
  };

  -- With compression level
  CREATE TABLE mytable (
      id int PRIMARY KEY,
      data text
  ) WITH compression = {
      'class': 'ZstdDictionaryCompressor',
      'compression_level': '3'
  };

  -- Alter existing table
  ALTER TABLE mytable
  WITH compression = {
      'class': 'ZstdDictionaryCompressor'
  };
  {code}
*Key difference:* Use {{ZstdDictionaryCompressor}} class instead of 
{{ZstdCompressor}}
h2. How to Train a Dictionary

*Core components:*
 * Manager: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryManager.java}}
 * Trainer: 
{{src/java/org/apache/cassandra/db/compression/ZstdDictionaryTrainer.java}}
 * Scheduler: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryScheduler.java}}
 * Training options: 
{{src/java/org/apache/cassandra/db/compression/ManualTrainingOptions.java}}

h3. Manual training via nodetool
{code:bash}
  # Start training
  nodetool traincompressiondictionary mykeyspace mytable \
    --max-sampling-duration 600 \
    --sampling-rate 1.0

  # The process:
  # 1. Collects samples from chunk data during SSTable writes
  # 2. After sampling period, trains dictionary using collected samples
  # 3. Persists dictionary to system_distributed.compression_dictionaries
  # 4. Dictionary becomes available for compression of new SSTables
  {code}
*Note:* Consider running {{nodetool flush}} alongside training command to make 
chunk data available for sampling.
h3. Automatic training
 * Set {{compression_dictionary_training_auto_train_enabled: true}} in 
cassandra.yaml
 * System automatically samples writes based on 
{{compression_dictionary_training_sampling_rate}}
 * Trains dictionaries in background

h2. Dictionary Storage

*Storage location:* {{system_distributed.compression_dictionaries}} table

*Implementation:*
 * System table: 
{{src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java}} 
(COMPRESSION_DICTIONARIES_CQL)
 * Dictionary class: 
{{src/java/org/apache/cassandra/db/compression/ZstdCompressionDictionary.java}}
 * Cache: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryCache.java}}
 * Event handler: 
{{src/java/org/apache/cassandra/db/compression/CompressionDictionaryEventHandler.java}}

*Schema:*
{code:sql}
  CREATE TABLE system_distributed.compression_dictionaries (
      keyspace_name text,
      table_name text,
      kind text,           -- e.g., 'ZSTD'
      dict_id bigint,      -- Higher ID = newer dictionary
      dict blob,           -- Raw dictionary bytes
      PRIMARY KEY ((keyspace_name, table_name), dict_id)
  ) WITH CLUSTERING ORDER BY (dict_id DESC);
  {code}
*Key points:*
 * Dictionaries are cluster-wide (stored in distributed system keyspace)
 * Each table can have multiple dictionary versions (current + historical for 
reading old SSTables)
 * Newest dictionary has highest {{dict_id}}
 * Dictionaries are cached locally per node according to 
{{compression_dictionary_cache_size}} and 
{{compression_dictionary_cache_expire}} settings

> Enhance Zstd support in Cassandra with dictionaries
> ---------------------------------------------------
>
>                 Key: CASSANDRA-17021
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17021
>             Project: Apache Cassandra
>          Issue Type: Improvement
>          Components: Feature/Compression
>            Reporter: Dinesh Joshi
>            Assignee: Yifan Cai
>            Priority: Normal
>
> Currently Cassandra supports zstd compression. However, Zstd also supports 
> dictionaries to enhance not only the compression ratio but also the speed. 
> Dictionaries can show 3-4x savings. We should add support to train 
> dictionaries, ideally per SSTable this will yield the maximum gains.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to