If you have a Kafka Streams application or ksqlDB application which uses KTables from a topic in Kafka, 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:
table.toStream()
.process(new TTLEmitter<String, String, String, String>(MAX_AGE,
SCAN_FREQUENCY, STATE_STORE_NAME), STATE_STORE_NAME)
.to(INPUT_TOPIC_TABLE, Produced.with(Serdes.String(), Serdes.String()));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("CLEANING key=" + record.key());
stateStore.delete((Kout) record.key());
} else {
System.out.println("UPDATING key=" + record.key());
stateStore.put((Kout) record.key(), context.currentStreamTimeMs());
}
}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:
context.schedule(scanFrequency, PunctuationType.STREAM_TIME, timestamp -> {
final long cutoff = timestamp - maxAge.toMillis();
try (final KeyValueIterator<Kout, Long> all = stateStore.all()) {
while (all.hasNext()) {
final KeyValue<Kout, Long> record = all.next();
if (record.value != null && record.value < cutoff) {
// record's last update was older than the cutoff, so emit a tombstone.
context.forward(new Record(record.key, null, 0, null));
}
}
}
});The following steps use Confluent Cloud. To run the tutorial locally with Docker, skip to the Docker instructions section at the bottom.
git clone git@github.com:confluentinc/tutorials.git
cd tutorialsLogin to your Confluent Cloud account:
confluent login --prompt --saveInstall a CLI plugin that will streamline the creation of resources in Confluent Cloud:
confluent plugin install confluent-quickstartRun 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-env \
--kafka-cluster-name kafka-streams-schedule-ktable-ttl-cluster \
--create-kafka-key \
--kafka-java-properties-file ./schedule-ktable-ttl/kstreams/src/main/resources/cloud.propertiesThe plugin should complete in under a minute.
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-topicStart 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-2Enter 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-2Enter Ctrl+C to exit the console producer.
Compile the application from the top-level tutorials repository directory:
./gradlew schedule-ktable-ttl:kstreams:shadowJarNavigate into the application's home directory:
cd schedule-ktable-ttl/kstreamsRun the application, passing the Kafka client configuration file generated when you created Confluent Cloud resources:
java -cp ./build/libs/ktable-ttl-standalone.jar \
io.confluent.developer.KTableTTL \
./src/main/resources/cloud.propertiesNow 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-3
alice:stream-value-4Validate 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-4When you are finished, delete the kafka-streams-schedule-ktable-ttl-env environment by first getting the environment ID of the form env-123456 corresponding to it:
confluent environment listDelete the environment, including all resources created for this tutorial:
confluent environment delete <ENVIRONMENT ID>git clone git@github.com:confluentinc/tutorials.git
cd tutorialsStart Kafka with the following command run from the top-level tutorials repository directory:
docker compose -f ./docker/docker-compose-kafka.yml up -dOpen a shell in the broker container:
docker exec -it broker /bin/bashCreate 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-topicStart 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-2Enter 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-2Enter Ctrl+C to exit the console producer.
On your local machine, compile the app:
./gradlew schedule-ktable-ttl:kstreams:shadowJarNavigate into the application's home directory:
cd schedule-ktable-ttl/kstreamsRun 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-standalone.jar \
io.confluent.developer.KTableTTL \
./src/main/resources/local.propertiesNow 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-4Validate 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-4From your local machine, stop the broker container:
docker compose -f ./docker/docker-compose-kafka.yml down