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
10 changes: 10 additions & 0 deletions doc/modules/cassandra/pages/managing/operating/compression.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,16 @@ require retraining dictionaries to maintain optimal compression ratios.
Monitor the `SSTable Compression Ratio` via `nodetool tablestats` to
detect degradation.

=== Available nodetool commands for compressiondictionary

There are these four commands for now related to compression dictionaries:

* train - training a dictionary, described above.
* list - a user can list all dictionaries for given keyspace and table
* export - a user can export a compression dictionary of a keyspace and a table, either the last one or
by a specific id, to a file.
* import - a user can import a compression dictionary, exported by above command, from a file to a cluster.

== Advanced Use

Advanced users can provide their own compression class by implementing
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,13 @@ public interface CompressionDictionary extends AutoCloseable
*/
byte[] rawDictionary();

/**
* Get checksum of this dictionary.
*
* @return checksum of this dictionary
*/
int checksum();

/**
* Get the kind of the compression algorithm
*
Expand Down Expand Up @@ -126,7 +133,7 @@ static CompressionDictionary deserialize(DataInput input, @Nullable CompressionD
throw new IOException("Compression dictionary checksum does not match. " +
"Expected: " + checksum + "; actual: " + calculatedChecksum);

CompressionDictionary dictionary = kind.createDictionary(dictId, dict);
CompressionDictionary dictionary = kind.createDictionary(dictId, dict, checksum);

// update the dictionary manager if it exists
if (manager != null)
Expand All @@ -137,6 +144,29 @@ static CompressionDictionary deserialize(DataInput input, @Nullable CompressionD
return dictionary;
}

static LightweightCompressionDictionary createFromRowLightweight(UntypedResultSet.Row row)
{
String kindStr = row.getString("kind");
long dictId = row.getLong("dict_id");
int checksum = row.getInt("dict_checksum");
int size = row.getInt("dict_length");
String keyspaceName = row.getString("keyspace_name");
String tableName = row.getString("table_name");

try
{
return new LightweightCompressionDictionary(keyspaceName,
tableName,
new DictId(CompressionDictionary.Kind.valueOf(kindStr), dictId),
checksum,
size);
}
catch (IllegalArgumentException ex)
{
throw new IllegalStateException(kindStr + " compression dictionary is not created for dict id " + dictId);
}
}

static CompressionDictionary createFromRow(UntypedResultSet.Row row)
{
String kindStr = row.getString("kind");
Expand Down Expand Up @@ -164,7 +194,7 @@ static CompressionDictionary createFromRow(UntypedResultSet.Row row)
kindStr, dictId, storedChecksum, calculatedChecksum));
}

return kind.createDictionary(new DictId(kind, dictId), dict);
return kind.createDictionary(new DictId(kind, dictId), row.getByteArray("dict"), storedChecksum);
}
catch (IllegalArgumentException ex)
{
Expand All @@ -188,9 +218,10 @@ enum Kind
// Order matters: the enum ordinal is serialized
ZSTD
{
public CompressionDictionary createDictionary(DictId dictId, byte[] dict)
@Override
public CompressionDictionary createDictionary(DictId dictId, byte[] dict, int checksum)
{
return new ZstdCompressionDictionary(dictId, dict);
return new ZstdCompressionDictionary(dictId, dict, checksum);
}

@Override
Expand Down Expand Up @@ -220,9 +251,10 @@ public ICompressionDictionaryTrainer createTrainer(String keyspaceName,
*
* @param dictId the dictionary identifier
* @param dict the raw dictionary bytes
* @param checksum checksum of this dictionary
* @return a compression dictionary instance
*/
public abstract CompressionDictionary createDictionary(CompressionDictionary.DictId dictId, byte[] dict);
public abstract CompressionDictionary createDictionary(CompressionDictionary.DictId dictId, byte[] dict, int checksum);

/**
* Creates a dictionary compressor for this kind
Expand Down Expand Up @@ -281,4 +313,32 @@ public String toString()
'}';
}
}

/**
* The purpose of lightweight dictionary is to not carry the actual dictionary bytes for performance reasons.
* Handy for situations when retrieval from the database does not need to contain dictionary
* or the instantiation of a proper dictionary object is not desirable or unnecessary for other,
* mostly performance-related, reasons.
*/
class LightweightCompressionDictionary
{
public final String keyspaceName;
public final String tableName;
public final DictId dictId;
public final int checksum;
public final int size;

public LightweightCompressionDictionary(String keyspaceName,
String tableName,
DictId dictId,
int checksum,
int size)
{
this.keyspaceName = keyspaceName;
this.tableName = tableName;
this.dictId = dictId;
this.checksum = checksum;
this.size = size;
}
}
}
Loading