Schema Registry Concepts for Confluent Platform¶
Schema Registry is a managed schema repository that supports data storage and exchange for both stream processing and data at rest, such as databases, files, and other static data storage. This section explains some fundamental concepts related to schemas, the registry, and utilities, and digs into the details of how it all works under the hood.
How it works¶
Confluent Schema Registry provides a serving layer for your metadata. It provides a RESTful interface for storing and retrieving your Avro®, JSON Schema, and Protobuf schemas. It stores a versioned history of all schemas based on a specified subject name strategy, provides multiple compatibility settings and allows evolution of schemas according to the configured compatibility settings and expanded support for these schema types. It provides serializers that plug into Apache Kafka® clients that handle schema storage and retrieval for Kafka messages that are sent in any of the supported formats.
Schema Registry lives outside of and separately from your Kafka brokers. Your producers and consumers still talk to Kafka to publish and read data (messages) to topics. Concurrently, they can also talk to Schema Registry to send and retrieve schemas that describe the data models for the messages.
Schema Registry is a distributed storage layer for schemas which uses Kafka as its underlying storage mechanism. Some key design decisions:
- Assigns globally unique ID [1] to each registered schema. Allocated IDs are guaranteed to be monotonically increasing and unique, but not necessarily consecutive.
- Kafka provides the durable backend, and functions as a write-ahead changelog for the state of Schema Registry and the schemas it contains.
- Schema Registry is designed to be distributed, with single-primary architecture, and ZooKeeper/Kafka coordinates primary election (based on the configuration).
[1] | Generally a Schema Registry cluster can include multiple tenants and each tenant can include multiple contexts. Schema IDs are unique within a (tenant, context) combination. For Confluent Platform Schema Registry you will only have one tenant, so IDs will be unique within a context. Considering the tenant/context combination as the realm in which schemas reside, schema IDs are unique within this “world”; that is, globally unique. |
Examples¶
To see working examples of Schema Registry, check out the following resources:
- Confluent Platform demo shows you how to deploy a Kafka streaming ETL, including Schema Registry, using ksqlDB for stream processing.
- On-Premises Schema Registry Tutorial shows how to enable client applications to read and write Avro data, check compatibility as schemas evolve, and use Confluent Control Center to work with schemas.
- Confluent Cloud Schema Registry Tutorial shows the basics of working with schemas on Confluent Cloud, including how to enable client applications to read and write Avro data, check compatibility as schemas evolve, and work with schemas in the Confluent Cloud Console.
Schemas, subjects, and topics¶
First, a quick review of terms and how they fit in the context of Schema Registry: what is a Kafka topic versus a schema versus a subject.
A Kafka Kafka topic contains messages, and each message is a key-value pair. Either the message key or the message value, or both, can be serialized as Avro, JSON, or Protobuf. A schema defines the structure of the data format. The Kafka topic name can be independent of the schema name. Schema Registry defines a scope in which schemas can evolve, and that scope is the subject. The name of the subject depends on the configured subject name strategy, which by default is set to derive subject name from topic name.
Modifying subject name strategy¶
You can modify the subject name strategy on a per-topic basis.
Following are some relevant features and guidelines to have a look at:
- The Schema Registry tutorials on Confluent Cloud and Confluent Platform show examples of schema definitions.
- The Confluent Platform documentation on how to Manage Schemas for Topics in Control Center includes a section on Best Practices and Pitfalls for Key Values with some good suggestions.
- Schema Linking is now available on both Confluent Platform: Schema Linking on Confluent Platform and Confluent Cloud: Schema Linking on Confluent Cloud. This is the preferred way to migrate schemas, with multiple options for dealing with subject naming.
- You can also use Confluent Replicator to migrate schemas from one Schema Registry to another, and automatically rename subjects on the target registry.
Subject aliases¶
On Confluent Cloud and on Confluent Platform 7.4.1 and later, you can set up a subject to be an alias
for another subject. Suppose you’ve registered a schema under the subject
my.package.Foo
. You now want to use schemas in that subject with the topic
mytopic
. With the default TopicNameStrategy, most clients will expect the
schema for the record values in mytopic
to be registered under the subject
mytopic-value
. Rather than having to re-register your schemas under a different
subject, or having to change the use of TopicNameStrategy, you can simple set up
an alias as follows:
curl -s -X PUT -u <Username>:<Password> \
-H "Content-Type: application/vnd.schemaregistry.v1+json" \
"<Schema Registry Hostname>:<Schema Registry Port>/config/mytopic-value" \
--data '{"alias": "my.package.foo"}'
Now when clients (using the default TopicNameStrategy) attempt to look up the
schemas from mytopic-value
, they will be given schemas from the subject
my.package.Foo
instead.
To learn more, see the alias
attribute on Config
API calls to Schema Registry in Confluent Platform Schema Registry API docs
and Confluent Cloud Schema Registry API docs.
Kafka serializers and deserializers background¶
When sending data over the network or storing it in a file, you need a way to encode the data into bytes. The area of data serialization has a long history, but has evolved quite a bit over the last few years. People started with programming language specific serialization such as Java serialization, which makes consuming the data in other languages inconvenient, then moved to language agnostic formats such as pure JSON, but without a strictly defined schema format.
Not having a strictly defined format has two significant drawbacks:
- Data consumers may not understand data producers: The lack of structure makes consuming data in these formats more challenging because fields can be arbitrarily added or removed, and data can even be corrupted. This drawback becomes more severe the more applications or teams across an organization begin consuming a data feed: if an upstream team can make arbitrary changes to the data format at their discretion, then it becomes very difficult to ensure that all downstream consumers will (continue to) be able to interpret the data. What’s missing is a “contract” (cf. schema below) for data between the producers and the consumers, similar to the contract of an API.
- Overhead and verbosity: They are verbose because field names and type information have to be explicitly represented in the serialized format, despite the fact that are identical across all messages.
A few cross-language serialization libraries have emerged that require the data structure to be formally defined by schemas. These libraries include Avro, Thrift, Protocol Buffers, and JSON Schema . The advantage of having a schema is that it clearly specifies the structure, the type and the meaning (through documentation) of the data. With a schema, data can also be encoded more efficiently. Avro was the default supported format for Confluent Platform.
For example, an Avro schema defines the data structure in a JSON format. The
following Avro schema specifies a user record with two fields: name
and
favorite_number
of type string
and int
, respectively.
{"namespace": "example.avro",
"type": "record",
"name": "user",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": "int"}
]
}
You can then use this Avro schema, for example, to serialize a Java object (POJO) into bytes, and deserialize these bytes back into the Java object.
Avro not only requires a schema during data serialization, but also during data deserialization. Because the schema is provided at decoding time, metadata such as the field names don’t have to be explicitly encoded in the data. This makes the binary encoding of Avro data very compact.
Avro, JSON, and Protobuf supported formats and extensibility¶
Avro was the original choice for the default supported schema format in Confluent Platform, with Kafka serializers and deserializers provided for the Avro format.
Schema Registry supports Protocol Buffers and JSON Schema along with Avro, the original default format. Support for these new serialization formats is not limited to Schema Registry, but provided throughout Confluent products. Additionally, Schema Registry is extensible to support adding custom schema formats as schema plugins.
New Kafka serializers and deserializers are available for Protobuf and JSON Schema, along with Avro. The serializers can automatically register schemas when serializing a Protobuf message or a JSON-serializable object. The Protobuf serializer can recursively register all imported schemas, .
The serializers and deserializers are available in multiple languages, including Java, .NET and Python.
Schema Registry supports multiple formats at the same time. For example, you can have Avro schemas in one subject and Protobuf schemas in another. Furthermore, both Protobuf and JSON Schema have their own compatibility rules, so you can have your Protobuf schemas evolve in a backward or forward compatible manner, just as with Avro.
Schema Registry also supports for schema references in Protobuf by modeling the import statement.
Ready to get started?
- Sign up for Confluent Cloud, the fully managed cloud-native service for Apache Kafka® and get started for free using the Cloud quick start.
- Download Confluent Platform, the self managed, enterprise-grade distribution of Apache Kafka and get started using the Confluent Platform quick start.
To learn more, see Formats, Serializers, and Deserializers.
Schema ID allocation¶
Schema ID allocation always happens in the primary node and Schema IDs are always monotonically increasing.
In Kafka primary election, the Schema ID is always based off the last ID that was
written to Kafka store. During a primary re-election, batch allocation happens only after the new
primary has caught up with all the records in the store <kafkastore.topic>
.
Schema Contexts¶
Schema contexts give you the ability to logically group schemas using any number of separate “sub-registries” within one Schema Registry. Contexts serve as the basis for Schema Linking, and are also useful in any scenario that requires more nuanced schema management.
To learn more, see the following topics:
Kafka backend¶
Kafka is used as Schema Registry storage backend.
The special Kafka topic <kafkastore.topic>
(default _schemas
), with a single partition, is used as a highly available write ahead log.
All schemas, subject/version and ID metadata, and compatibility settings are appended as messages to this log.
A Schema Registry instance therefore both produces and consumes messages under the _schemas
topic.
It produces messages to the log when, for example, new schemas are registered under a subject, or when updates to compatibility settings are registered.
Schema Registry consumes from the _schemas
log in a background thread, and updates its local caches on consumption of each new _schemas
message to reflect the newly added schema or compatibility setting.
Updating local state from the Kafka log in this manner ensures durability, ordering, and easy recoverability.
Tip
The Schema Registry topic is compacted and therefore the latest value of every key is retained forever, regardless of the Kafka retention policy. You can validate this with kafka-configs
:
kafka-configs --bootstrap-server localhost:9092 --entity-type topics --entity-name _schemas --describe
Your output should resemble:
Configs for topic '_schemas' are cleanup.policy=compact