Choose an Iceberg Mode

beta

In Iceberg-enabled clusters, the redpanda.iceberg.mode topic property determines how Redpanda maps topic data to the Iceberg table structure. You can have the generated Iceberg table match the structure of an Avro or Protobuf schema in the Schema Registry, or you can use the key_value mode where Redpanda stores the record values as-is in the table.

The JSON Schema format is not supported for Iceberg topics. If your topic data is in JSON, use the key_value mode.

Supported Iceberg modes

Redpanda supports the following modes for Iceberg topics:

key_value

Creates an Iceberg table using a simple schema, consisting of two columns, one for the record metadata including the key, and another binary column for the record’s value.

value_schema_id_prefix

Creates an Iceberg table whose structure matches the Redpanda schema for the topic, with columns corresponding to each field. You must register a schema in the Schema Registry and producers must write to the topic using the Schema Registry wire format.

In the Schema Registry wire format, a "magic byte" and schema ID are embedded in the message payload header. Producers to the topic must use the wire format in the serialization process so Redpanda can determine the schema used for each record, use the schema to define the Iceberg table, and store the topic values in the corresponding table columns.

value_schema_latest

Creates an Iceberg table whose structure matches the latest schema registered for the subject in the Schema Registry. You must register a schema in the Schema Registry. Unlike the value_schema_id_prefix mode, value_schema_latest does not require that producers use the wire format.

The latest schema is cached periodically. The cache period is defined by the cluster property iceberg_latest_schema_cache_ttl_ms (default: 5 minutes).

disabled

Default for redpanda.iceberg.mode. Disables writing to an Iceberg table for the topic.

Configure Iceberg mode for a topic

You can set the Iceberg mode for a topic when you create the topic, or you can update the mode for an existing topic.

Option 1. Create a new topic and set redpanda.iceberg.mode:
rpk topic create <topic-name> --topic-config=redpanda.iceberg.mode=<iceberg-mode>
Option 2. Set redpanda.iceberg.mode for an existing topic:
rpk topic alter-config <new-topic-name> --set redpanda.iceberg.mode=<iceberg-mode>

Override value_schema_latest default

In value_schema_latest mode, you only need to set the property value to the string value_schema_latest. This enables the default behavior of value_schema_latest mode, which determines the subject for the topic using the TopicNameStrategy. For example, if your topic is named sensor the schema is looked up in the sensor-value subject. For Protobuf data, the default behavior also deserializes records using the first message defined in the corresponding Protobuf schema stored in the Schema Registry.

If you use a different strategy other than the topic name to derive the subject name, you can override the default behavior of value_schema_latest mode and explicitly set the subject name.

To override the default behavior, use the following optional syntax:

value_schema_latest:subject=<subject-name>,protobuf_name=<protobuf-message-full-name>
  • For both Avro and Protobuf, specify a different subject name by using the key-value pair subject=<subject-name>, for example value_schema_latest:subject=sensor-data.

  • For Protobuf only:

    • Specify a different message definition by using a key-value pair protobuf_name=<message-name>, for example: value_schema_latest:protobuf_name=com.example.manufacturing.SensorData.

    • To specify both a different subject and message definition, separate the key-value pairs with a comma, for example: value_schema_latest:subject=my_protobuf_schema,protobuf_name=com.example.manufacturing.SensorData.

How Iceberg modes translate to table format

Redpanda generates an Iceberg table with the same name as the topic. In each mode, Redpanda writes to a redpanda table column that stores a single Iceberg struct per record, containing nested columns of the metadata from each record, including the record key, headers, timestamp, the partition it belongs to, and its offset.

For example, if you produce to a topic ClickEvent according to the following Avro schema:

{
    "type": "record",
    "name": "ClickEvent",
    "fields": [
        {
            "name": "user_id",
            "type": "int"
        },
        {
            "name": "event_type",
            "type": "string"
        },
        {
            "name": "ts",
            "type": "string"
        }
    ]
}

The key_value mode writes to the following table format:

CREATE TABLE ClickEvent (
    redpanda struct<
        partition: integer NOT NULL,
        timestamp: timestamp NOT NULL,
        offset:    long NOT NULL,
        headers:   array<struct<key: binary NOT NULL, value: binary>>,
        key:       binary
    >,
    value binary
)

Use key_value mode if the topic data is in JSON or if you are able to use the Iceberg data in its semi-structured format.

The value_schema_id_prefix and value_schema_latest modes can use the schema to translate to the following table format:

CREATE TABLE ClickEvent (
    redpanda struct<
        partition: integer NOT NULL,
        timestamp: timestamp NOT NULL,
        offset:    long NOT NULL,
        headers:   array<struct<key: binary NOT NULL, value: binary>>,
        key:       binary
    >,
    user_id integer NOT NULL,
    event_type string,
    ts string
)

As you produce records to the topic, the data also becomes available in object storage for Iceberg-compatible clients to consume. You can use the same analytical tools to read the Iceberg topic data in a data lake as you would for a relational database.

If Redpanda fails to translate the record to the columnar format as defined by the schema, it writes the record to a dead-letter queue (DLQ) table. See Manage dead-letter queue for more information.

Schema types translation

Redpanda supports direct translations of the following types to Iceberg value domains:

  • Avro

  • Protobuf

Avro type Iceberg type

boolean

boolean

int

int

long

long

float

float

double

double

bytes

binary

string

string

record

struct

array

list

maps

list

fixed

fixed

decimal

decimal

uuid

uuid

date

date

time

time

timestamp

timestamp

  • Different flavors of time (such as time-millis) and timestamp (such as timestamp-millis) types are translated to the same Iceberg time and timestamp types, respectively.

  • Avro unions are flattened to Iceberg structs with optional fields. For example:

    • The union ["int", "long", "float"] is represented as an Iceberg struct struct<0 INT NULLABLE, 1 LONG NULLABLE, 2 FLOAT NULLABLE>.

    • The union ["int", null, "float"] is represented as an Iceberg struct struct<0 INT NULLABLE, 1 FLOAT NULLABLE>.

  • All fields are required by default. (Avro always sets a default in binary representation.)

  • The Avro duration logical type is ignored.

  • The Avro null type is ignored and not represented in the Iceberg schema.

  • Recursive types are not supported.

Protobuf type Iceberg type

bool

boolean

double

double

float

float

int32

int

sint32

int

int64

long

sint64

long

sfixed32

int

sfixed64

int

string

string

bytes

binary

map

map

  • Repeated values are translated into Iceberg array types.

  • Enums are translated into Iceberg int types based on the integer value of the enumerated type.

  • uint32 and fixed32 are translated into Iceberg long types as that is the existing semantic for unsigned 32-bit values in Iceberg.

  • uint64 and fixed64 values are translated into their Base-10 string representation.

  • The timestamp type in Protobuf is translated into timestamp in Iceberg.

  • Messages are converted into Iceberg structs.

  • Recursive types are not supported.