How to Handle Debezium Tombstone Events in Streaming SQL

How to Handle Debezium Tombstone Events in Streaming SQL

A Debezium tombstone event is a Kafka message with a non-null key and a null value. It signals to Kafka's log compaction that all previous messages for that key can be deleted. If your downstream system doesn't handle tombstones correctly, you'll see null-pointer errors or silently dropped deletes in your streaming pipeline.

What Is a Tombstone Event?

When a row is deleted from the source database, Debezium emits two messages in sequence:

  1. Delete event: A normal Debezium message with "op": "d", containing the before image of the deleted row.
  2. Tombstone event: A message with the same key but a null value.

The tombstone has a specific purpose in Kafka: it instructs the log compaction process that this key is effectively "dead" and all earlier messages for the key can be purged during compaction. Without tombstones, log-compacted topics would retain all historical versions of a row indefinitely.

Example sequence in Kafka for a deleted order (key: {"id": 42}):

Message 1 (delete event):
  Key:   {"id": 42}
  Value: {"before": {"id": 42, "status": "cancelled"}, "after": null, "op": "d", "ts_ms": 1710001000}

Message 2 (tombstone):
  Key:   {"id": 42}
  Value: null

Why Tombstones Cause Problems

Many Kafka consumers fail when they encounter a null value message. JSON deserializers throw exceptions. Stream processors that assume non-null values will crash or silently skip the message. In either case, you either break your pipeline or miss deletes—both are bad.

RisingWave's FORMAT DEBEZIUM ENCODE JSON handles the delete event (message 1) correctly, converting it into a retraction in the materialized view. The tombstone (message 2) requires separate configuration consideration.

Controlling Tombstone Behavior

The tombstones.on.delete connector configuration controls whether tombstones are emitted:

{
  "tombstones.on.delete": "true"
}

Setting this to false disables tombstone emission. Do this only if your Kafka topics are not log-compacted, or if all consumers can handle the null values. For most production setups, keep tombstones enabled.

Step-by-Step Tutorial

Step 1: Configure Debezium Delete Handling

Choose how deletes are represented in Kafka using the ExtractNewRecordState SMT:

{
  "name": "postgres-orders-connector",
  "config": {
    "connector.class": "io.debezium.connector.postgresql.PostgresConnector",
    "database.hostname": "postgres",
    "database.port": "5432",
    "database.user": "debezium",
    "database.password": "secret",
    "database.dbname": "shop",
    "database.server.name": "pgserver1",
    "table.include.list": "public.orders",
    "plugin.name": "pgoutput",
    "tombstones.on.delete": "true",
    "transforms": "unwrap",
    "transforms.unwrap.type": "io.debezium.transforms.ExtractNewRecordState",
    "transforms.unwrap.drop.tombstones": "true",
    "transforms.unwrap.delete.handling.mode": "rewrite",
    "transforms.unwrap.add.fields": "op"
  }
}

Key configuration choices:

  • drop.tombstones: true — the SMT drops the null-value tombstone message before it reaches Kafka consumers (including RisingWave). The delete event still passes through.
  • delete.handling.mode: rewrite — instead of dropping delete events (which is the default), the SMT rewrites them as flat records with __op: d. This is necessary for downstream consumers to know the row was deleted.
  • delete.handling.mode: drop — silently drops delete events. Only use if you genuinely don't care about deletes downstream.

Step 2: Connect RisingWave to Handle Debezium Deletes

Without SMT (raw Debezium envelope, tombstones present in topic):

-- For Debezium → Kafka → RisingWave pipeline:
-- RisingWave FORMAT DEBEZIUM handles the delete event (op: d) natively
-- Tombstone messages (null value) are skipped by the Kafka consumer
CREATE SOURCE orders_cdc (
    id          BIGINT,
    customer_id BIGINT,
    total       NUMERIC,
    status      VARCHAR,
    updated_at  TIMESTAMPTZ,
    _op         VARCHAR  -- debezium op field: c/u/d/r
) WITH (
    connector = 'kafka',
    topic = 'pgserver1.public.orders',
    properties.bootstrap.server = 'kafka:9092',
    scan.startup.mode = 'earliest'
) FORMAT DEBEZIUM ENCODE JSON;

RisingWave's Kafka consumer skips tombstone (null-value) messages automatically when using FORMAT DEBEZIUM. The op: d delete event causes the corresponding row to be retracted from all materialized views that reference orders_cdc.

With SMT + drop.tombstones: true and delete.handling.mode: rewrite, use FORMAT PLAIN instead:

-- With ExtractNewRecordState SMT applied upstream:
CREATE SOURCE orders_flat (
    id         BIGINT,
    customer_id BIGINT,
    total       NUMERIC,
    status      VARCHAR,
    updated_at  TIMESTAMPTZ,
    __op        VARCHAR
) WITH (
    connector = 'kafka',
    topic = 'pgserver1.public.orders',
    properties.bootstrap.server = 'kafka:9092',
    scan.startup.mode = 'earliest'
) FORMAT PLAIN ENCODE JSON;

Step 3: Track Deletes in a Materialized View

-- Active orders: deletes are automatically retracted
CREATE MATERIALIZED VIEW active_orders AS
SELECT id, customer_id, total, status
FROM orders_cdc
WHERE _op != 'd';

-- Delete audit log: capture delete events with timestamp
CREATE MATERIALIZED VIEW deleted_orders_log AS
SELECT
    id,
    customer_id,
    total,
    updated_at AS deleted_at
FROM orders_cdc
WHERE _op = 'd';

Step 4: Validate Delete Handling

Test that a delete in the source database produces the expected retraction:

-- Before delete: should return the row
SELECT * FROM active_orders WHERE id = 42;

-- After DELETE FROM orders WHERE id = 42 in PostgreSQL:
-- active_orders should no longer contain id=42
-- deleted_orders_log should contain id=42 with deleted_at timestamp
SELECT * FROM active_orders WHERE id = 42;  -- empty
SELECT * FROM deleted_orders_log WHERE id = 42;  -- one row

Comparison Table

ConfigurationTombstone in KafkaDelete Event in KafkaDownstream Behavior
Default (no SMT)Yes (null value)Yes (op: d)Some consumers crash on null
drop.tombstones: true SMTNoYes (op: d)All consumers see deletes safely
delete.handling.mode: drop SMTDependsNoSilent delete loss downstream
delete.handling.mode: rewrite SMTDependsYes (flat __op: d)Works with FORMAT PLAIN
tombstones.on.delete: falseNoYes (op: d)Safe for non-compacted topics

FAQ

Q: If I drop tombstones, will Kafka log compaction keep old delete event messages forever? Yes. Tombstones are the signal for log compaction to purge earlier messages for a key. Without them, the delete event itself will persist indefinitely in the compacted topic. This may be acceptable if your topics have a time-based retention policy, but for truly compacted topics (retention.bytes-based), you should keep tombstones enabled and let them propagate.

Q: Does RisingWave crash if it encounters a tombstone message? No. RisingWave's Kafka consumer with FORMAT DEBEZIUM ENCODE JSON skips messages with null values rather than crashing. The delete event that precedes the tombstone is what drives the retraction in materialized views.

Q: How do I handle soft deletes (a deleted_at column) differently from hard deletes? For soft deletes, the row is updated (not deleted), so there's no tombstone or op: d event. Create a materialized view that filters on deleted_at IS NULL to get the active set. Hard deletes (which produce tombstones) and soft deletes can coexist—handle them in separate views or combine them with a CASE expression.

Key Takeaways

  • Tombstone events are null-value Kafka messages that enable log compaction; they are separate from the delete event that carries the row's final state.
  • RisingWave's FORMAT DEBEZIUM ENCODE JSON handles both delete events and tombstones correctly—delete events trigger retractions in materialized views, tombstones are skipped.
  • Use ExtractNewRecordState with drop.tombstones: true if your downstream consumers can't handle null-value messages.
  • Use delete.handling.mode: rewrite (not drop) to ensure delete events are preserved and visible to downstream consumers.
  • Disabling tombstones (tombstones.on.delete: false) is only safe for topics without log compaction enabled.

Best-in-Class Event Streaming
for Agents, Apps, and Analytics
GitHubXLinkedInSlackYouTube
Sign up for our to stay updated.