This documentation is for an unreleased version of Apache Flink. We recommend you use the latest stable version.
Confluent Avro Format #
Format: Serialization Schema Format: Deserialization Schema
The Avro Schema Registry (avro-confluent
) format allows you to read records that were serialized by the io.confluent.kafka.serializers.KafkaAvroSerializer
and to write records that can in turn be read by the io.confluent.kafka.serializers.KafkaAvroDeserializer
.
When reading (deserializing) a record with this format the Avro writer schema is fetched from the configured Confluent Schema Registry based on the schema version id encoded in the record while the reader schema is inferred from table schema.
When writing (serializing) a record with this format the Avro schema is inferred from the table schema and used to retrieve a schema id to be encoded with the data. The lookup is performed with in the configured Confluent Schema Registry under the subject given in avro-confluent.subject
.
The Avro Schema Registry format can only be used in conjunction with the Apache Kafka SQL connector or the Upsert Kafka SQL Connector.
Dependencies #
In order to use the Avro Schema Registry format the following dependencies are required for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
Maven dependency | SQL Client |
---|---|
|
Only available for stable releases. |
For Maven, SBT, Gradle, or other build automation tools, please also ensure that Confluent’s maven repository at https://packages.confluent.io/maven/
is configured in your project’s build files.
How to create tables with Avro-Confluent format #
Example of a table using raw UTF-8 string as Kafka key and Avro records registered in the Schema Registry as Kafka values:
CREATE TABLE user_created (
-- one column mapped to the Kafka raw UTF-8 key
the_kafka_key STRING,
-- a few columns mapped to the Avro fields of the Kafka value
id STRING,
name STRING,
email STRING
) WITH (
'connector' = 'kafka',
'topic' = 'user_events_example1',
'properties.bootstrap.servers' = 'localhost:9092',
-- UTF-8 string as Kafka keys, using the 'the_kafka_key' table column
'key.format' = 'raw',
'key.fields' = 'the_kafka_key',
'value.format' = 'avro-confluent',
'value.avro-confluent.url' = 'http://localhost:8082',
'value.fields-include' = 'EXCEPT_KEY'
)
We can write data into the kafka table as follows:
INSERT INTO user_created
SELECT
-- replicating the user id into a column mapped to the kafka key
id as the_kafka_key,
-- all values
id, name, email
FROM some_table
Example of a table with both the Kafka key and value registered as Avro records in the Schema Registry:
CREATE TABLE user_created (
-- one column mapped to the 'id' Avro field of the Kafka key
kafka_key_id STRING,
-- a few columns mapped to the Avro fields of the Kafka value
id STRING,
name STRING,
email STRING
) WITH (
'connector' = 'kafka',
'topic' = 'user_events_example2',
'properties.bootstrap.servers' = 'localhost:9092',
-- Watch out: schema evolution in the context of a Kafka key is almost never backward nor
-- forward compatible due to hash partitioning.
'key.format' = 'avro-confluent',
'key.avro-confluent.url' = 'http://localhost:8082',
'key.fields' = 'kafka_key_id',
-- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id'
-- => adding a prefix to the table column associated to the Kafka key field avoids clashes
'key.fields-prefix' = 'kafka_key_',
'value.format' = 'avro-confluent',
'value.avro-confluent.url' = 'http://localhost:8082',
'value.fields-include' = 'EXCEPT_KEY',
-- subjects have a default value since Flink 1.13, though can be overridden:
'key.avro-confluent.subject' = 'user_events_example2-key2',
'value.avro-confluent.subject' = 'user_events_example2-value2'
)
Example of a table using the upsert-kafka connector with the Kafka value registered as an Avro record in the Schema Registry:
CREATE TABLE user_created (
-- one column mapped to the Kafka raw UTF-8 key
kafka_key_id STRING,
-- a few columns mapped to the Avro fields of the Kafka value
id STRING,
name STRING,
email STRING,
-- upsert-kafka connector requires a primary key to define the upsert behavior
PRIMARY KEY (kafka_key_id) NOT ENFORCED
) WITH (
'connector' = 'upsert-kafka',
'topic' = 'user_events_example3',
'properties.bootstrap.servers' = 'localhost:9092',
-- UTF-8 string as Kafka keys
-- We don't specify 'key.fields' in this case since it's dictated by the primary key of the table
'key.format' = 'raw',
-- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id'
-- => adding a prefix to the table column associated to the kafka key field avoids clashes
'key.fields-prefix' = 'kafka_key_',
'value.format' = 'avro-confluent',
'value.avro-confluent.url' = 'http://localhost:8082',
'value.fields-include' = 'EXCEPT_KEY'
)
Format Options #
Option | Required | Forwarded | Default | Type | Description |
---|---|---|---|---|---|
format |
required | no | (none) | String | Specify what format to use, here should be 'avro-confluent' . |
avro-confluent.basic-auth.credentials-source |
optional | yes | (none) | String | Basic auth credentials source for Schema Registry |
avro-confluent.basic-auth.user-info |
optional | yes | (none) | String | Basic auth user info for schema registry |
avro-confluent.bearer-auth.credentials-source |
optional | yes | (none) | String | Bearer auth credentials source for Schema Registry |
avro-confluent.bearer-auth.token |
optional | yes | (none) | String | Bearer auth token for Schema Registry |
avro-confluent.properties |
optional | yes | (none) | Map | Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. |
avro-confluent.ssl.keystore.location |
optional | yes | (none) | String | Location / File of SSL keystore |
avro-confluent.ssl.keystore.password |
optional | yes | (none) | String | Password for SSL keystore |
avro-confluent.ssl.truststore.location |
optional | yes | (none) | String | Location / File of SSL truststore |
avro-confluent.ssl.truststore.password |
optional | yes | (none) | String | Password for SSL truststore |
avro-confluent.schema |
optional | no | (none) | String | The schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the table schema. |
avro-confluent.subject |
optional | yes | (none) | String | The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. |
avro-confluent.url |
required | yes | (none) | String | The URL of the Confluent Schema Registry to fetch/register schemas. |
Data Type Mapping #
Currently, Apache Flink always uses the table schema to derive the Avro reader schema during deserialization and Avro writer schema during serialization. Explicitly defining an Avro schema is not supported yet. See the Apache Avro Format for the mapping between Avro and Flink DataTypes.
In addition to the types listed there, Flink supports reading/writing nullable types. Flink maps nullable types to Avro union(something, null)
, where something
is the Avro type converted from Flink type.
You can refer to Avro Specification for more information about Avro types.