smiklosovic commented on code in PR #4399:
URL: https://github.com/apache/cassandra/pull/4399#discussion_r2410329650


##########
conf/cassandra.yaml:
##########
@@ -617,6 +617,54 @@ counter_cache_save_period: 7200s
 # Disabled by default, meaning all keys are going to be saved
 # counter_cache_keys_to_save: 100
 
+# Dictionary compression settings for ZSTD dictionary-based compression
+# These settings control the automatic training and caching of compression 
dictionaries
+# for tables that use ZSTD dictionary compression.
+
+# How often to refresh compression dictionaries across the cluster.
+# During refresh, nodes will check for newer dictionary versions and update 
their caches.
+# Min unit: s
+compression_dictionary_refresh_interval: 3600s
+
+# Initial delay before starting the first dictionary refresh cycle after node 
startup.
+# This prevents all nodes from refreshing simultaneously when the cluster 
starts.
+# Min unit: s
+compression_dictionary_refresh_initial_delay: 10s
+
+# Maximum number of compression dictionaries to cache per table.
+# Each table using dictionary compression can have multiple dictionaries cached
+# (current version plus recently used versions for reading older SSTables).
+compression_dictionary_cache_size: 10
+
+# How long to keep compression dictionaries in the cache before they expire.
+# Expired dictionaries will be removed from memory but can be reloaded if 
needed.
+# Min unit: s
+compression_dictionary_cache_expire: 3600s
+
+# Dictionary training configuration (advanced settings)
+# These settings control how compression dictionaries are trained from sample 
data.
+
+# Maximum size of a trained compression dictionary in bytes.
+# Larger dictionaries may provide better compression but use more memory.
+# Min unit: B
+compression_dictionary_training_max_dictionary_size: 65536

Review Comment:
   could this be converted to `64KiB`? I think that is more expressive then 
just a number.  That would lead to `DataStorageSpec.IntKibibytesBound` type of 
compression_dictionary_training_max_dictionary_size. By doing it like this, 
people are really forced to put there just integers and nothing else. This is 
in general not a style which is used through this yaml where data / capacity 
related configuration parameters are descendants of `DataStorageSpec`. 
   
   Same way you used `DurationSpec.IntSecondsBound` for properties above, we 
should use `DataStorageSpec` for these. 



##########
src/java/org/apache/cassandra/db/compression/ZstdDictionaryTrainer.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.compression;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.luben.zstd.Zstd;
+import com.github.luben.zstd.ZstdDictTrainer;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compression.CompressionDictionary.DictId;
+import org.apache.cassandra.db.compression.CompressionDictionary.Kind;
+import org.apache.cassandra.io.compress.IDictionaryCompressor;
+import org.apache.cassandra.io.compress.ZstdDictionaryCompressor;
+import org.apache.cassandra.schema.CompressionParams;
+
+/**
+ * Zstd implementation of dictionary trainer with lifecycle management.
+ */
+public class ZstdDictionaryTrainer implements ICompressionDictionaryTrainer
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(ZstdDictionaryTrainer.class);
+
+    private final String keyspaceName;
+    private final String tableName;
+    private final CompressionDictionaryTrainingConfig config;
+    private final AtomicLong totalSampleSize;
+    private final AtomicLong sampleCount;
+    private final int compressionLevel; // optimal if using the same level for 
training as when compressing.
+
+    // Sampling rate can be updated during training
+    private volatile int samplingRate;
+
+    // Minimum number of samples required by ZSTD library
+    private static final int MIN_SAMPLES_REQUIRED = 10;
+
+    private volatile Consumer<CompressionDictionary> dictionaryTrainedListener;
+    // TODO: manage the samples in this class for auto-train (follow-up). The 
ZstdDictTrainer cannot be re-used for multiple training runs.
+    private ZstdDictTrainer zstdTrainer;
+    private volatile boolean closed = false;
+    private volatile TrainingStatus currentTrainingStatus;
+
+    public ZstdDictionaryTrainer(String keyspaceName, String tableName,
+                                 CompressionDictionaryTrainingConfig config,
+                                 int compressionLevel)
+    {
+        this.keyspaceName = keyspaceName;
+        this.tableName = tableName;
+        this.config = config;
+        this.totalSampleSize = new AtomicLong(0);
+        this.sampleCount = new AtomicLong(0);
+        this.compressionLevel = compressionLevel;
+        this.samplingRate = config.samplingRate;
+        this.currentTrainingStatus = TrainingStatus.NOT_STARTED;
+    }
+
+    @Override
+    public boolean shouldSample()
+    {
+        return zstdTrainer != null && 
ThreadLocalRandom.current().nextInt(samplingRate) == 0;
+    }
+
+    @Override
+    public void addSample(ByteBuffer sample)
+    {
+        if (closed || sample == null || !sample.hasRemaining() || zstdTrainer 
== null)
+            return;
+
+        byte[] sampleBytes = new byte[sample.remaining()];
+        sample.duplicate().get(sampleBytes);
+
+        if (zstdTrainer.addSample(sampleBytes))
+        {
+            // Update the totalSampleSize and sampleCount if the sample is 
added
+            totalSampleSize.addAndGet(sampleBytes.length);
+            sampleCount.incrementAndGet();
+        }
+    }
+
+    @Override
+    public CompressionDictionary trainDictionary(boolean force)
+    {
+        boolean isReady = isReady();
+        if (!force && !isReady)
+        {
+            currentTrainingStatus = TrainingStatus.FAILED;
+            throw new IllegalStateException("Trainer is not ready");
+        }
+
+        long currentSampleCount = sampleCount.get();
+        if (currentSampleCount < MIN_SAMPLES_REQUIRED) // minimum samples 
should be required even if force training
+        {
+            currentTrainingStatus = TrainingStatus.FAILED;
+            String errorMsg = String.format("Insufficient samples for 
training: %d (minimum required: %d)",
+                                            currentSampleCount, 
MIN_SAMPLES_REQUIRED);
+            throw new IllegalStateException(errorMsg);
+        }
+
+        currentTrainingStatus = TrainingStatus.TRAINING;
+        try
+        {
+            logger.debug("Training with sample count: {}, sample size: {}, 
isReady: {}",
+                        currentSampleCount, totalSampleSize.get(), isReady);
+            byte[] dictBytes = zstdTrainer.trainSamples();
+            long zstdDictId = Zstd.getDictIdFromDict(dictBytes);
+            DictId dictId = new DictId(Kind.ZSTD, 
makeDictionaryId(System.currentTimeMillis(), zstdDictId));
+            currentTrainingStatus = TrainingStatus.COMPLETED;
+            logger.debug("New dictionary is trained with {}", dictId);
+            CompressionDictionary dictionary = new 
ZstdCompressionDictionary(dictId, dictBytes);
+            notifyDictionaryTrainedListener(dictionary);
+            return dictionary;
+        }
+        catch (Exception e)
+        {
+            currentTrainingStatus = TrainingStatus.FAILED;
+            throw new RuntimeException("Failed to train Zstd dictionary", e);
+        }
+    }
+
+    @Override
+    public boolean isReady()
+    {
+        return currentTrainingStatus != TrainingStatus.TRAINING
+               && !closed
+               && zstdTrainer != null
+               && totalSampleSize.get() >= config.acceptableTotalSampleSize
+               && sampleCount.get() > MIN_SAMPLES_REQUIRED;
+    }
+
+    @Override
+    public TrainingStatus getTrainingStatus()
+    {
+        return currentTrainingStatus;
+    }
+
+    @Override
+    public boolean start(boolean manualTraining)
+    {
+        if (closed || !(manualTraining || shouldAutoStartTraining()))
+            return false;
+
+        try
+        {
+            // reset on starting; a new zstdTrainer instance is created during 
reset
+            reset();
+            logger.info("Started dictionary training for {}.{}", keyspaceName, 
tableName);
+            currentTrainingStatus = TrainingStatus.SAMPLING;
+            return true;
+        }
+        catch (Exception e)
+        {
+            logger.warn("Failed to create ZstdDictTrainer for {}.{}", 
keyspaceName, tableName, e);
+            currentTrainingStatus = TrainingStatus.FAILED;
+        }
+        return false;
+    }
+
+    /**
+     * Determines if training should auto-start based on configuration.
+     */
+    private boolean shouldAutoStartTraining()
+    {
+        return 
DatabaseDescriptor.getCompressionDictionaryTrainingAutoTrainEnabled();
+    }
+
+    @Override
+    public void reset()
+    {
+        if (closed)
+        {
+            return;
+        }
+
+        currentTrainingStatus = TrainingStatus.NOT_STARTED;
+        synchronized (this)
+        {
+            totalSampleSize.set(0);
+            sampleCount.set(0);
+            zstdTrainer = new ZstdDictTrainer(config.maxTotalSampleSize, 
config.maxDictionarySize, compressionLevel);
+        }
+    }
+
+    @Override
+    public Kind kind()
+    {
+        return Kind.ZSTD;
+    }
+
+    @Override
+    public void setDictionaryTrainedListener(Consumer<CompressionDictionary> 
listener)
+    {
+        this.dictionaryTrainedListener = listener;
+    }
+
+    @Override
+    public void updateSamplingRate(int newSamplingRate)
+    {
+        if (newSamplingRate <= 0)
+        {
+            throw new IllegalArgumentException("Sampling rate must be 
positive, got: " + newSamplingRate);
+        }
+        this.samplingRate = newSamplingRate;
+        logger.debug("Updated sampling rate to {} for {}.{}", newSamplingRate, 
keyspaceName, tableName);
+    }
+
+    /**
+     * Notifies the registered listener that a dictionary has been trained.
+     *
+     * @param dictionary the newly trained dictionary
+     */
+    private void notifyDictionaryTrainedListener(CompressionDictionary 
dictionary)
+    {
+        Consumer<CompressionDictionary> listener = 
this.dictionaryTrainedListener;
+        if (listener != null)
+        {
+            try
+            {
+                listener.accept(dictionary);
+            }
+            catch (Exception e)
+            {
+                logger.warn("Error notifying dictionary trained listener for 
{}.{}", keyspaceName, tableName, e);
+            }
+        }
+    }
+
+    @Override
+    public boolean isCompatibleWith(CompressionParams newParams)
+    {
+        if (!newParams.isDictionaryCompressionEnabled())
+        {
+            return false;
+        }
+
+        IDictionaryCompressor newCompressor = (IDictionaryCompressor) 
newParams.getSstableCompressor();
+
+        // Check if the compressor type is compatible with this trainer
+        if (newCompressor.acceptableDictionaryKind() != Kind.ZSTD)
+        {
+            return false;
+        }
+
+        ZstdDictionaryCompressor zstdDictionaryCompressor = 
(ZstdDictionaryCompressor) newCompressor;
+        // For Zstd compressors, check if compression level matches
+        return this.compressionLevel == 
zstdDictionaryCompressor.compressionLevel();
+    }
+
+    @Override
+    public void close()
+    {
+        if (closed)
+            return;
+
+        closed = true;
+        currentTrainingStatus = TrainingStatus.NOT_STARTED;
+
+        synchronized (this)
+        {
+            // Permanent shutdown: clear all state and prevent restart
+            totalSampleSize.set(0);
+            sampleCount.set(0);
+            zstdTrainer = null;
+        }
+
+        logger.info("Permanently closed dictionary trainer for {}.{}", 
keyspaceName, tableName);
+    }
+
+    /**
+     * Creates a monotonically increasing dictionary ID by combining timestamp 
and Zstd dictionary ID.
+     *
+     * The resulting dictionary ID has the following structure:
+     * - Upper 32 bits: timestamp in minutes (signed int)
+     * - Lower 32 bits: Zstd dictionary ID (unsigned int, passed as long due 
to Java limitations)
+     *
+     * This ensures dictionary IDs are monotonically increasing over time, 
which helps to identify
+     * the latest dictionary.
+     *
+     * The implementation assumes that dictionary training frequency is 
significantly larger than
+     * every minute, which a healthy system should do. In the scenario when 
multiple dictionaries
+     * are trained in the same minute (only possible using manual training), 
there should not be
+     * correctness concerns since the dictionary is attached to the SSTables, 
but leads to performance
+     * hit from having too many dictionary. Therefore, such scenario should be 
avoided at the best.
+     *
+     * @param currentTimeMillis the current time in milliseconds
+     * @param zstdDictId Zstd dictionary ID (unsigned 32-bit value represented 
as long)
+     * @return combined dictionary ID that is monotonically increasing over 
time
+     */
+    static long makeDictionaryId(long currentTimeMillis, long zstdDictId)

Review Comment:
   is this really ZSTD specific? seems like moving this to DictId or similar 
would be better so it can be used for other compressors too. I just dont see 
why is this ZSTD specific only. 



##########
src/java/org/apache/cassandra/db/compression/CompressionDictionaryEventHandler.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.compression;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.schema.SystemDistributedKeyspace;
+import org.apache.cassandra.tcm.ClusterMetadata;
+import org.apache.cassandra.utils.FBUtilities;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Handles compression dictionary events including training completion and 
cluster notifications.
+ * <p>
+ * This class handles:
+ * - Broadcasting dictionary updates to cluster nodes
+ * - Retrieving new dictionaries when notified by other nodes
+ * - Managing dictionary cache updates
+ */
+public class CompressionDictionaryEventHandler implements 
ICompressionDictionaryEventHandler
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CompressionDictionaryEventHandler.class);
+
+    private final ColumnFamilyStore cfs;
+    private final String keyspaceName;
+    private final String tableName;
+    private final ICompressionDictionaryCache cache;
+
+    public CompressionDictionaryEventHandler(ColumnFamilyStore cfs, 
ICompressionDictionaryCache cache)
+    {
+        this.cfs = cfs;
+        this.keyspaceName = cfs.keyspace.getName();
+        this.tableName = cfs.getTableName();
+        this.cache = cache;
+    }
+
+    @Override
+    public void onNewDictionaryTrained(long dictionaryId)

Review Comment:
   dont we need to include `DictId`, actually? That also happens to contain 
`Kind`. I think that is way more descriptive and robust.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to