Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.integration;

import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
Expand All @@ -35,15 +36,19 @@
import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.kstream.SessionWindows;
import org.apache.kafka.streams.kstream.SlidingWindows;
import org.apache.kafka.streams.kstream.TimeWindowedDeserializer;
import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.kstream.WindowedSerdes;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.api.ContextualProcessor;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.AggregationWithHeaders;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.SessionStore;
import org.apache.kafka.streams.state.SessionStoreWithHeaders;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.TimestampedKeyValueStore;
import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders;
Expand Down Expand Up @@ -556,4 +561,245 @@ public void process(final Record<String, String> record) {
assertEquals(KeyValue.pair("key1", "value1"), outputTopic.readKeyValue());
}
}

@Test
public void processorShouldAccessKStreamAggregatedKTableStoreAsHeadersStoreViaSupplier() {
final StreamsBuilder builder = new StreamsBuilder();

final Materialized<String, String, KeyValueStore<Bytes, byte[]>> materialized =
Materialized.as(Stores.persistentTimestampedKeyValueStoreWithHeaders("table-store"));

builder
.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.aggregate(
() -> "",
(key, value, aggregate) -> value,
materialized.withKeySerde(Serdes.String()).withValueSerde(Serdes.String())
)
.toStream()
.process(() -> new ContextualProcessor<String, String, String, String>() {
@Override
public void process(final Record<String, String> record) {
final TimestampedKeyValueStoreWithHeaders<String, String> store = context().getStateStore("table-store");

try (final KeyValueIterator<String, ValueTimestampHeaders<String>> it = store.all()) {
while (it.hasNext()) {
final KeyValue<String, ValueTimestampHeaders<String>> row = it.next();
context().forward(new Record<>(row.key, row.value.value(), row.value.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(Serdes.String(), Serdes.String()));

try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<String, String> outputTopic = testDriver.createOutputTopic("output-topic", new StringDeserializer(), new StringDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals(KeyValue.pair("key1", "value1"), outputTopic.readKeyValue());
}
}

@Test
public void processorShouldAccessKStreamReducedKTableStoreAsHeadersStoreViaSupplier() {
final StreamsBuilder builder = new StreamsBuilder();

final Materialized<String, String, KeyValueStore<Bytes, byte[]>> materialized =
Materialized.as(Stores.persistentTimestampedKeyValueStoreWithHeaders("table-store"));

builder
.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.reduce(
(value, aggregate) -> value,
materialized.withKeySerde(Serdes.String()).withValueSerde(Serdes.String())
)
.toStream()
.process(() -> new ContextualProcessor<String, String, String, String>() {
@Override
public void process(final Record<String, String> record) {
final TimestampedKeyValueStoreWithHeaders<String, String> store = context().getStateStore("table-store");

try (final KeyValueIterator<String, ValueTimestampHeaders<String>> it = store.all()) {
while (it.hasNext()) {
final KeyValue<String, ValueTimestampHeaders<String>> row = it.next();
context().forward(new Record<>(row.key, row.value.value(), row.value.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(Serdes.String(), Serdes.String()));

try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<String, String> outputTopic = testDriver.createOutputTopic("output-topic", new StringDeserializer(), new StringDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals(KeyValue.pair("key1", "value1"), outputTopic.readKeyValue());
}
}

@Test
public void processorShouldAccessKStreamCountKTableStoreAsHeadersStoreViaSupplier() {
final StreamsBuilder builder = new StreamsBuilder();

builder.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.count(Materialized.as(Stores.persistentTimestampedKeyValueStoreWithHeaders("table-store")))
.toStream()
.process(() -> new ContextualProcessor<String, Long, String, Long>() {
@Override
public void process(final Record<String, Long> record) {
final TimestampedKeyValueStoreWithHeaders<String, Long> store = context().getStateStore("table-store");

try (final KeyValueIterator<String, ValueTimestampHeaders<Long>> it = store.all()) {
while (it.hasNext()) {
final KeyValue<String, ValueTimestampHeaders<Long>> row = it.next();
context().forward(new Record<>(row.key, row.value.value(), row.value.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(Serdes.String(), Serdes.Long()));

try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<String, Long> outputTopic = testDriver.createOutputTopic("output-topic", new StringDeserializer(), new LongDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals(KeyValue.pair("key1", 1L), outputTopic.readKeyValue());
}
}

@Test
public void processorShouldBuildTopologyWithWindowStoreWithHeadersViaSupplier() {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test is failing

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great catch! -- Opened a PR to fix it: #22004

final StreamsBuilder builder = new StreamsBuilder();

final Materialized<String, String, WindowStore<Bytes, byte[]>> materialized =
Materialized.as(Stores.persistentTimestampedWindowStoreWithHeaders("table-store", Duration.ofHours(24L), Duration.ofHours(1L), false));

builder
.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofHours(1L)))
.aggregate(
() -> "",
(key, value, aggregate) -> value,
materialized.withKeySerde(Serdes.String()).withValueSerde(Serdes.String())
)
.toStream()
.process(() -> new ContextualProcessor<Windowed<String>, String, Windowed<String>, String>() {
@Override
public void process(final Record<Windowed<String>, String> record) {
final WindowStore<String, ValueTimestampHeaders<String>> store = context().getStateStore("table-store");

try (final KeyValueIterator<Windowed<String>, ValueTimestampHeaders<String>> it = store.all()) {
while (it.hasNext()) {
final KeyValue<Windowed<String>, ValueTimestampHeaders<String>> row = it.next();
context().forward(new Record<>(row.key, row.value.value(), row.value.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(WindowedSerdes.timeWindowedSerdeFrom(String.class, Duration.ofHours(1L).toMillis()), Serdes.String()));

// Verify topology can be built and run with window headers store supplier
try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<Windowed<String>, String> outputTopic = testDriver.createOutputTopic("output-topic", new TimeWindowedDeserializer<>(new StringDeserializer(), Duration.ofHours(1L).toMillis()), new StringDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals("value1", outputTopic.readKeyValue().value);
}
}

@Test
public void processorShouldAccessKStreamSessionAggregatedKTableStoreAsHeadersStoreViaSupplier() {
final StreamsBuilder builder = new StreamsBuilder();

final Materialized<String, String, SessionStore<Bytes, byte[]>> materialized =
Materialized.as(Stores.persistentSessionStoreWithHeaders("table-store", Duration.ofHours(1L)));

builder
.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.windowedBy(SessionWindows.ofInactivityGapWithNoGrace(Duration.ofHours(1L)))
.aggregate(
() -> "",
(key, value, aggregate) -> value,
(key, left, right) -> left,
materialized.withKeySerde(Serdes.String()).withValueSerde(Serdes.String())
)
.toStream((windowedKey, value) -> windowedKey.key())
.process(() -> new ContextualProcessor<String, String, String, String>() {
@Override
public void process(final Record<String, String> record) {
final SessionStoreWithHeaders<String, String> store = context().getStateStore("table-store");

try (final KeyValueIterator<Windowed<String>, AggregationWithHeaders<String>> it = store.findSessions("key1", 0L, Long.MAX_VALUE)) {
while (it.hasNext()) {
final KeyValue<Windowed<String>, AggregationWithHeaders<String>> row = it.next();
context().forward(new Record<>(row.key.key(), row.value.aggregation(), record.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(Serdes.String(), Serdes.String()));

try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<String, String> outputTopic = testDriver.createOutputTopic("output-topic", new StringDeserializer(), new StringDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals(KeyValue.pair("key1", "value1"), outputTopic.readKeyValue());
}
}

@Test
public void processorShouldAccessKStreamSessionReducedKTableStoreAsHeadersStoreViaSupplier() {
final StreamsBuilder builder = new StreamsBuilder();

final Materialized<String, String, SessionStore<Bytes, byte[]>> materialized =
Materialized.as(Stores.persistentSessionStoreWithHeaders("table-store", Duration.ofHours(1L)));

builder
.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey()
.windowedBy(SessionWindows.ofInactivityGapWithNoGrace(Duration.ofHours(1L)))
.reduce(
(value, aggregate) -> value,
materialized.withKeySerde(Serdes.String()).withValueSerde(Serdes.String())
)
.toStream((windowedKey, value) -> windowedKey.key())
.process(() -> new ContextualProcessor<String, String, String, String>() {
@Override
public void process(final Record<String, String> record) {
final SessionStoreWithHeaders<String, String> store = context().getStateStore("table-store");

try (final KeyValueIterator<Windowed<String>, AggregationWithHeaders<String>> it = store.findSessions("key1", 0L, Long.MAX_VALUE)) {
while (it.hasNext()) {
final KeyValue<Windowed<String>, AggregationWithHeaders<String>> row = it.next();
context().forward(new Record<>(row.key.key(), row.value.aggregation(), record.timestamp()));
}
}
}
}, "table-store")
.to("output-topic", Produced.with(Serdes.String(), Serdes.String()));

try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build())) {
final TestInputTopic<String, String> inputTopic = testDriver.createInputTopic("input-topic", new StringSerializer(), new StringSerializer());
final TestOutputTopic<String, String> outputTopic = testDriver.createOutputTopic("output-topic", new StringDeserializer(), new StringDeserializer());

inputTopic.pipeInput("key1", "value1");

assertEquals(KeyValue.pair("key1", "value1"), outputTopic.readKeyValue());
}
}
}