Skip to content

Commit 9303ca0

Browse files
committed
Introduce a check for minimum time to pass to train or import a compression dictionary from the last one
patch by Stefan Miklosovic; reviewed by Yifan Cai for CASSANDRA-21179
1 parent d41a4f7 commit 9303ca0

6 files changed

Lines changed: 349 additions & 16 deletions

File tree

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
5.1
2+
* Introduce a check for minimum time to pass to train or import a compression dictionary from the last one (CASSANDRA-21179)
23
* Allow overriding compaction strategy parameters during startup (CASSANDRA-21169)
34
* Introduce created_at column to system_distributed.compression_dictionaries (CASSANDRA-21178)
45
* Be able to detect and remove orphaned compression dictionaries (CASSANDRA-21157)

doc/modules/cassandra/pages/managing/operating/compression.adoc

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -302,6 +302,7 @@ These parameters are meant to be configured via CQL for each respective table if
302302
* `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
303303
table's compression options for `ZstdDictionaryCompressor`.
304304
* `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.
305+
* `training_min_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 `training_min_frequency` is in minutes. Accepts minutes (`m`), hours (`h`) and days (`d`).
305306

306307
Example:
307308

@@ -312,13 +313,13 @@ ALTER TABLE keyspace.table
312313
'class': 'ZstdDictionaryCompressor',
313314
'compression_level': '3',
314315
'training_max_total_sample_size': '20MiB',
315-
'training_max_dictionary_size': '128KiB'
316+
'training_max_dictionary_size': '128KiB',
317+
'training_min_frequency': '1d'
316318
};
317319
----
318320

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

323324
== Other options
324325

src/java/org/apache/cassandra/db/compression/CompressionDictionaryManager.java

Lines changed: 60 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818

1919
package org.apache.cassandra.db.compression;
2020

21+
import java.time.Instant;
22+
import java.time.temporal.ChronoUnit;
2123
import java.util.List;
2224
import java.util.Map;
2325
import java.util.Set;
@@ -33,20 +35,24 @@
3335
import org.slf4j.LoggerFactory;
3436

3537
import org.apache.cassandra.config.DataStorageSpec;
38+
import org.apache.cassandra.config.DurationSpec;
3639
import org.apache.cassandra.db.ColumnFamilyStore;
3740
import org.apache.cassandra.db.compression.CompressionDictionary.LightweightCompressionDictionary;
3841
import org.apache.cassandra.db.compression.CompressionDictionaryDetailsTabularData.CompressionDictionaryDataObject;
3942
import org.apache.cassandra.io.sstable.format.SSTableReader;
4043
import org.apache.cassandra.schema.CompressionParams;
4144
import org.apache.cassandra.schema.SystemDistributedKeyspace;
45+
import org.apache.cassandra.utils.FBUtilities;
4246
import org.apache.cassandra.utils.MBeanWrapper;
4347
import org.apache.cassandra.utils.MBeanWrapper.OnException;
4448

4549
import static java.lang.String.format;
4650
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE;
4751
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE;
52+
import static org.apache.cassandra.io.compress.IDictionaryCompressor.DEFAULT_TRAINING_MIN_FREQUENCY;
4853
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME;
4954
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME;
55+
import static org.apache.cassandra.io.compress.IDictionaryCompressor.TRAINING_MIN_FREQUENCY_PARAMETER_NAME;
5056

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

228+
LightweightCompressionDictionary dictionary = SystemDistributedKeyspace.retrieveLightweightLatestCompressionDictionary(columnFamilyStore.getKeyspaceName(),
229+
columnFamilyStore.getTableName(),
230+
columnFamilyStore.metadata.id.toLongString());
231+
232+
checkTrainingFrequency(dictionary);
233+
222234
// SSTable-based training: sample from existing SSTables
223235
Set<SSTableReader> sstables = columnFamilyStore.getLiveSSTables();
224236
if (sstables.isEmpty())
@@ -319,10 +331,15 @@ public synchronized void importCompressionDictionary(CompositeData compositeData
319331
CompressionDictionary.DictId dictId = new CompressionDictionary.DictId(kind, dataObject.dictId);
320332

321333
LightweightCompressionDictionary latestCompressionDictionary = SystemDistributedKeyspace.retrieveLightweightLatestCompressionDictionary(keyspaceName, tableName, tableId);
322-
if (latestCompressionDictionary != null && latestCompressionDictionary.dictId.id > dictId.id)
334+
if (latestCompressionDictionary != null)
323335
{
324-
throw new IllegalArgumentException(format("Dictionary to import has older dictionary id (%s) than the latest compression dictionary (%s) for table %s.%s",
325-
dictId.id, latestCompressionDictionary.dictId.id, keyspaceName, tableName));
336+
if (latestCompressionDictionary.dictId.id > dictId.id)
337+
{
338+
throw new IllegalArgumentException(format("Dictionary to import has older dictionary id (%s) than the latest compression dictionary (%s) for table %s.%s",
339+
dictId.id, latestCompressionDictionary.dictId.id, keyspaceName, tableName));
340+
}
341+
342+
checkTrainingFrequency(latestCompressionDictionary);
326343
}
327344

328345
handleNewDictionary(kind.createDictionary(dictId, dataObject.dict, dataObject.dictChecksum));
@@ -394,6 +411,46 @@ private int getCompressionDictionaryTrainingMaxTotalSampleSize(CompressionParams
394411
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE);
395412
}
396413

414+
private DurationSpec.IntMinutesBound getCompressionDictionaryMinTrainingFrequency(CompressionParams compressionParams)
415+
{
416+
String resolvedValue = compressionParams.getOtherOptions().getOrDefault(TRAINING_MIN_FREQUENCY_PARAMETER_NAME, DEFAULT_TRAINING_MIN_FREQUENCY);
417+
418+
try
419+
{
420+
return new DurationSpec.IntMinutesBound(resolvedValue);
421+
}
422+
catch (Throwable t)
423+
{
424+
throw new IllegalArgumentException(String.format("Invalid value for %s: %s. Reason: %s",
425+
TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
426+
resolvedValue,
427+
t.getMessage()));
428+
}
429+
}
430+
431+
private void checkTrainingFrequency(LightweightCompressionDictionary lastDictionary)
432+
{
433+
Instant lastTraining = lastDictionary == null ? null : lastDictionary.createdAt;
434+
DurationSpec.IntMinutesBound minTrainingFrequency = getCompressionDictionaryMinTrainingFrequency(columnFamilyStore.metadata().params.compression);
435+
436+
// if there is no dictionary trained so far or min frequency is 0 - that is we can train as often as we want -
437+
// then do not check if we can
438+
if (lastTraining != null && minTrainingFrequency.toMinutes() != 0)
439+
{
440+
Instant now = FBUtilities.now();
441+
int minTrainingFrequencyMinutes = minTrainingFrequency.toMinutes();
442+
if (lastTraining.isAfter(now.minus(minTrainingFrequencyMinutes, ChronoUnit.MINUTES)))
443+
{
444+
Instant nextEarliestTraining = lastTraining.plus(minTrainingFrequencyMinutes, ChronoUnit.MINUTES);
445+
throw new IllegalArgumentException(format("The next training or importing can occur only at least after %s from the last training which happened at %s. " +
446+
"You can train again no earlier than at %s.",
447+
minTrainingFrequency,
448+
lastTraining,
449+
nextEarliestTraining));
450+
}
451+
}
452+
}
453+
397454
private int internalTrainingParameterResolution(CompressionParams compressionParams,
398455
String userSuppliedValue,
399456
String parameterName,

src/java/org/apache/cassandra/io/compress/IDictionaryCompressor.java

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.cassandra.io.compress;
2020

2121
import org.apache.cassandra.config.DataStorageSpec;
22+
import org.apache.cassandra.config.DurationSpec;
2223
import org.apache.cassandra.db.compression.CompressionDictionary;
2324
import org.apache.cassandra.exceptions.ConfigurationException;
2425

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

45+
String TRAINING_MIN_FREQUENCY_PARAMETER_NAME = "training_min_frequency";
46+
// 0m means there is no limit how often we can train, if this is set to e.g. 1h, that means
47+
// that once we train a dictionary for given table, then we can train again after at least 1 hour.
48+
String DEFAULT_TRAINING_MIN_FREQUENCY = "0m";
49+
4450
/**
4551
* Validates value of a parameter for training purposes. The value to validate should
4652
* be accepted by {@link DataStorageSpec.IntKibibytesBound}. This method is used upon validation
@@ -49,7 +55,7 @@ public interface IDictionaryCompressor<T extends CompressionDictionary>
4955
* @param parameterName name of a parameter to validate
5056
* @param resolvedValue value to validate
5157
*/
52-
static void validateTrainingParameter(String parameterName, String resolvedValue)
58+
static void validateSizeBasedTrainingParameter(String parameterName, String resolvedValue)
5359
{
5460
try
5561
{
@@ -62,6 +68,27 @@ static void validateTrainingParameter(String parameterName, String resolvedValue
6268
}
6369
}
6470

71+
/**
72+
* Validates value of a parameter for training purposes. The value to validate should
73+
* be accepted by {@link DurationSpec.IntMinutesBound}. This method is used upon validation of input parameters
74+
* in the implementation of dictionary compressor.
75+
*
76+
* @param parameterName name of a parameter to validate
77+
* @param resolvedValue value to validate
78+
*/
79+
static void validateDurationBasedTrainingParameter(String parameterName, String resolvedValue)
80+
{
81+
try
82+
{
83+
new DurationSpec.IntMinutesBound(resolvedValue).toMinutes();
84+
}
85+
catch (Throwable t)
86+
{
87+
throw new ConfigurationException(format("Unable to set value to parameter %s: %s. Reason: %s",
88+
parameterName, resolvedValue, t.getMessage()));
89+
}
90+
}
91+
6592
/**
6693
* Returns a compressor instance configured with the specified compression dictionary.
6794
* <br>

src/java/org/apache/cassandra/io/compress/ZstdDictionaryCompressor.java

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,8 @@
3939
import org.apache.cassandra.db.compression.ZstdCompressionDictionary;
4040
import org.apache.cassandra.utils.concurrent.Ref;
4141

42-
import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateTrainingParameter;
42+
import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateDurationBasedTrainingParameter;
43+
import static org.apache.cassandra.io.compress.IDictionaryCompressor.validateSizeBasedTrainingParameter;
4344

4445
public class ZstdDictionaryCompressor extends ZstdCompressorBase implements ICompressor, IDictionaryCompressor<ZstdCompressionDictionary>
4546
{
@@ -77,12 +78,15 @@ public static ZstdDictionaryCompressor create(Map<String, String> options)
7778
{
7879
int level = getOrDefaultCompressionLevel(options);
7980
validateCompressionLevel(level);
80-
validateTrainingParameter(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
81-
options.getOrDefault(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
82-
DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE));
83-
validateTrainingParameter(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
84-
options.getOrDefault(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
85-
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE));
81+
validateSizeBasedTrainingParameter(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
82+
options.getOrDefault(TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
83+
DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE));
84+
validateSizeBasedTrainingParameter(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
85+
options.getOrDefault(TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
86+
DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE));
87+
validateDurationBasedTrainingParameter(TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
88+
options.getOrDefault(TRAINING_MIN_FREQUENCY_PARAMETER_NAME,
89+
DEFAULT_TRAINING_MIN_FREQUENCY));
8690
return getOrCreate(level, null);
8791
}
8892

@@ -119,7 +123,8 @@ private ZstdDictionaryCompressor(int level, ZstdCompressionDictionary dictionary
119123
{
120124
super(level, Set.of(COMPRESSION_LEVEL_OPTION_NAME,
121125
TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME,
122-
TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME));
126+
TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME,
127+
TRAINING_MIN_FREQUENCY_PARAMETER_NAME));
123128
this.dictionary = dictionary;
124129
this.dictionaryRef = dictionaryRef;
125130
}

0 commit comments

Comments
 (0)