Interface GlobalKTable<K,V>
-
- Type Parameters:
K
- Type of primary keysV
- Type of value changes
public interface GlobalKTable<K,V>
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.GlobalKTable
can only be used as right-hand side input forstream
-table joins.In contrast to a
KTable
that is partitioned over allKafkaStreams
instances, aGlobalKTable
is fully replicated perKafkaStreams
instance. Every partition of the underlying topic is consumed by eachGlobalKTable
, such that the full set of data is available in everyKafkaStreams
instance. This provides the ability to perform joins withKStream
without having to repartition the input stream. All joins with theGlobalKTable
require that aKeyValueMapper
is provided that can map from theKeyValue
of the left hand sideKStream
to the key of the right hand sideGlobalKTable
.A
GlobalKTable
is created via aStreamsBuilder
. For example:
allbuilder.globalTable("topic-name", "queryable-store-name");
GlobalKTable
s are backed by aReadOnlyKeyValueStore
and are therefore queryable via the interactive queries API. For example:
Note that in contrast tofinal GlobalKTable globalOne = builder.globalTable("g1", "g1-store"); final GlobalKTable globalTwo = builder.globalTable("g2", "g2-store"); ... final KafkaStreams streams = ...; streams.start() ... ReadOnlyKeyValueStore view = streams.store("g1-store", QueryableStoreTypes.timestampedKeyValueStore()); view.get(key); // can be done on any key, as all keys are present
KTable
aGlobalKTable
's state holds a full copy of the underlying topic, thus all keys can be queried locally.Records from the source topic that have null keys are dropped.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description String
queryableStoreName()
Get the name of the local state store that can be used to query thisGlobalKTable
.
-
-
-
Method Detail
-
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.
-
-