Schema Evolution and Compatibility

Looking for Schema Management Confluent Cloud docs? You are currently viewing Confluent Platform documentation. If you are looking for Confluent Cloud docs, check out Schema Management on Confluent Cloud.

Schema Evolution

An important aspect of data management is schema evolution. After the initial schema is defined, applications may need to evolve it over time. When this happens, it’s critical for the downstream consumers to be able to handle data encoded with both the old and the new schema seamlessly. This is an area that tends to be overlooked in practice until you run into your first production issues. Without thinking through data management and schema evolution carefully, people often pay a much higher cost later on.

When using Avro or other schema formats, one of the most important things is to manage the schemas and consider how these schemas should evolve. Confluent Schema Registry is built for exactly that purpose. Schema compatibility checking is implemented in Schema Registry by versioning every single schema. The compatibility type determines how Schema Registry compares the new schema with previous versions of a schema, for a given subject. When a schema is first created for a subject, it gets a unique id and it gets a version number, i.e., version 1. When the schema is updated (if it passes compatibility checks), it gets a new unique id and it gets an incremented version number, i.e., version 2.

Compatibility Types

Summary

The following table presents a summary of the types of schema changes allowed for the different compatibility types, for a given subject. The Confluent Schema Registry default compatibility type is BACKWARD. All the compatibility types are described in more detail in the sections below. To learn more, see configuration options on connectors to Schema Registry that provide further control over compatibility requirements.

Compatibility Type Changes allowed Check against which schemas Upgrade first
BACKWARD
  • Delete fields
  • Add optional fields
Last version Consumers
BACKWARD_TRANSITIVE
  • Delete fields
  • Add optional fields
All previous versions Consumers
FORWARD
  • Add fields
  • Delete optional fields
Last version Producers
FORWARD_TRANSITIVE
  • Add fields
  • Delete optional fields
All previous versions Producers
FULL
  • Add optional fields
  • Delete optional fields
Last version Any order
FULL_TRANSITIVE
  • Add optional fields
  • Delete optional fields
All previous versions Any order
NONE
  • All changes are accepted
Compatibility checking disabled Depends

Note

A REST API call to compatibility mode is global meaning it overrides any compatibility parameters set in schema registry properties files. This is discussed in Using Compatibility Types below and shown in the API usage example Update compatibility requirements globally.

Backward Compatibility

BACKWARD compatibility means that consumers using the new schema can read data produced with the last schema. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then BACKWARD compatibility ensures that consumers using the new schema X can process data written by producers using schema X or X-1, but not necessarily X-2. If the consumer using the new schema needs to be able to process data written by all registered schemas, not just the last two schemas, then use BACKWARD_TRANSITIVE instead of BACKWARD. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then BACKWARD_TRANSITIVE compatibility ensures that consumers using the new schema X can process data written by producers using schema X, X-1, or X-2.

  • BACKWARD: consumer using schema X can process data produced with schema X or X-1
  • BACKWARD_TRANSITIVE: consumer using schema X can process data produced with schema X, X-1, or X-2

Important

The Confluent Schema Registry default compatibility type is BACKWARD, not BACKWARD_TRANSITIVE.

The main reason that BACKWARD compatibility mode is the default, and preferred for Kafka, is so that you can rewind consumers to the beginning of the topic. With FORWARD compatibility mode, you aren’t guaranteed the ability to read old messages.

Also FORWARD compatibility mode is harder to work with. In a sense, you need to anticipate all future changes. For example, in FORWARD compatibility mode with Protobuf, you cannot add new message types to a schema.

An example of a backward compatible change is a removal of a field. A consumer that was developed to process events without this field will be able to process events written with the old schema and contain the field – the consumer will just ignore that field.

Consider the case where all of the data in Kafka is also loaded into HDFS, and you want to run SQL queries (for example, using Apache Hive) over all the data. Here, it is important that the same SQL queries continue to work even as the data is undergoing changes over time. To support this kind of use case, you can evolve the schemas in a backward compatible way. All supported schema formats have rules as to what changes are allowed in the new schema for it to be backward compatible. For example, here are the Avro rules for compatibility If all schemas are evolved in a backward compatible way, we can always use the latest schema to query all the data uniformly.

For example, an application can evolve the user schema from the previous section to the following by adding a new field favorite_color:

{"namespace": "example.avro",
 "type": "record",
 "name": "user",
 "fields": [
     {"name": "name", "type": "string"},
     {"name": "favorite_number",  "type": "int"},
     {"name": "favorite_color", "type": "string", "default": "green"}
 ]
}

Note that the new field favorite_color has the default value “green”. This allows data encoded with the old schema to be read with the new one. The default value specified in the new schema will be used for the missing field when deserializing the data encoded with the old schema. Had the default value been omitted in the new field, the new schema would not be backward compatible with the old one since it’s not clear what value should be assigned to the new field, which is missing in the old data.

Note

Avro implementation details: Take a look at ResolvingDecoder in the Apache Avro project to understand how, for data that was encoded with an older schema, Avro decodes that data with a newer, backward-compatible schema.

Important

For Kafka Streams, only BACKWARD compatibility is supported. To learn more, see the note about Kafka Streams under Order of Upgrading Clients.

Forward Compatibility

FORWARD compatibility means that data produced with a new schema can be read by consumers using the last schema, even though they may not be able to use the full capabilities of the new schema. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then FORWARD compatibility ensures that data written by producers using the new schema X can be processed by consumers using schema X or X-1, but not necessarily X-2. If data produced with a new schema needs to be read by consumers using all registered schemas, not just the last two schemas, then use FORWARD_TRANSITIVE instead of FORWARD. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then FORWARD_TRANSITIVE compatibility ensures that data written by producers using the new schema X can be processed by consumers using schema X, X-1, or X-2.

  • FORWARD: data produced using schema X can be read by consumers with schema X or X-1
  • FORWARD_TRANSITIVE: data produced using schema X can be read by consumers with schema X, X-1, or X-2

An example of a forward compatible schema modification is adding a new field. In most data formats, consumers that were written to process events without the new field will be able to continue doing so even when they receive new events that contain the new field.

Consider a use case where a consumer has application logic tied to a particular version of the schema. When the schema evolves, the application logic may not be updated immediately. Therefore, you need to be able to project data with newer schemas onto the (older) schema that the application understands. To support this use case, you can evolve the schemas in a forward compatible way: data encoded with the new schema can be read with the old schema. For example, the new user schema shown in the previous section on backward compatibility is also forward compatible with the old one. When projecting data written with the new schema to the old one, the new field is simply dropped. Had the new schema dropped the original field favorite_number (number, not color), it would not be forward compatible with the original user schema since consumers wouldn’t know how to fill in the value for favorite_number for the new data because the original schema did not specify a default value for that field.

Full Compatibility

FULL compatibility means schemas are both backward and forward compatible. Schemas evolve in a fully compatible way: old data can be read with the new schema, and new data can also be read with the last schema. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then FULL compatibility ensures that consumers using the new schema X can process data written by producers using schema X or X-1, but not necessarily X-2, and that data written by producers using the new schema X can be processed by consumers using schema X or X-1, but not necessarily X-2. If the new schema needs to be forward and backward compatible with all registered schemas, not just the last two schemas, then use FULL_TRANSITIVE instead of FULL. For example, if there are three schemas for a subject that change in order X-2, X-1, and X then FULL_TRANSITIVE compatibility ensures that consumers using the new schema X can process data written by producers using schema X, X-1, or X-2, and that data written by producers using the new schema X can be processed by consumers using schema X, X-1, or X-2.

  • FULL: backward and forward compatibile between schemas X and X-1
  • FULL_TRANSITIVE: backward and forward compatibile between schemas X, X-1, and X-2

In Avro and Protobuf, you can define fields with default values. In that case, adding or removing a field with a default value is a fully compatible change.

Compatibility rules for supported schema types are described in Compatibility Checks in Formats, Serializers, and Deserializers.

JSON Schema does not explicitly define compatibility rules, so this blog post further explains how JSON Schema compatibility works, including full compatibility.

No Compatibility Checking

NONE compatibility type means schema compatibility checks are disabled.

Sometimes we make incompatible changes. For example, modifying a field type from Number to String. In this case, you will either need to upgrade all producers and consumers to the new schema version at the same time, or more likely – create a brand-new topic and start migrating applications to use the new topic and new schema, avoiding the need to handle two incompatible versions in the same topic.

Transitive Property

Transitive compatibility checking is important once you have more than two versions of a schema for a given subject. If compatibility is configured as transitive, then it checks compatibility of a new schema against all previously registered schemas; otherwise, it checks compatibility of a new schema only against the latest schema.

For example, if there are three schemas for a subject that change in order X-2, X-1, and X then:

  • transitive: ensures compatibility between X-2 <==> X-1 and X-1 <==> X and X-2 <==> X
  • non-transitive: ensures compatibility between X-2 <==> X-1 and X-1 <==> X, but not necessarily X-2 <==> X

Refer to an example of schema changes which are incrementally compatible, but not transitively so.

The Confluent Schema Registry default compatibility type BACKWARD is non-transitive, which means that it’s not BACKWARD_TRANSITIVE. As a result, new schemas are checked for compatibility only against the latest schema.

Order of Upgrading Clients

The configured compatibility type has an implication on the order for upgrading client applications, i.e., the producers using schemas to write events to Kafka and the consumers using schemas to read events from Kafka. Depending on the compatibility type:

  • BACKWARD or BACKWARD_TRANSITIVE: there is no assurance that consumers using older schemas can read data produced using the new schema. Therefore, upgrade all consumers before you start producing new events.
  • FORWARD or FORWARD_TRANSITIVE: there is no assurance that consumers using the new schema can read data produced using older schemas. Therefore, first upgrade all producers to using the new schema and make sure the data already produced using the older schemas are not available to consumers, then upgrade the consumers.
  • FULL or FULL_TRANSITIVE: there are assurances that consumers using older schemas can read data produced using the new schema and that consumers using the new schema can read data produced using older schemas. Therefore, you can upgrade the producers and consumers independently.
  • NONE: compatibility checks are disabled. Therefore, you need to be cautious about when to upgrade clients.

Important

For Kafka Streams only FULL, TRANSITIVE, and BACKWARD compatibility is supported.

For a plain consumer, it is safe to upgrade the consumer to the new schema after the producer is upgraded because a plain consumer reads only from the input topic. For Kafka Streams, the scenario is different. When you upgrade Kafka Streams, it also can read from the input topic (that now contains data with the new schema). However, in contrast to a plain consumer, Kafka Streams must also be able to read the old schema (from the state/changelog); therefore, only BACKWARD compatibility is supported. The Kafka Streams apps must be upgraded first, then it safe to upgrade the upstream producer that writes into the input topic.

FULL and TRANSITIVE compatibility are always supported for Kafka Streams, as they include backward compatibility and so are, in effect, “stronger” settings than BACKWARD.

Specify Schema Compatibility Requirements per Subject

You can configure and update chema compatibility requirements globally or on a per-subject basis. To learn more, see Compatibility in the Schema Registry API Reference and the following topics in the Schema Registry API Usage Examples:

Examples

Each of the sections above has an example of the compatibility type. An additional reference for Avro is Avro compatibility test suite, which presents multiple test cases with two schemas and the respective result of the compatibility test between them.

Using Compatibility Types

Compatibility rules and references for all supported schema types are described in Compatibility Checks in Formats, Serializers, and Deserializers.

You can find out the details on how to use Schema Registry to store Avro schemas and enforce certain compatibility rules during schema evolution by looking at the Schema Registry API Reference. Here are some tips to get you started.

To check the currently configured compatibility type, view the configured setting:

  1. Using the Schema Registry REST API.

To set the compatibility level, you can configure it in the following ways:

  1. In your client application.
  2. Using the Schema Registry REST API.
  3. Using the Control Center Edit Schema feature. See Manage Schemas for Topics in Control Center.

Note

A REST API call to compatibility mode is global meaning it overrides any compatibility parameters set in schema registry properties files, as shown in the API usage example Update compatibility requirements globally.

To validate the compatibility of a given schema, you may test it one of two ways:

  1. Using the Schema Registry Maven Plugin.
  2. Using the Schema Registry REST API.

Refer to the Confluent Schema Registry Tutorial which has an example of checking schema compatibility.