From 7c74f3983b1cdbca59bb17e9070c8b4386c563f8 Mon Sep 17 00:00:00 2001
From: Victoria Xia
Note: Following store types are used regardless of the possibly specified type (via the parameter materialized):
materialized is versionedA Kafka Streams application typically runs on multiple instances. The state that is locally available on any given instance is only a subset of the application’s entire state. Querying the local stores on an instance will only return data locally available on that particular instance.
-The method KafkaStreams#store(...) finds an application instance’s local state stores by name and type.
The method KafkaStreams#store(...) finds an application instance’s local state stores by name and type.
+ Note that interactive queries are not supported for versioned state stores at this time.
Every application instance can directly query any of its local state stores.
diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html index 3ceaacd503d..7f5bc09cef5 100644 --- a/docs/streams/developer-guide/memory-mgmt.html +++ b/docs/streams/developer-guide/memory-mgmt.html @@ -150,9 +150,10 @@ props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); Serdes.String(), Serdes.Long()) .withCachingEnabled(); +Record caches are not supported for versioned state stores.
Each instance of RocksDB allocates off-heap memory for a block cache, index and filter blocks, and memtable (write buffer). Critical configs (for RocksDB version 4.1.0) include
block_cache_size, write_buffer_size and max_write_buffer_number. These can be specified through the
rocksdb.config.setter configuration.
// Creating a persistent key-value store:
// here, we create a `KeyValueStore<String, Long>` named "persistent-counts".
@@ -300,6 +306,7 @@ KeyValueStore<String, Long> countStore = countStoreSupplier.build();
// Creating an in-memory key-value store:
// here, we create a `KeyValueStore<String, Long>` named "inmemory-counts".
@@ -396,12 +403,76 @@ StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Store
Versioned key-value state stores are available since Kafka Streams 3.5. + Rather than storing a single record version (value and timestamp) per key, + versioned state stores may store multiple record versions per key. This + allows versioned state stores to support timestamped retrieval operations + to return the latest record (per key) as of a specified timestamp.
+You can create a persistent, versioned state store by passing a + VersionedBytesStoreSupplier + to the + versionedKeyValueStoreBuilder, + or by implementing your own + VersionedKeyValueStore.
+Each versioned store has an associated, fixed-duration history retention + parameter which specifies long old record versions should be kept for. + In particular, a versioned store guarantees to return accurate results for + timestamped retrieval operations where the timestamp being queried is within + history retention of the current observed stream time.
+History retention also doubles as its grace period, which determines + how far back in time out-of-order writes to the store will be accepted. A + versioned store will not accept writes (inserts, updates, or deletions) if + the timestamp associated with the write is older than the current observed + stream time by more than the grace period. Stream time in this context is + tracked per-partition, rather than per-key, which means it's important + that grace period (i.e., history retention) be set high enough to + accommodate a record with one key arriving out-of-order relative to a + record for another key.
+Because the memory footprint of versioned key-value stores is higher than + that of non-versioned key-value stores, you may want to adjust your + RocksDB memory settings + accordingly. Benchmarking your application with versioned stores is also + advised as performance is expected to be worse than when using non-versioned + stores.
+Versioned stores do not support caching or interactive queries at this time. + Also, window stores may not be versioned.
+ Upgrade note: Versioned state stores are opt-in only; no automatic + upgrades from non-versioned to versioned stores will take place. +Upgrades are supported from persistent, non-versioned key-value stores + to persistent, versioned key-value stores as long as the original store + has the same changelog topic format as the versioned store being upgraded + to. Both persistent + key-value stores + and timestamped key-value stores + share the same changelog topic format as + persistent versioned key-value stores, + and therefore both are eligible for upgrades.
+If you wish to upgrade an application using persistent, non-versioned + key-value stores to use persistent, versioned key-value stores + instead, you can perform the following procedure:
+min.compaction.lag.ms
+ to be at least your desired history retention. History retention plus
+ one day is recommended as buffer for the use of broker wall clock time
+ during compaction.You can use the built-in state store types or implement your own.
The primary interface to implement for the store is
org.apache.kafka.streams.processor.StateStore. Kafka Streams also has a few extended interfaces such
- as KeyValueStore.
KeyValueStore and VersionedKeyValueStore.
Note that your customized org.apache.kafka.streams.processor.StateStore implementation also needs to provide the logic on how to restore the state
via the org.apache.kafka.streams.processor.StateRestoreCallback or org.apache.kafka.streams.processor.BatchingStateRestoreCallback interface.
Details on how to instantiate these interfaces can be found in the javadocs.
StreamsConfig#TOPOLOGY_OPTIMIZATION can be found in KIP-295.
+ + A new state store type, versioned key-value stores, was introduced in + KIP-889. + Rather than storing a single record version (value and timestamp) per key, + versioned state stores may store multiple record versions per key. This + allows versioned state stores to support timestamped retrieval operations + to return the latest record (per key) as of a specified timestamp. + For more information, including how to upgrade from a non-versioned key-value + store to a versioned store in an existing application, see the + Developer Guide section. + Versioned key-value stores are opt-in only; existing applications will not be + affected upon upgrading to 3.5 without explicit code changes. +
+The semantics of left/outer stream-stream join got improved via diff --git a/docs/upgrade.html b/docs/upgrade.html index 65ddb4c000f..14c6a218b64 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -19,6 +19,16 @@