1818
1919package org .apache .cassandra .db .compression ;
2020
21+ import java .util .Map ;
22+
2123import com .google .common .base .Preconditions ;
2224
25+ import org .apache .cassandra .config .DataStorageSpec ;
26+ import org .apache .cassandra .config .DurationSpec ;
27+ import org .apache .cassandra .exceptions .ConfigurationException ;
28+ import org .apache .cassandra .schema .CompressionParams ;
29+
30+ import static java .lang .String .format ;
31+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE ;
32+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE ;
33+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .DEFAULT_TRAINING_MIN_FREQUENCY ;
34+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME ;
35+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME ;
36+ import static org .apache .cassandra .io .compress .IDictionaryCompressor .TRAINING_MIN_FREQUENCY_PARAMETER_NAME ;
37+
2338/**
2439 * Configuration for dictionary training parameters.
2540 */
@@ -29,13 +44,15 @@ public class CompressionDictionaryTrainingConfig
2944 public final int maxTotalSampleSize ;
3045 public final int acceptableTotalSampleSize ;
3146 public final int chunkSize ;
47+ public final int minTrainingFrequency ;
3248
3349 private CompressionDictionaryTrainingConfig (Builder builder )
3450 {
3551 this .maxDictionarySize = builder .maxDictionarySize ;
3652 this .maxTotalSampleSize = builder .maxTotalSampleSize ;
3753 this .acceptableTotalSampleSize = builder .maxTotalSampleSize / 10 * 8 ;
3854 this .chunkSize = builder .chunkSize ;
55+ this .minTrainingFrequency = builder .minTrainingFrequency ;
3956 }
4057
4158 public static Builder builder ()
@@ -48,6 +65,7 @@ public static class Builder
4865 private int maxDictionarySize = 65536 ; // 64KB default
4966 private int maxTotalSampleSize = 10 * 1024 * 1024 ; // 10MB total
5067 private int chunkSize = 64 * 1024 ; // 64KB default
68+ private int minTrainingFrequency = 0 ; // in minutes
5169
5270 public Builder maxDictionarySize (int size )
5371 {
@@ -67,12 +85,121 @@ public Builder chunkSize(int chunkSize)
6785 return this ;
6886 }
6987
88+ public Builder minTrainingFrequency (int minTrainingFrequency )
89+ {
90+ this .minTrainingFrequency = minTrainingFrequency ;
91+ return this ;
92+ }
93+
7094 public CompressionDictionaryTrainingConfig build ()
7195 {
7296 Preconditions .checkArgument (maxDictionarySize > 0 , "maxDictionarySize must be positive" );
7397 Preconditions .checkArgument (maxTotalSampleSize > 0 , "maxTotalSampleSize must be positive" );
7498 Preconditions .checkArgument (chunkSize > 0 , "chunkSize must be positive" );
99+ Preconditions .checkArgument (minTrainingFrequency >= 0 , "min training frequency must be non-negative" );
75100 return new CompressionDictionaryTrainingConfig (this );
76101 }
77102 }
103+
104+ public static int getMaxDictionarySize (Map <String , String > params )
105+ {
106+ return validateSizeBasedTrainingParameter (TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME ,
107+ params .getOrDefault (TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME ,
108+ DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE ));
109+ }
110+
111+ public static int getMaxTotalSampleSize (Map <String , String > params )
112+ {
113+ return validateSizeBasedTrainingParameter (TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME ,
114+ params .getOrDefault (TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME ,
115+ DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE ));
116+ }
117+
118+ public static int getMinTrainingFrequency (Map <String , String > params )
119+ {
120+ return validateDurationBasedTrainingParameter (TRAINING_MIN_FREQUENCY_PARAMETER_NAME ,
121+ params .getOrDefault (TRAINING_MIN_FREQUENCY_PARAMETER_NAME ,
122+ DEFAULT_TRAINING_MIN_FREQUENCY ));
123+ }
124+
125+ public static int getMaxDictionarySizeWithUserSuppliedParams (CompressionParams compressionParams , Map <String , String > parameters )
126+ {
127+ return internalTrainingParameterResolution (compressionParams ,
128+ parameters .get (TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME ),
129+ TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_NAME ,
130+ DEFAULT_TRAINING_MAX_DICTIONARY_SIZE_PARAMETER_VALUE );
131+ }
132+
133+ public static int getMaxTotalSampleSizeWithUserSuppliedParams (CompressionParams compressionParams , Map <String , String > parameters )
134+ {
135+ return internalTrainingParameterResolution (compressionParams ,
136+ parameters .get (TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME ),
137+ TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_NAME ,
138+ DEFAULT_TRAINING_MAX_TOTAL_SAMPLE_SIZE_PARAMETER_VALUE );
139+ }
140+
141+ private static int internalTrainingParameterResolution (CompressionParams compressionParams ,
142+ String userSuppliedValue ,
143+ String parameterName ,
144+ String defaultParameterValue )
145+ {
146+ String resolvedValue = null ;
147+ try
148+ {
149+ if (userSuppliedValue == null )
150+ resolvedValue = compressionParams .getOtherOptions ().getOrDefault (parameterName , defaultParameterValue );
151+ else
152+ resolvedValue = userSuppliedValue ;
153+
154+ return new DataStorageSpec .IntBytesBound (resolvedValue ).toBytes ();
155+ }
156+ catch (Throwable t )
157+ {
158+ throw new IllegalArgumentException (String .format ("Invalid value for %s: %s" , parameterName , resolvedValue ));
159+ }
160+ }
161+
162+ /**
163+ * Validates value of a parameter for training purposes. The value to validate should
164+ * be accepted by {@link DataStorageSpec.IntKibibytesBound}. This method is used upon validation
165+ * of input parameters in the implementations of dictionary compressor.
166+ *
167+ * @param parameterName name of a parameter to validate
168+ * @param resolvedValue value to validate
169+ * @return resolved value in bytes
170+ */
171+ static int validateSizeBasedTrainingParameter (String parameterName , String resolvedValue )
172+ {
173+ try
174+ {
175+ return new DataStorageSpec .IntBytesBound (resolvedValue ).toBytes ();
176+ }
177+ catch (Throwable t )
178+ {
179+ throw new ConfigurationException (format ("Unable to set value to parameter %s: %s. Reason: %s" ,
180+ parameterName , resolvedValue , t .getMessage ()));
181+ }
182+ }
183+
184+ /**
185+ * Validates value of a parameter for training purposes. The value to validate should
186+ * be accepted by {@link DurationSpec.IntMinutesBound}. This method is used upon validation of input parameters
187+ * in the implementation of dictionary compressor.
188+ *
189+ * @param parameterName name of a parameter to validate
190+ * @param resolvedValue value to validate
191+ * @return resolved value in minutes
192+ */
193+ static int validateDurationBasedTrainingParameter (String parameterName , String resolvedValue )
194+ {
195+ try
196+ {
197+ return new DurationSpec .IntMinutesBound (resolvedValue ).toMinutes ();
198+ }
199+ catch (Throwable t )
200+ {
201+ throw new ConfigurationException (format ("Unable to set value to parameter %s: %s. Reason: %s" ,
202+ parameterName , resolvedValue , t .getMessage ()));
203+ }
204+ }
78205}
0 commit comments