Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -302,6 +302,7 @@ These parameters are meant to be configured via CQL for each respective table if
* `training_max_total_sample_size` (default: `10MiB`): Maximum total size of sample data to collect for training, approximately 10MB. This parameter is configured in the
table's compression options for `ZstdDictionaryCompressor`.
* `training_max_dictionary_size` (default: `64KiB`): Maximum size of trained dictionaries in bytes. Larger dictionaries can capture more patterns but increase memory overhead. This is a parameter of `ZstdDictionaryCompressor` of a table, in `compression` section.
* `min_training_frequency` (default: `0m`): Minimum time which needs to pass until we can train another compression dictionary. For example, if this property is set to `1h`, then we can train another dictionary no earlier than 1 hour after the last training was conducted. `0m`, default, means we can train as frequently as we want. The purpose of this parameter is to prevent excessive training which might not make sense from operational and performance perspective. If an operator wants to prevent training altogether, they can set this property to overly big value, like 36500d which means that next training can occur at least after 100 years from the last one, effectively disabling any training. The minimum resolution of `min_training_frequency` is in minutes. Accepts minutes (`m`), hours (`h`) and days (`d`).

Example:

Expand All @@ -312,13 +313,13 @@ ALTER TABLE keyspace.table
'class': 'ZstdDictionaryCompressor',
'compression_level': '3',
'training_max_total_sample_size': '20MiB',
'training_max_dictionary_size': '128KiB'
'training_max_dictionary_size': '128KiB',
'min_training_frequency': '1d'
};
----

It is possible to override these training parameters by `nodetool compressiondictionary train` command as
explained in the section futher down below. If `train` subcommand do not override them, CQL parameters are
taken into account.
It is possible to override these training parameters by `nodetool compressiondictionary train` command (except `min_training_frequency` which is configurable via CQL only) as
explained in the section further down below. If `train` subcommand do not override them, CQL parameters are taken into account.

== Other options

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.cassandra.db.compression;

import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand All @@ -33,20 +35,24 @@
import org.slf4j.LoggerFactory;

import org.apache.cassandra.config.DataStorageSpec;
import org.apache.cassandra.config.DurationSpec;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.compression.CompressionDictionary.LightweightCompressionDictionary;
import org.apache.cassandra.db.compression.CompressionDictionaryDetailsTabularData.CompressionDictionaryDataObject;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.schema.CompressionParams;
import org.apache.cassandra.schema.SystemDistributedKeyspace;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.MBeanWrapper;
import org.apache.cassandra.utils.MBeanWrapper.OnException;

import static java.lang.String.format;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MIN_FREQUENCY;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MIN_FREQUENCY_PARAMETER_NAME;

public class CompressionDictionaryManager implements CompressionDictionaryManagerMBean,
ICompressionDictionaryCache,
Expand Down Expand Up @@ -219,6 +225,12 @@ public synchronized void train(boolean force, Map<String, String> parameters)
// resolve training config and fail fast when invalid, so we do not reach logic which would e.g. flush unnecessarily.
CompressionDictionaryTrainingConfig trainingConfig = createTrainingConfig(parameters);

LightweightCompressionDictionary dictionary = SystemDistributedKeyspace.retrieveLightweightLatestCompressionDictionary(columnFamilyStore.getKeyspaceName(),
columnFamilyStore.getTableName(),
columnFamilyStore.metadata.id.toLongString());

checkTrainingFrequency(dictionary);

// SSTable-based training: sample from existing SSTables
Set<SSTableReader> sstables = columnFamilyStore.getLiveSSTables();
if (sstables.isEmpty())
Expand Down Expand Up @@ -319,10 +331,15 @@ public synchronized void importCompressionDictionary(CompositeData compositeData
CompressionDictionary.DictId dictId = new CompressionDictionary.DictId(kind, dataObject.dictId);

LightweightCompressionDictionary latestCompressionDictionary = SystemDistributedKeyspace.retrieveLightweightLatestCompressionDictionary(keyspaceName, tableName, tableId);
if (latestCompressionDictionary != null && latestCompressionDictionary.dictId.id > dictId.id)
if (latestCompressionDictionary != null)
{
throw new IllegalArgumentException(format("Dictionary to import has older dictionary id (%s) than the latest compression dictionary (%s) for table %s.%s",
dictId.id, latestCompressionDictionary.dictId.id, keyspaceName, tableName));
if (latestCompressionDictionary.dictId.id > dictId.id)
{
throw new IllegalArgumentException(format("Dictionary to import has older dictionary id (%s) than the latest compression dictionary (%s) for table %s.%s",
dictId.id, latestCompressionDictionary.dictId.id, keyspaceName, tableName));
}

checkTrainingFrequency(latestCompressionDictionary);
}

handleNewDictionary(kind.createDictionary(dictId, dataObject.dict, dataObject.dictChecksum));
Expand Down Expand Up @@ -394,6 +411,46 @@ private int getCompressionDictionaryTrainingMaxTotalSampleSize(CompressionParams
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE);
}

private DurationSpec.IntMinutesBound getCompressionDictionaryMinTrainingFrequency(CompressionParams compressionParams)
{
String resolvedValue = compressionParams.getOtherOptions().getOrDefault(TRAINING_MIN_FREQUENCY_PARAMETER_NAME, DEFAULT_TRAINING_MIN_FREQUENCY);

try
{
return new DurationSpec.IntMinutesBound(resolvedValue);
}
catch (Throwable t)
{
throw new IllegalArgumentException(String.format("Invalid value for %s: %s. Reason: %s",
TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
resolvedValue,
t.getMessage()));
}
}

private void checkTrainingFrequency(LightweightCompressionDictionary lastDictionary)
{
Instant lastTraining = lastDictionary == null ? null : lastDictionary.createdAt;
DurationSpec.IntMinutesBound minTrainingFrequency = getCompressionDictionaryMinTrainingFrequency(columnFamilyStore.metadata().params.compression);

// if there is no dictionary trained so far or min frequency is 0 - that is we can train as often as we want -
// then do not check if we can
if (lastTraining != null && minTrainingFrequency.toMinutes() != 0)
{
Instant now = FBUtilities.now();
int minTrainingFrequencyMinutes = minTrainingFrequency.toMinutes();
if (lastTraining.isAfter(now.minus(minTrainingFrequencyMinutes, ChronoUnit.MINUTES)))
{
Instant nextEarliestTraining = lastTraining.plus(minTrainingFrequencyMinutes, ChronoUnit.MINUTES);
throw new IllegalArgumentException(format("The next training or importing can occur only at least after %s from the last training which happened at %s. " +
"You can train again no earlier than at %s.",
minTrainingFrequency,
lastTraining,
nextEarliestTraining));
}
}
}

private int internalTrainingParameterResolution(CompressionParams compressionParams,
String userSuppliedValue,
String parameterName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.cassandra.io.compress;

import org.apache.cassandra.config.DataStorageSpec;
import org.apache.cassandra.config.DurationSpec;
import org.apache.cassandra.db.compression.CompressionDictionary;
import org.apache.cassandra.exceptions.ConfigurationException;

Expand All @@ -41,6 +42,11 @@ public interface IDictionaryCompressor<T extends CompressionDictionary>
String TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME = "training_max_total_sample_size";
String DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE = "10MiB";

String TRAINING_MIN_FREQUENCY_PARAMETER_NAME = "min_training_frequency";
// 0m means there is no limit how often we can train, if this is set to e.g. 1h, that means
// that once we train a dictionary for given table, then we can train again after at least 1 hour.
String DEFAULT_TRAINING_MIN_FREQUENCY = "0m";

/**
* Validates value of a parameter for training purposes. The value to validate should
* be accepted by {@link DataStorageSpec.IntKibibytesBound}. This method is used upon validation
Expand All @@ -49,7 +55,7 @@ public interface IDictionaryCompressor<T extends CompressionDictionary>
* @param parameterName name of a parameter to validate
* @param resolvedValue value to validate
*/
static void validateTrainingParameter(String parameterName, String resolvedValue)
static void validateSizeBasedTrainingParameter(String parameterName, String resolvedValue)
{
try
{
Expand All @@ -62,6 +68,27 @@ static void validateTrainingParameter(String parameterName, String resolvedValue
}
}

/**
* Validates value of a parameter for training purposes. The value to validate should
* be accepted by {@link DurationSpec.IntMinutesBound}. This method is used upon validation of input parameters
* in the implementation of dictionary compressor.
*
* @param parameterName name of a parameter to validate
* @param resolvedValue value to validate
*/
static void validateDurationBasedTrainingParameter(String parameterName, String resolvedValue)
{
try
{
new DurationSpec.IntMinutesBound(resolvedValue).toMinutes();
}
catch (Throwable t)
{
throw new ConfigurationException(format("Unable to set value to parameter %s: %s. Reason: %s",
parameterName, resolvedValue, t.getMessage()));
}
}

/**
* Returns a compressor instance configured with the specified compression dictionary.
* <br>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,8 @@
import org.apache.cassandra.db.compression.ZstdCompressionDictionary;
import org.apache.cassandra.utils.concurrent.Ref;

import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateTrainingParameter;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateDurationBasedTrainingParameter;
import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateSizeBasedTrainingParameter;

public class ZstdDictionaryCompressor extends ZstdCompressorBase implements ICompressor, IDictionaryCompressor<ZstdCompressionDictionary>
{
Expand Down Expand Up @@ -77,12 +78,15 @@ public static ZstdDictionaryCompressor create(Map<String, String> options)
{
int level = getOrDefaultCompressionLevel(options);
validateCompressionLevel(level);
validateTrainingParameter(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
options.getOrDefault(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE));
validateTrainingParameter(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
options.getOrDefault(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE));
validateSizeBasedTrainingParameter(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
options.getOrDefault(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE));
validateSizeBasedTrainingParameter(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
options.getOrDefault(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE));
validateDurationBasedTrainingParameter(TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
options.getOrDefault(TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
DEFAULT_TRAINING_MIN_FREQUENCY));
return getOrCreate(level, null);
}

Expand Down Expand Up @@ -119,7 +123,8 @@ private ZstdDictionaryCompressor(int level, ZstdCompressionDictionary dictionary
{
super(level, Set.of(COMPRESSION_LEVEL_OPTION_NAME,
TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME));
TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
TRAINING_MIN_FREQUENCY_PARAMETER_NAME));
this.dictionary = dictionary;
this.dictionaryRef = dictionaryRef;
}
Expand Down
Loading