pip/pip-464.md
In Pulsar, SchemaType.JSON is used for topics where producers and consumers exchange JSON-encoded messages with a defined schema. The schema definition is stored in SchemaInfo.schema (the schema_data field) and is used by the broker for validation and compatibility checking, and by consumers for deserialization.
There are two relevant schema definition formats:
Apache Avro schema format: {"type":"record","name":"MyRecord","fields":[{"name":"field1","type":"string"}]} — the standard format since Pulsar 2.1. This is what consumers (AvroBaseStructSchema, GenericJsonSchema, AutoConsumeSchema) require to function correctly.
Jackson JSON Schema Draft format: {"type":"object","properties":{"field1":{"type":"string"}}} — the legacy format from Pulsar 2.0, generated by Jackson's JsonSchemaGenerator. This was superseded in Pulsar 2.1 (commit 1893323bc2, PR #2071) when the project standardized on Avro format for all structured schemas.
To maintain backward compatibility with schemas created during the 2.0 era, fallback logic was added in several components:
StructSchemaDataValidator — the broker-side validator that checks whether a schema definition is structurally valid. It first attempts to parse the schema as Avro; if that fails, it falls back to Jackson JsonSchema parsing.JsonSchemaCompatibilityCheck — the broker-side compatibility checker. It has permissive handling for mixed-format scenarios (Avro↔Jackson, Jackson↔Jackson).ProducerImpl — the Java client's producer implementation. It detects the broker's protocol version and sends the old Jackson format to brokers below protocol version 13.The broker-side fallback logic is too lenient. When Avro parsing fails, the Jackson fallback accepts any valid JSON as a schema definition for SchemaType.JSON, not just the legacy Jackson format. This has caused real issues for non-Java clients (e.g., the Rust client) where users accidentally register a JSON Schema Draft 2020-12 definition:
StructSchemaDataValidator accepts it — Avro parse fails, Jackson fallback succeeds because it accepts any JSON.AutoConsumeSchema or GenericJsonSchema, it fails with SchemaParseException: Type not supported: object because AvroBaseStructSchema strictly requires Avro format — no fallback on the consumer side.The result is that the broker stores a schema that no Java consumer can read. The failure is deferred from producer registration time (where it should be caught) to consumer read time (where it is confusing and unrecoverable without schema deletion).
There is an asymmetry in the system today: the broker side is lenient (accepts any JSON), but the consumer side is strict (requires Avro). This PIP resolves the asymmetry by making the broker side equally strict.
SchemaType.JSON.StructSchemaDataValidator and JsonSchemaCompatibilityCheck to reject non-Avro schemas when the legacy flag is disabled.ProducerImpl client-side code that sends old Jackson format to brokers below protocol version 13.schema_data for SchemaType.JSON must be an Apache Avro schema definition.ProducerImpl client-side backward-compatibility code — this will be addressed in a future major release.SchemaType values (AVRO, PROTOBUF, etc.).A new broker configuration schemaJsonAllowLegacyJacksonFormat (default false) controls whether the old Jackson JSON Schema format is accepted for SchemaType.JSON schema definitions.
When disabled (default):
StructSchemaDataValidator requires valid Avro schema format. If Avro parsing fails, the schema is rejected immediately with the Avro SchemaParseException — no Jackson fallback.JsonSchemaCompatibilityCheck requires both the existing and new schema to be valid Avro format. Mixed-format compatibility is rejected.When enabled:
On the client side, the existing ProducerImpl code that sends old Jackson format to pre-v13 brokers is annotated as @Deprecated with a reference to this PIP.
StructSchemaDataValidatorFile: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/validator/StructSchemaDataValidator.java
Current behavior:
try {
parse Avro schema
} catch (SchemaParseException) {
try {
parse Jackson JsonSchema // accepts ANY valid JSON
} catch (...) {
throw invalid schema
}
}
Proposed behavior:
try {
parse Avro schema
} catch (SchemaParseException e) {
if (schemaJsonAllowLegacyJacksonFormat) {
try {
parse Jackson JsonSchema
} catch (...) {
throw invalid schema
}
} else {
throw invalid schema (propagate original Avro SchemaParseException)
}
}
When schemaJsonAllowLegacyJacksonFormat=false (default), the Avro SchemaParseException is propagated directly. This reuses the existing error message without modification.
JsonSchemaCompatibilityCheckFile: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/JsonSchemaCompatibilityCheck.java
Current behavior: The compatibility check has empty/permissive handling for mixed-format scenarios (Avro↔Jackson, Jackson↔Jackson).
Proposed behavior: When schemaJsonAllowLegacyJacksonFormat=false, all schema definitions passed to compatibility checking must be valid Avro format. If either the existing or new schema fails Avro parsing, the compatibility check returns incompatible. This is consistent since StructSchemaDataValidator will have already rejected non-Avro schemas at registration time, so this serves as a defense-in-depth check.
ProducerImpl Client-Side Code (Deprecation Only)File: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
The existing code that detects broker protocol version and falls back to sending the old Jackson format is annotated with @Deprecated and a code comment referencing this PIP. No behavioral change to the client in this PIP — removal is deferred to a future major release.
The schemaJsonAllowLegacyJacksonFormat config value must be accessible from both StructSchemaDataValidator and JsonSchemaCompatibilityCheck. This will be threaded through the existing SchemaRegistryService → validator/checker dependency chain, consistent with how other schema-related broker configs are propagated.
No changes to the public client API, admin API, or REST API.
No changes to the Pulsar binary protocol.
Add a new broker configuration parameter:
| Property | Type | Default | Description |
|---|---|---|---|
schemaJsonAllowLegacyJacksonFormat | boolean | false | Whether to allow legacy Jackson JsonSchema format for SchemaType.JSON schema definitions. When false, only valid Apache Avro schema format is accepted, consistent with what the consumer side requires. When true, the pre-2.1 backward-compatible behavior is preserved for deployments that still have topics with legacy-format schemas. |
@FieldContext(
category = CATEGORY_SCHEMA,
doc = "Whether to allow legacy Jackson JsonSchema format for SchemaType.JSON schema definitions. "
+ "When false (default), only valid Apache Avro schema format is accepted, consistent with "
+ "what the consumer side requires. When true, the pre-2.1 backward-compatible behavior is "
+ "preserved for deployments that still have topics with legacy-format schemas."
)
private boolean schemaJsonAllowLegacyJacksonFormat = false;
No CLI changes.
No new metrics.
No specific monitoring changes. When schemaJsonAllowLegacyJacksonFormat=false (default), producers attempting to register non-Avro schema definitions will receive an error response at registration time. Operators can monitor for increased schema registration failures after upgrading if they suspect legacy schemas may exist in their deployment.
No security implications. This change tightens input validation, which marginally improves the broker's input handling by rejecting malformed schema definitions earlier.
This is a breaking change in default behavior.
JSONSchema.of() has generated Avro format since Pulsar 2.1.schemaJsonAllowLegacyJacksonFormat=true in broker.conf to restore the previous behavior.The legacy Jackson format has been superseded since Pulsar 2.1, released in 2018. Any active topics with old-format schemas have likely been migrated or recreated over the past 7+ years. The Java client has not generated Jackson format schemas since 2.1.
Rolling back to a prior Pulsar version will restore the lenient fallback behavior. No data migration is needed — the configuration flag is purely a runtime behavioral switch.
No impact on geo-replication. Schema definitions are replicated as-is between clusters. If one cluster has schemaJsonAllowLegacyJacksonFormat=false and receives a replicated topic with a legacy-format schema, the schema was already stored — this PIP only affects new schema registrations, not existing stored schemas. However, operators should ensure consistent configuration across geo-replicated clusters to avoid asymmetric behavior where a schema is accepted on one cluster but rejected on another.
true (backward-compatible default)Rejected. The legacy Jackson format has been superseded since Pulsar 2.1 (2018). The Java client's JSONSchema.of() has generated Avro format for over 7 years. Defaulting to true perpetuates a silent failure mode where non-Java clients can register schemas that Java consumers cannot read. The primary value of this PIP is fixing the default behavior.
Considered adding logic to detect JSON Schema Draft format (e.g., presence of "$schema" or "type":"object" with "properties") and return a targeted error message. Rejected in favor of propagating the existing Avro SchemaParseException to minimize code change surface. The PIP documentation and Pulsar schema documentation updates will serve as the guide for non-Java client developers.
Considered defaulting to true in the first release and flipping to false in the next. Rejected because the legacy format is 7+ years old, the Java client has not generated it since 2.1, and any active topics with old-format schemas have likely been migrated or recreated. An immediate default flip is appropriate.
Considered adding WARN-level logging or a counter metric when schemaJsonAllowLegacyJacksonFormat=true and a legacy schema is encountered. Rejected to keep the opt-in path simple and silent — users who enable the flag are making a conscious choice.
Considered providing an admin CLI command to scan stored schemas and report which topics use the old format. Rejected as out of scope — this can be added later if demand materializes. Operators can identify legacy schemas by attempting to parse stored schema definitions with an Avro parser.
The documentation for SchemaType.JSON should be updated to clearly state that schema_data must be an Apache Avro schema definition, not a JSON Schema Draft definition. This is particularly important for non-Java client implementations (Rust, Go, Python, C++, Node.js, .NET) that construct schema definitions manually rather than using the Java client's JSONSchema.of() helper.