3030import java .util .LinkedHashSet ;
3131import java .util .List ;
3232import java .util .Map ;
33+ import java .util .Random ;
3334import java .util .Set ;
3435import java .util .UUID ;
3536import java .util .concurrent .ExecutionException ;
5859import org .apache .cassandra .cql3 .functions .types .TypeCodec ;
5960import org .apache .cassandra .cql3 .functions .types .UDTValue ;
6061import org .apache .cassandra .cql3 .functions .types .UserType ;
62+ import org .apache .cassandra .db .compression .CompressionDictionary ;
63+ import org .apache .cassandra .db .compression .CompressionDictionary .DictId ;
64+ import org .apache .cassandra .db .compression .CompressionDictionaryTrainingConfig ;
65+ import org .apache .cassandra .db .compression .ZstdCompressionDictionary ;
66+ import org .apache .cassandra .db .compression .ZstdDictionaryTrainer ;
6167import org .apache .cassandra .db .marshal .FloatType ;
6268import org .apache .cassandra .db .marshal .UTF8Type ;
6369import org .apache .cassandra .dht .ByteOrderedPartitioner ;
8490import org .apache .cassandra .utils .FBUtilities ;
8591import org .apache .cassandra .utils .JavaDriverUtils ;
8692import org .apache .cassandra .utils .OutputHandler ;
87- import org .assertj .core .api .Assertions ;
8893
94+ import static org .apache .cassandra .db .compression .CompressionDictionary .Kind .ZSTD ;
8995import static org .apache .cassandra .utils .Clock .Global .currentTimeMillis ;
9096import static org .assertj .core .api .Assertions .assertThat ;
97+ import static org .assertj .core .api .Assertions .assertThatThrownBy ;
9198import static org .junit .Assert .assertEquals ;
9299import static org .junit .Assert .assertFalse ;
93100import static org .junit .Assert .assertNotNull ;
@@ -1671,7 +1678,7 @@ public void testConstraintViolation() throws Exception
16711678
16721679 writer .addRow (1 , 4 );
16731680
1674- Assertions . assertThatThrownBy (() -> writer .addRow (2 , 11 ))
1681+ assertThatThrownBy (() -> writer .addRow (2 , 11 ))
16751682 .describedAs ("Should throw when adding a row that violates constraints" )
16761683 .isInstanceOf (ConstraintViolationException .class )
16771684 .hasMessageContaining ("Column value does not satisfy value constraint for column 'v1'. It should be v1 < 5" );
@@ -1690,6 +1697,111 @@ public void testConstraintViolation() throws Exception
16901697 }
16911698 }
16921699
1700+ @ Test
1701+ public void testWritingWithZstdDictionaryWhenUsingInvalidCompressor ()
1702+ {
1703+ // the compressor is not dictionary-aware so we will fail
1704+ final String schema = "CREATE TABLE " + qualifiedTable + " ("
1705+ + " k int,"
1706+ + " v1 text,"
1707+ + " PRIMARY KEY (k)"
1708+ + ") WITH compression = {'class': 'ZstdCompressor'}" ;
1709+
1710+ assertThatThrownBy (() -> CQLSSTableWriter .builder ()
1711+ .inDirectory (dataDir )
1712+ .forTable (schema )
1713+ .using ("INSERT INTO " + keyspace + '.' + table + " (k, v1) VALUES (?, ?)" )
1714+ // does not matter, we will fail anyway
1715+ .withCompressionDictionary (new ZstdCompressionDictionary (new DictId (ZSTD , 1 ), new byte [0 ]))
1716+ .build ())
1717+ .hasMessage ("Table's compressor can not accept any dictionary: {chunk_length_in_kb=16, class=org.apache.cassandra.io.compress.ZstdCompressor}" )
1718+ .isInstanceOf (IllegalStateException .class );
1719+ }
1720+
1721+ @ Test
1722+ public void testWritingWithZstdDictionary () throws Exception
1723+ {
1724+ final String schema = "CREATE TABLE " + qualifiedTable + " ("
1725+ + " k int,"
1726+ + " v1 text,"
1727+ + " PRIMARY KEY (k)"
1728+ + ") WITH compression = {'class': 'ZstdDictionaryCompressor'}" ;
1729+
1730+ CompressionDictionary dictionary = DictionaryHelper .trainDictionary (keyspace , table );
1731+
1732+ CQLSSTableWriter writer = CQLSSTableWriter .builder ()
1733+ .inDirectory (dataDir )
1734+ .forTable (schema )
1735+ .using ("INSERT INTO " + keyspace + '.' + table + " (k, v1) VALUES (?, ?)" )
1736+ .withCompressionDictionary (dictionary )
1737+ .build ();
1738+
1739+ for (int i = 0 ; i < 500 ; i ++)
1740+ {
1741+ writer .addRow (i , DictionaryHelper .INSTANCE .getRandomSample ());
1742+ }
1743+
1744+ writer .close ();
1745+
1746+ loadSSTables (dataDir , keyspace , table );
1747+
1748+ if (verifyDataAfterLoading )
1749+ {
1750+ UntypedResultSet resultSet = QueryProcessor .executeInternal ("SELECT * FROM " + qualifiedTable );
1751+ assertNotNull (resultSet );
1752+ Iterator <UntypedResultSet .Row > iter = resultSet .iterator ();
1753+ for (int i = 0 ; i < 500 ; i ++)
1754+ {
1755+ UntypedResultSet .Row row = iter .next ();
1756+ assertEquals (i , row .getInt ("k" ));
1757+ assertNotNull (row .getString ("v1" ));
1758+ }
1759+ }
1760+ }
1761+
1762+ /**
1763+ * Simple generator of random data for Zstd compression dictionary and dictionary trainer.
1764+ */
1765+ private static class DictionaryHelper
1766+ {
1767+ public static final DictionaryHelper INSTANCE = new DictionaryHelper ();
1768+ private static final Random random = new Random ();
1769+
1770+ private static final String [] dates = new String [] {"2025-10-20" ,"2025-10-19" ,"2025-10-18" ,"2025-10-17" ,"2025-10-16" };
1771+ private static final String [] times = new String [] {"11:00:01" ,"11:00:02" ,"11:00:03" ,"11:00:04" ,"11:00:05" };
1772+ private static final String [] levels = new String [] {"TRACE" , "DEBUG" , "INFO" , "WARN" , "ERROR" };
1773+ private static final String [] services = new String [] {"com.example.UserService" , "com.example.DatabasePool" , "com.example.PaymentService" , "com.example.OrderService" };
1774+
1775+ private String getRandomSample ()
1776+ {
1777+ return dates [random .nextInt (dates .length )] + ' ' +
1778+ times [random .nextInt (times .length )] + ' ' +
1779+ levels [random .nextInt (levels .length )] + ' ' +
1780+ services [random .nextInt (services .length )] + ' ' +
1781+ UUID .randomUUID (); // message
1782+ }
1783+
1784+ private static CompressionDictionary trainDictionary (String keyspace , String table )
1785+ {
1786+ CompressionDictionaryTrainingConfig config = CompressionDictionaryTrainingConfig
1787+ .builder ()
1788+ .maxDictionarySize (65536 )
1789+ .maxTotalSampleSize (1024 * 1024 ) // 1MB total
1790+ .build ();
1791+
1792+ try (ZstdDictionaryTrainer trainer = new ZstdDictionaryTrainer (keyspace , table , config , 3 ))
1793+ {
1794+ trainer .start (true );
1795+ for (int i = 0 ; i < 25000 ; i ++)
1796+ {
1797+ trainer .addSample (UTF8Type .instance .fromString (DictionaryHelper .INSTANCE .getRandomSample ()));
1798+ }
1799+
1800+ return trainer .trainDictionary (false );
1801+ }
1802+ }
1803+ }
1804+
16931805 protected static void loadSSTables (File dataDir , final String ks , final String tb ) throws ExecutionException , InterruptedException
16941806 {
16951807 SSTableLoader loader = new SSTableLoader (dataDir , new SSTableLoader .Client ()
0 commit comments