Interface GlobalKTable<K,V>
- Type Parameters:
K
- the key type of this tableV
- the value type of this table
GlobalKTable
is an abstraction of a changelog stream from a primary-keyed table.
Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key.
Primary-keys in a table cannot be null
, and thus, null
-key key-value
pairs are not
supported, and corresponding records will be dropped.
KTables
follow Kafka "tombstone" semantics, and null
-value key-value
pairs are
interpreted and processed as deletes for the corresponding key.
A GlobalKTable
is defined from a single Kafka topic
that is
consumed message by message.
A GlobalKTable
can only be used as right-hand side input for a
stream-globalTable join
.
In contrast to a KTable
that is partitioned over all KafkaStreams
instances, a GlobalKTable
is fully replicated per KafkaStreams
instance.
Every partition of the underlying topic is consumed by each GlobalKTable
, such that the full set of data is
available in every KafkaStreams
instance.
This provides the ability to perform joins with KStream
without having to repartition the input stream.
Furthermore, GlobalKTable
are "bootstrapped" on startup, and are maintained by a separate thread.
Thus, updates to a GlobalKTable
are not "stream-time synchronized" what may lead to non-deterministic results.
Furthermore, all GlobalKTable
have an internal state store
which can be accessed from
"outside" using the Interactive Queries (IQ) API (see KafkaStreams#store(...)
and KafkaStreams#query(...) [new API; evolving]
for details).
For example:
builder.globalTable("topic-name", "queryable-store-name");
...
KafkaStreams streams = ...;
streams.start()
...
StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>>> storeQueryParams =
StoreQueryParameters.fromNameAndType("queryable-store-name", QueryableStoreTypes.timestampedKeyValueStore());
ReadOnlyKeyValueStore view = streams.store(storeQueryParams);
// query the value for a key
ValueAndTimestamp value = view.get(key);
Note that in contrast to KTable
a GlobalKTable
's state holds a full copy of the underlying topic,
thus all keys can be queried locally.-
Method Summary
Modifier and TypeMethodDescriptionGet the name of the local state store that can be used to query thisGlobalKTable
.
-
Method Details
-
queryableStoreName
String queryableStoreName()Get the name of the local state store that can be used to query thisGlobalKTable
.- Returns:
- the underlying state store name, or
null
if thisGlobalKTable
cannot be queried.
-