Get Started Free
Tutorial

How to expire KTable rows created via `aggregate` based on TTL in Kafka Streams

How to expire KTable rows created via aggregate based on TTL in Kafka Streams

If you have a Kafka Streams application or ksqlDB application which creates a KTable in the middle of a topology by calling aggregate(), you can purge older data by making clever use of tombstones. Specifically, by maintaining a state store containing TTLs, you can write tombstones out to the topic underlying the KTable in order to expire data.

The tombstones can be generated by first using the Kafka Streams Processor API to maintain the latest stream processing time per key:

    KTable<String, AggregateObject> table = stream2
        .process(new TTLEmitter<String, String, String, ValueWrapper>(MAX_AGE,
        SCAN_FREQUENCY, STATE_STORE_NAME), STATE_STORE_NAME)
        .groupByKey(Grouped.with(Serdes.String(), vwSerde))

        .aggregate(AggregateObject::new, (key, value, aggregate) -> {
            if (value.deleted())
                return null; // signal to tombstone this key in KTable
            else
                return aggregate.add(value.value().toString());
            }, Materialized.<String, AggregateObject, KeyValueStore<Bytes, byte[]>>as("eventstore")
                    .withKeySerde(Serdes.String()).withValueSerde(aggObjectSerde));

The TTLEmitter class implements the ProcessorSupplier interface with the method to process each record looking like this. Note that because the processor will also receive tombstones, we use tombstones to clean out TTLs from the state store:

    @Override
    public void process(Record<Kin, Vin> record) {
        // this gets invoked for each new record we consume. If it's a tombstone, delete
        // it from our state store. Otherwise, store the record timestamp.
        if (record.value() == null) {
            System.out.println("CLEARING key=" + record.key());
            stateStore.delete((Kout) record.key());
        } else {
            System.out.println("UPDATING key=" + record.key());
            stateStore.put((Kout) record.key(), record.timestamp());
        }
        context.forward(new Record(record.key(), new ValueWrapper(record.value(), false), record.timestamp()));
    }

The final piece of the puzzle is to use ProcessorContext.schedule to periodically pass over the state store and emit tombstones for any records that are older than a specified cutoff. Note that we use a ValueWrapper object containing a deleted flag because null values are not passed to aggregate().

    context.schedule(scanFrequency, PunctuationType.STREAM_TIME, timestamp -> {
        final long cutoff = timestamp - maxAge.toMillis();

        // scan over all the keys in this partition's store
        // this can be optimized, but just keeping it simple.
        // this might take a while, so the Streams timeouts should take this into account
        try (final KeyValueIterator<Kout, Long> all = stateStore.all()) {
            while (all.hasNext()) {
                final KeyValue<Kout, Long> record = all.next();
                System.out.println("RECORD "+record.key+":"+record.value);
    
                if (record.value != null && record.value < cutoff) {
                    // if a record's last update was older than our cutoff, emit a tombstone.
                    ValueWrapper vw = new ValueWrapper(record.value, true);
                    context.forward(new Record(record.key, vw, 0, null));
                    stateStore.delete(record.key);
                }
            }
        }
    });

The following steps use Confluent Cloud. To run the tutorial locally with Docker, skip to the Docker instructions section at the bottom.

Prerequisites

  • A Confluent Cloud account
  • The Confluent CLI installed on your machine
  • Apache Kafka or Confluent Platform (both include the Kafka Streams application reset tool)
  • Clone the confluentinc/tutorials repository and navigate into its top-level directory:
    git clone git@github.com:confluentinc/tutorials.git
    cd tutorials

Create Confluent Cloud resources

Login to your Confluent Cloud account:

confluent login --prompt --save

Install a CLI plugin that will streamline the creation of resources in Confluent Cloud:

confluent plugin install confluent-quickstart

Run the plugin from the top-level directory of the tutorials repository to create the Confluent Cloud resources needed for this tutorial. Note that you may specify a different cloud provider (gcp or azure) or region. You can find supported regions in a given cloud provider by running confluent kafka region list --cloud <CLOUD>.

confluent quickstart \
  --environment-name kafka-streams-schedule-ktable-ttl-aggregate-env \
  --kafka-cluster-name kafka-streams-schedule-ktable-ttl-aggregate-cluster \
  --create-kafka-key \
  --kafka-java-properties-file ./schedule-ktable-ttl-aggregate/kstreams/src/main/resources/cloud.properties

The plugin should complete in under a minute.

Create topics

Create the input and output topics for the application:

confluent kafka topic create input-topic-for-stream
confluent kafka topic create input-topic-for-table
confluent kafka topic create output-topic

Start a console producer:

confluent kafka topic produce input-topic-for-table --parse-key --delimiter :

Enter a few strings for keys alice and bobby:

alice:table-value-1
bobby:table-value-2

Enter Ctrl+C to exit the console producer.

Similarly, start a console producer for the stream:

confluent kafka topic produce input-topic-for-stream --parse-key --delimiter :

Enter a few strings for keys alice and bobby:

alice:stream-value-1
bobby:stream-value-2

Enter Ctrl+C to exit the console producer.

Compile and run the application

Compile the application from the top-level tutorials repository directory:

./gradlew schedule-ktable-ttl-aggregate:kstreams:shadowJar

Navigate into the application's home directory:

cd schedule-ktable-ttl-aggregate/kstreams

Run the application, passing the Kafka client configuration file generated when you created Confluent Cloud resources:

java -cp ./build/libs/ktable-ttl-aggregate-standalone.jar \
    io.confluent.developer.KTableTTLAggregate \
    ./src/main/resources/cloud.properties

Now wait a minute and produce a few more strings for the key alice:

confluent kafka topic produce input-topic-for-stream --parse-key --delimiter :

Enter these key-value pairs:

alice:stream-value-7
alice:stream-value-8

Validate that you see both the stream and table values for alice in the output topic for the initial messages that you produced but not the ones that you produced after the KTable TTL had passed.

confluent kafka topic consume output-topic -b --print-key --delimiter :

You should see:

alice:stream-value-1 table-value-1
bobby:stream-value-2 table-value-2
alice:stream-value-3
alice:stream-value-4

Clean up

When you are finished, delete the kafka-streams-schedule-ktable-ttl-aggregate-env environment by first getting the environment ID of the form env-123456 corresponding to it:

confluent environment list

Delete the environment, including all resources created for this tutorial:

confluent environment delete <ENVIRONMENT ID>
Docker instructions

Prerequisites

  • Docker running via Docker Desktop or Docker Engine
  • Docker Compose. Ensure that the command docker compose version succeeds.
  • Clone the confluentinc/tutorials repository and navigate into its top-level directory:
    git clone git@github.com:confluentinc/tutorials.git
    cd tutorials

Start Kafka in Docker

Start Kafka with the following command run from the top-level tutorials repository directory:

docker compose -f ./docker/docker-compose-kafka.yml up -d

Create topics

Open a shell in the broker container:

docker exec -it broker /bin/bash

Create the input and output topics for the application:

kafka-topics --bootstrap-server localhost:9092 --create --topic input-topic-for-stream
kafka-topics --bootstrap-server localhost:9092 --create --topic input-topic-for-table
kafka-topics --bootstrap-server localhost:9092 --create --topic output-topic

Start a console producer:

kafka-console-producer --bootstrap-server localhost:9092 --topic input-topic-for-table \
    --property "parse.key=true" --property "key.separator=:"

Enter a few strings for keys alice and bobby:

alice:table-value-1
bobby:table-value-2

Enter Ctrl+C to exit the console producer.

Similarly, start a console producer for the stream:

kafka-console-producer --bootstrap-server localhost:9092 --topic input-topic-for-stream \
    --property "parse.key=true" --property "key.separator=:"

Enter a few strings for keys alice and bobby:

alice:stream-value-1
bobby:stream-value-2

Enter Ctrl+C to exit the console producer.

Compile and run the application

On your local machine, compile the app:

./gradlew schedule-ktable-ttl-aggregate:kstreams:shadowJar

Navigate into the application's home directory:

cd schedule-ktable-ttl-aggregate/kstreams

Run the application, passing the local.properties Kafka client configuration file that points to the broker's bootstrap servers endpoint at localhost:9092:

java -cp ./build/libs/ktable-ttl-aggregate-standalone.jar \
    io.confluent.developer.KTableTTLAggregate \
    ./src/main/resources/local.properties

Now wait a minute and produce a few more strings for the key alice:

kafka-console-producer --bootstrap-server localhost:9092 --topic input-topic-for-stream \
    --property "parse.key=true" --property "key.separator=:"

Enter these key-value pairs:

alice:stream-value-3
alice:stream-value-4

Validate that you see both the stream and table values for alice in the output topic for the initial messages that you produced but not the ones that you produced after the KTable TTL had passed.

kafka-console-consumer --bootstrap-server localhost:9092 --topic output-topic --from-beginning \
  --property "print.key=true" --property "key.separator=:"

You should see:

alice:stream-value-1 table-value-1
bobby:stream-value-2 table-value-2
alice:stream-value-3
alice:stream-value-4

Clean up

From your local machine, stop the broker container:

docker compose -f ./docker/docker-compose-kafka.yml down
Do you have questions or comments? Join us in the #confluent-developer community Slack channel to engage in discussions with the creators of this content.