Kafka Streams State Store implementation that persists data to Apache Cassandra. For now, only KeyValueStore type is supported.
ℹ️ Kafka Streams is a Java client library for building stream-processing applications and microservices, where the input and output data are stored in Kafka clusters.
ℹ️ Apache Cassandra is a free and open-source, distributed, wide-column store, NoSQL database management system designed to handle large amounts of data across many commodity servers, providing high availability with no single point of failure.
- Blog: https://thriving.dev/blog/introducing-kafka-streams-cassandra-state-store
- Demo: https://youtu.be/2Co9-8E-uJE
- Java 17
- kafka-streams 3.6
- datastax java-driver-core 4.17.0
- Kafka Streams 2.7.0+ (maybe even earlier versions, but wasn't tested further back)
- Datastax java client (v4)
'com.datastax.oss:java-driver-core:4.17.0'
- ScyllaDB shard-aware datastax java client (v4) fork
'com.scylladb:java-driver-core:4.17.0.0'
- Apache Cassandra 3.11
- Apache Cassandra 4.0
- Apache Cassandra 4.1
- ScyllaDB (tested from 4.3+)
- JUnit 5, AssertJ
- testcontainers
kafka-streams-cassandra-state-store is available on Maven Central:
<dependency>
<groupId>dev.thriving.oss</groupId>
<artifactId>kafka-streams-cassandra-state-store</artifactId>
<version>${version}</version>
</dependency>
Classes of this library are in the package dev.thriving.oss.kafka.streams.cassandra.state.store
.
implementation 'dev.thriving.oss:kafka-streams-cassandra-state-store:${version}'
To avoid library collisions, the cassandra java driver is non-transitive.
Therefore you have to choose and add a datastax driver based java client dependency to your project.
- Datastax java client (v4)
'com.datastax.oss:java-driver-core:4.17.0'
(works for Cassandra 3.11, 4.0, 4.11) - ScyllaDB shard-aware datastax java client (v4) fork
'com.scylladb:java-driver-core:4.17.0.0'
- Disable logging =>
withLoggingDisabled()
...enabled by default, kafka streams is 'logging' the events making up the store's state against a changelog topic to be able to restore state following a rebalance or application restart. Since cassandra is a permanent external store, state does not need to be restored but is always available. - Disable caching =>
withCachingDisabled()
...enabled by default, kafka streams is buffering writes - which is not what we want when working with cassandra state store - Do not use standby replicas =>
num.standby.replicas=0
...standby replicas are used to minimize the latency of task failover by keeping shadow copies of local state stores as a hot standby. The state store backed by cassandra does not need to be restored or re-balanced since all streams instances can directly access any partitions state.
When using the high-level DSL, i.e., StreamsBuilder
, users create StoreSupplier
s that can be further customized via Materialized
.
For example, a topic read as KTable
can be materialized into a cassandra k/v store with custom key/value serdes, with logging and caching disabled:
StreamsBuilder builder = new StreamsBuilder();
KTable<Long,String> table = builder.table(
"topicName",
Materialized.<Long,String>as(
CassandraStores.builder(session, "store-name")
.partitionedKeyValueStore()
)
.withKeySerde(Serdes.Long())
.withValueSerde(Serdes.String())
.withLoggingDisabled()
.withCachingDisabled());
When using the Processor API, i.e., Topology
, users create StoreBuilder
s that can be attached to Processor
s.
For example, you can create a cassandra stringKey value store with custom key/value serdes, logging and caching disabled like:
StoreBuilder<KeyValueStore<String, Long>> sb =
Stores.keyValueStoreBuilder(
CassandraStores.builder(session, "store-name")
.partitionedKeyValueStore(),
Serdes.String(),
Serdes.Long())
.withLoggingDisabled()
.withCachingDisabled();
topology.addStateStore(sb);
Examples (incl. docker-compose setup) can be found in the /examples folder.
Instructions on how to run and work with the example apps can be found at the individual example root folder's README file.
Take a look at the notorious word-count example with Cassandra 4 -> /examples/word-count-cassandra4.
- Docker to run
- kcat for interacting with Kafka (consume/produce)
kafka-streams-cassandra-state-store comes with 4 different store types:
- partitionedKeyValueStore
- globalKeyValueStore
- partitionedVersionedKeyValueStore
- globalVersionedKeyValueStore
A persistent KeyValueStore<Bytes, byte[]>
.
The underlying cassandra table is partitioned by the store context task partition.
Therefore, it behaves exactly like the regular state stores (RocksDB/InMemory/MemoryLRUCache).
All CRUD operations against this store always query by and return results for a single stream task.
A persistent KeyValueStore<Bytes, byte[]>
.
The underlying cassandra table uses the record key as sole PRIMARY KEY.
Therefore, all CRUD operations against this store work from any streams task and therefore always are “global”.
Due to the nature of cassandra tables having a single PK (no clustering key), this store supports only a limited number of operations.
This global store should not be confused with a Kafka Streams Global Store! It has to be used as a non-global (regular!) streams KeyValue state store - though it allows to read entries from any streams context (streams task/thread).
Tip: This store type can be useful when exposing state store access via an API. Each running instance of your app can serve all requests without the need to proxy the request to the right instance having the streams task assigned for the key in question.
WrappingStoreProvider
to a single (assigned) partition.
The KafkaStreams instance returns a CompositeReadOnlyKeyValueStore
that holds the WrappingStoreProvider
, wrapping all assigned tasks' stores. Without the correct StoreQueryParameters
the same query is executed multiple times (for all assigned partitions) and combines multiple identical results.
A persistent VersionedKeyValueStore<Bytes, byte[]>
.
The underlying cassandra table is partitioned by the store context task partition.
Therefore, it behaves exactly like the regular versioned state store (RocksDB).
All CRUD operations against this store always query by and return results for a single stream task.
A persistent VersionedKeyValueStore<Bytes, byte[]>
.
The underlying cassandra table uses the record key + validTo as composite PRIMARY KEY (validTo as the clustering key).
Therefore, all CRUD operations against this store work from any streams task and therefore always are “global”.
The CassandraStateStore
interface provides static helper methods to get a correctly configured read-only store facade:
💡Please read the blog post for more details: https://thriving.dev/blog/interactive-queries-with-kafka-streams-cassandra-state-store
globalKeyValueStore:
// get a read-only store to exec interactive queries ('global' type cassandra KeyValueStore)
ReadOnlyKeyValueStore<String, Long> store = CassandraStateStore.readOnlyGlobalKeyValueStore(streams, STORE_NAME);
// Get the value from the store
Long value = store.get(key);
Example provided: examples/global-store-restapi
partitionedKeyValueStore:
Get an optimised special implementation of {@link ReadOnlyKeyValueStore} for 'local' type CassandraKeyValueStore.
The returned object can be used to query the state directly from the underlying Cassandra table.
No 'RPC layer' is required since queries for all/individual partitions are executed from this instance, and query
results are merged where necessary.
// get a read-only store to exec interactive queries ('partitioned' type cassandra KeyValueStore)
ReadOnlyKeyValueStore<String, Long> store = CassandraStateStore.readOnlyPartitionedKeyValueStore(
streams, // streams
"word-count", // storeName
session, // session
"kstreams_wordcount", // keyspace
true, // isCountAllEnabled
"dml", // dmlExecutionProfile
stringSerde, // keySerde
longSerde, // valueSerde
CassandraStateStore.DEFAULT_TABLE_NAME_FN, // tableNameFn
new DefaultStreamPartitioner<>(keySerde.serializer()) // partitioner
);
// Get the value from the store
Long value = store.get(key);
CassandraPartitionedReadOnlyKeyValueStore
requires application.server
config to be set (to be able to access metadata).
Example provided: examples/partitioned-store-restapi
More examples can also be found in the integration tests.
partitionedVersionedKeyValueStore/globalVersionedKeyValueStore:
With Kafka 3.5 interactive queries interfaces are not yet available for versioned key value stores. Plans exist to add this in the future.
Following KIPs have been identified (asOfTImestamp 2023-08-25): KIP-960, KIP-968, KIP-969.
partitionedKeyValueStore | globalKeyValueStore | |
---|---|---|
get | ✅ | ✅ |
put | ✅ | ✅ |
putIfAbsent | ✅ | ✅ |
putAll | ✅ | ✅ |
delete | ✅ | ✅ |
range | ✅ | ❌ |
reverseRange | ✅ | ❌ |
all | ✅ | ✅ |
reverseAll | ✅ | ❌ |
prefixScan | ✅ | ❌ |
approximateNumEntries | ✅* | ✅* |
query::RangeQuery | ✅ | ❌ |
query::KeyQuery | ✅ | ✅ |
query::WindowKeyQuery | ❌ | ❌ |
query::WindowRangeQuery | ❌ | ❌ |
*opt-in required
partitionedVersionedKeyValueStore | globalVersionedKeyValueStore | |
---|---|---|
get(key) | ✅ | ✅ |
get(key, asOfTimestamp) | ✅ | ✅ |
put(key, value, timestamp) | ✅ | ✅ |
delete(key, timestamp) | ✅ | ✅ |
The CassandraStores
class provides a method public static CassandraStores builder(final CqlSession session, final String name)
that returns an instance of CassandraStores which ultimately is used to build an instance of KeyValueBytesStoreSupplier
to add to your topology.
Basic usage example:
CassandraStores.builder(session, "word-grouped-count")
.withKeyspace("")
.partitionedKeyValueStore()
Advanced usage example:
CassandraStores.builder(session, "word-grouped-count")
.withKeyspace("poc")
.withCountAllEnabled()
.withTableOptions("""
compaction = { 'class' : 'LeveledCompactionStrategy' }
AND default_time_to_live = 86400
""")
.withTableNameFn(storeName ->
String.format("%s_kstreams_store", storeName.toLowerCase().replaceAll("[^a-z0-9_]", "_")))
.partitionedKeyValueStore()
Please also see Quick start for full kafka-streams example.
The keyspace for the state store to operate in. By default, the provided CqlSession
session-keyspace is used.
A CQL table has a number of options that can be set at creation.
Please omit WITH
prefix.
Multiple options can be added using AND
, e.g. "table_option1 AND table_option2"
.
Recommended compaction strategy is 'LeveledCompactionStrategy' which is applied by default.
-> Do not forget to add when overwriting table options.
Please refer to table options of your cassandra cluster.
Please note this config will only apply upon initial table creation. ('ALTER TABLE' is not yet supported).
Default: "compaction = { 'class' : 'LeveledCompactionStrategy' }"
Customize how the state store cassandra table is named, based on the kstreams store name.
Default: ${normalisedStoreName}_kstreams_store
- normalise := lowercase, replaces all [^a-z0-9_] with '_'
e.g. ("TEXT3.word-count2") -> "text3_word_count2_kstreams_store"
Enable (opt-in) the CassandraKeyValueStore to use SELECT COUNT(*)
when ReadOnlyKeyValueStore#approximateNumEntries() is invoked.
SELECT COUNT(*)
requires significant CPU and I/O resources and may be quite slow depending on store size... use with care!
Disabled by default.
Disable (opt-out) automatic table creation during store initialization.
Enabled by default.
Set the execution profile to be used by the driver for all DDL (Data Definition Language) queries.
ℹ️ Note: Only applies if table creation ({@link CassandraStores#withCreateTableDisabled()}) is enabled (default).
If no profile is set - DDL queries are executed with consistency ALL
.
When using a custom profile, it is recommended to also set consistency=ALL
(Reason: avoid issues with concurrent schema updates)
Must be a non-blank String.
Set to null
to disable (basic applies).
Default: null
Set the execution profile to be used by the driver for all DML (Data Manipulation Language) queries.
Reference: https://docs.datastax.com/en/developer/java-driver/4.15/manual/core/configuration/#execution-profiles"
Must be a non-blank String.
Set to null
to disable (basic applies).
Default: null
Adding additional infrastructure for data persistence external to Kafka comes with certain risks and constraints.
Kafka Streams supports at-least-once and exactly-once processing guarantees. At-least-once semantics is enabled by default.
Kafka Streams exactly-once processing guarantees is using Kafka transactions. These transactions wrap the entirety of processing a message throughout your streams topology, including messages published to outbound topic(s), changelog topic(s), and consumer offsets topic(s).
This is possible through transactional interaction with a single distributed system (Apache Kafka). Bringing an external system (Cassandra) into play breaks this pattern. Once data is written to the database it can't be rolled back in the event of a subsequent error / failure to complete the current message processing.
processing.guarantee="exactly_once_v2"
), and/or your processing logic is not fully idempotent then using kafka-streams-cassandra-state-store is discouraged!
ℹ️ Please note this is also true when using kafka-streams with the native state stores (RocksDB/InMemory) with at-least-once processing.guarantee (default).
For more information on Kafka Streams processing guarantees, check the sources referenced below.
- https://medium.com/lydtech-consulting/kafka-streams-transactions-exactly-once-messaging-82194b50900a
- https://docs.confluent.io/platform/current/streams/developer-guide/config-streams.html#processing-guarantee
- https://docs.confluent.io/platform/current/streams/concepts.html#processing-guarantees
Not all methods have been implemented. Please check store types method support table above for more details.
Using defaults, for a state store named "word-count" following CQL Schema applies:
CREATE TABLE IF NOT EXISTS word_count_kstreams_store (
partition INT,
key BLOB,
value BLOB,
time TIMESTAMP,
PRIMARY KEY ((partition), key)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }
Using defaults, for a state store named "clicks-global" following CQL Schema applies:
CREATE TABLE IF NOT EXISTS clicks_global_kstreams_store (
key BLOB,
value BLOB,
time TIMESTAMP,
PRIMARY KEY (key)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }
Using defaults, for a state store named "word-count" following CQL Schema applies:
CREATE TABLE IF NOT EXISTS word_count_kstreams_store (
partition INT,
key BLOB,
validFrom TIMESTAMP,
validTo TIMESTAMP,
value BLOB,
time TIMESTAMP,
PRIMARY KEY ((partition), key, validTo)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }
Using defaults, for a state store named "clicks-global" following CQL Schema applies:
CREATE TABLE IF NOT EXISTS clicks_global_kstreams_store (
key BLOB,
validFrom TIMESTAMP,
validTo TIMESTAMP,
value BLOB,
time TIMESTAMP,
PRIMARY KEY ((key), validTo)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }
💡 Tip: Cassandra has a table option default_time_to_live
(default expiration time (“TTL”) in seconds for a table) which can be useful for certain use cases where data (state) can or should expire.
Please note writes to cassandra are made with system time. The table TTL will therefore apply based on the time of write (not stream time).
Kafka is persisting data in segments and is built for sequential r/w. As long as there's sufficient disk storage space available to brokers, a high number of messages for a single topic partition is not a problem.
Apache Cassandra on the other hand can get inefficient (up to severe failures such as load shedding, dropped messages, and to crashed and downed nodes) when partition size grows too large. The reason is that searching becomes too slow as search within partition is slow. Also, it puts a lot of pressure on (JVM) heap.
For the current implementation, the cassandra table created for the 'default' key-value store is partitioned by the kafka partition key ("wide partition pattern").
Please keep these issues in mind when working with relevant data volumes.
In case you don't need to query your store / only lookup by key ('range', 'prefixScan'; ref Supported operations by store type) it's recommended to use globalKeyValueStore
rather than keyValueStore
since it is partitioned by the event key (:= primary key).
ℹ️ References:
- blog post on Wide Partitions in Apache Cassandra 3.11
Note: in case anyone has funded knowledge if/how this has changed with Cassandra 4, please share! - stackoverflow question
- Java 17
- Docker (integration tests with testcontainers)
This library is bundled with Gradle. Please note The build task also depends on task testInt which runs integration tests using testcontainers (build <- check <- intTest).
./gradlew clean build
Integration tests can be run separately via
./gradlew :kafka-streams-cassandra-state-store:intTest
- MVP
- CQL Schema
- implementation
- restructure code
- split implementation & examples
- Abstract store, introduce Repo, KeySerdes (Byte <> ByteBuffer|String)
- CassandraStores Builder, configurable
- table name fn
- keyspace
-
table default ttl -
compaction strategy -
compression - fully customizable table options (support both Cassandra & ScyllaDB)
- examples
- WordCount Cassandra 4
- WordCount Cassandra 3 (v4 client lib)
- WordCount ScyllaDB
- WordCount Processor + all + range + prefixScan + approximateNumEntries
- GlobalCassandraStore + KStream enrichment
- Quarkus examples app as GraalVM native image (#7)
- additional features
-
Prefix scan with(removed with v0.3)stringKeyValueStore
(ScyllaDB only) -
Prefix scan withstringKeyValueStore
(Cassandra with SASIIndex? https://stackoverflow.com/questions/49247092/order-by-and-like-in-same-cassandra-query/49268543#49268543) -
ReadOnlyKeyValueStore.prefixScan
implementation using range (see InMemoryKeyValueStore implementation) - Implement
globalKeyValueStore
- Support KIP-889: Versioned State Stores (to be delivered with kafka 3.5.0) (#21)
-
- OpenSource
- choose + add license
- add CHANGELOG.md
- add CODE_OF_CONDUCT.md
- add CONTRIBUTING.md
- polishing
- make repo public
- Publish to maven central (?) https://h4pehl.medium.com/publish-your-gradle-artifacts-to-maven-central-f74a0af085b1
- request namespace ownership
- add JavaDocs
- other -> maven central compliant https://central.sonatype.org/publish/requirements/
- gradle plugin to publish to maven central https://julien.ponge.org/blog/publishing-from-gradle-to-maven-central-with-github-actions/
- publish snapshot version 0.1.0-SNAPSHOT
- add gradle release plugin
- tag + publish initial version 0.1.0
- Ops
- github actions to build (+test)
- ? add renovate
- github actions to publish to maven central (snapshot, releases) https://julien.ponge.org/blog/publishing-from-gradle-to-maven-central-with-github-actions/
- github actions for triggering 'gradle release' from repo with automatic semver
- Write Documentation
- summary
- compatibility cassandra 3.11, 4.x, ScyllaDB
- cleanup README
- install
- quick start
- link to examples
- overview store types
- usage, builder, config options
- limitations
- Cassandra Specifics
- Underlying CQL Schema
- Feat: Cassandra table with default TTL
- r/w consistency
- profiles for DDL/DML
- retry-policy https://docs.datastax.com/en/developer/java-driver/4.15/manual/core/retries/
- ? request throttling, e.g. rate-based to avoid overloading the db? https://docs.datastax.com/en/developer/java-driver/4.15/manual/core/throttling/
- (Caching options)
- Exception/error handling
- Security
- test against 'CQL injection' via
withTableOptions(..)
=> tried to addcompaction = { 'class' : 'LeveledCompactionStrategy' };DROP TABLE xyz
which fails due to wrong syntax in Cassandra 3.11/4.1 & ScyllaDB 5.1
- test against 'CQL injection' via
- bugs
- tests
- unit tests (?)
- integration test using testcontainers
- WordCountTest
- WordCountInteractiveQueriesTest
- WordCountGlobalStoreTest
- Other
- migrate to gradle version catalogs https://docs.gradle.org/current/userguide/platforms.html, https://developer.android.com/build/migrate-to-catalogs
- Advanced/Features/POCs Planned/Considered
- correctness / completeness (#14)
-
wrap stores with MeteredKeyValueStore ?-> done automatically via builders -
provide-> no use casetimestampedKeyValueStore
-
? (TBC) logging / caching is always disabled (because it's not implemented to wrap store by CassandraStores...)-
always disable logging + caching?
-
-
- add additional store types
- WindowedStore functionality, example, ...
- ...?
- Add builder config options
- (?) simple inMemory read cache -> Caffeine? (separate lib?) (#18)
- Benchmark
- Explore buffered writes ('caching') -> parallel writes to Cassandra to boost performance?
- add Metrics?
- (?) Metrics also for Caches?
- move (parts of) documentation to separate pages/wiki?
- explore using indexes (e.g. global secondary indexes) for partitioned kv store
- Custom ReadOnlyKeyValueStore for 'partitionedKeyValueStore' type optimised interactive queries
- Follow-up tasks on 'Versioned State Stores'
- Add interactive queries support once follow-up KIPs are delivered
- Benchmark
- Consider (in-memory) caching options to improve performance (ref #18)
- correctness / completeness (#14)