You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
confluent,avro,salesforce: replace linkedin/goavro and hamba/avro with twmb/avro
Remove all linkedin/goavro and hamba/avro imports from the confluent,
avro, and salesforce packages. All Avro encoding and decoding now uses
github.com/twmb/avro.
--- Schema registry encoder ---
Delete normalize_for_avro_schema.go (572 lines). The old encoder path
was: AsStructuredMut → normalizeForAvroSchema → goavro.BinaryFromNative.
The normalize shim walked the parsed JSON schema tree to coerce types
from json.Unmarshal output (float64 for ints, string for bytes, bare
values for unions, float64 for timestamps) into the types goavro
expected. The new path is: AsStructuredMut → schema.Encode. twmb/avro's
Encode handles all these input types natively.
The old code had separate codecs for avroRawJSON mode
(goavro.NewCodecForStandardJSONFull) vs Avro JSON mode
(goavro.NewCodec). twmb/avro's Encode accepts both bare values and
tagged union maps, so both modes use the same encoder path.
--- Schema registry decoder (default mode) ---
Old: goavro.NativeFromBinary → goavro.TextualFromNative → SetBytes.
New: schema.Decode → schema.EncodeJSON → SetBytes.
EncodeJSON calls now include avro.LinkedinFloats() to match goavro's
JSON encoding of NaN (null), +Inf/−Inf (1e999/−1e999), and −0 (−0).
Without LinkedinFloats, twmb/avro uses RFC-compliant string
representations ("NaN", "Infinity") which would change user output.
EncodeJSON produces schema-ordered fields (cosmetic difference from
goavro's hash-map-ordered output). Uses the same shorthand JSON escapes
as goavro (\b, \t, \n, \f, \r).
--- Schema registry decoder (preserve_logical_types mode) ---
Old: hamba/avro Reader.ReadNext → avroSchemaWalker.walk → SetStructuredMut.
New: schema.Decode(opts) → SetStructuredMut.
The old hamba walker manually traversed the decoded tree, converting
types and wrapping/unwrapping unions. The new code uses twmb/avro's
TaggedUnions and TagLogicalTypes decode options, plus CustomType
registrations in avro_walker.go for connect-specific conversions:
- time-millis/time-micros: int→time.Duration (built-in) → time.Time
(CustomType). The old hamba walker asserted time.Duration from
ReadNext, converted to time.Time. The new code registers CustomType
that converts the raw int32/int64 directly to time.Time.
- duration: The old hamba walker had a latent crash — ReadNext returned
[12]uint8 for fixed fields, but the walker asserted time.Duration.
The new code registers a CustomType that converts []byte (the raw
fixed bytes) to an ISO 8601 duration string via avro.DurationFromBytes.
- Kafka Connect (Debezium) types: Same behavior, now implemented as a
CustomType checking connect.name schema property instead of walking
PropertySchema. Supports the same types: Date, Year, Timestamp,
Time, MicroTimestamp, MicroTime, NanoTimestamp, NanoTime,
ZonedTimestamp.
--- Avro processor (to_json) ---
Old: goavro.NativeFromTextual/NativeFromBinary/NativeFromSingle →
SetStructuredMut.
New: schema.DecodeJSON/Decode/DecodeSingleObject → SetStructuredMut.
Decode options include TaggedUnions and TagLogicalTypes so unions are
wrapped in map[string]any and logical types produce enriched Go types
(time.Time, time.Duration, json.Number, avro.Duration).
Removed jsonNativeOpts() — 40 lines of CustomType registrations that
suppressed built-in logical type handlers to keep raw int32/int64/[]byte
values. These were added in the initial twmb/avro port to exactly match
goavro's NativeFromBinary output types, but are unnecessary: the
enriched types from TagLogicalTypes (time.Time for timestamps,
time.Duration for time-of-day, json.Number for decimals) are strictly
more useful than raw integers when set as structured data, and
json.Marshal handles all of them correctly.
--- Avro processor (from_json) ---
Textual encoding changed from AsBytes→DecodeJSON→EncodeJSON to
AsStructured→EncodeJSON. Both old goavro and new code use AsStructured
for this path. EncodeJSON now includes LinkedinFloats for NaN/Inf compat.
Binary and single encoding changed from AsStructured→goavro.BinaryFromNative
to AsStructured→schema.Encode/AppendSingleObject.
--- Avro scanner ---
Old: goavro.OCFReader → goavro.Codec.TextualFromNative → SetBytes.
New: ocf.Reader → schema.Decode → schema.EncodeJSON → SetBytes.
Scanner fingerprint field changed from []byte to uint64 to match the
goavro Rabin field type. EncodeJSON calls include LinkedinFloats.
Scanner Close now also calls reader.Close() which closes the OCF codec
(decompressor), in addition to closing the underlying io.ReadCloser.
The old code only closed the ReadCloser.
--- Salesforce ---
Old: goavro.Codec.NativeFromBinary → type-assert map[string]any.
New: schema.Decode(&result, TaggedUnions()) → map[string]any.
TaggedUnions added to match goavro's default behavior of wrapping union
values in map[string]any{"type": value}. Without this, twmb/avro returns
bare values for unions, which would break extractCDCFields in client.go.
--- File renames ---
serde_goavro.go → serde_avro.go
serde_hamba_avro.go → avro_walker.go
useHamba → preserveLogicalTypes
--- User-visible behavioral differences ---
1. Scanner @avro_schema metadata: For schemas with logical types, the
canonical form differs. goavro retained object wrappers like
{"type":"long"} after stripping logicalType; twmb/avro correctly
reduces to "long" per the Avro Parsing Canonical Form spec (§4.1.5).
The @avro_schema_fingerprint (Rabin) also differs for these schemas.
Schemas without logical types produce identical canonical forms and
fingerprints.
2. Avro processor to_json binary/single with duration logical type:
goavro returned []byte (raw 12 bytes), json.Marshal produces base64.
twmb/avro returns avro.Duration struct, json.Marshal produces
{"Months":N,"Days":N,"Milliseconds":N}. This only affects schemas
with the rare duration logical type.
3. Avro processor from_json textual rejects int32 overflow: goavro's
TextualFromNative silently emitted out-of-range values for int fields.
twmb/avro's EncodeJSON returns an error. This is spec-correct.
4. Avro processor from_json textual accepts RFC 3339 timestamp strings:
twmb/avro's EncodeJSON accepts "2025-03-19T12:00:00Z" for
timestamp-millis/micros fields. goavro required integer values only.
5. Decimal decode: goavro returned *big.Rat, twmb/avro returns
json.Number. In JSON output, goavro's TextualFromNative produced
escaped raw bytes (e.g. "\u0001:" for 3.14) — a goavro bug where
legacy decimal encoding routed two's-complement bytes through the
Avro bytes textual encoder. twmb/avro produces the proper number
(3.14). This affects both the avro processor to_json and the schema
registry decoder default mode.
6. Fixed/bytes in preserve_logical_types structured output: now base64
strings instead of integer arrays. Affects Bloblang expressions
accessing fixed field values as arrays.
7. EncodeJSON field ordering: twmb/avro produces schema-ordered fields.
goavro used Go map iteration order. Semantically identical JSON.
8. Encoder is more permissive: accepts json.Number for numeric fields,
tagged union maps in raw_json mode, RFC 3339 strings for timestamps.
Inputs that previously errored now work.
Subsumes PR #4124 (Fix avro timestamp encoding): twmb/avro's Encode
natively accepts RFC 3339 strings for timestamp fields.
Error messages differ in wording but all invalid inputs are still
rejected.
Copy file name to clipboardExpand all lines: docs/modules/components/pages/processors/schema_registry_decode.adoc
+1-1Lines changed: 1 addition & 1 deletion
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -134,7 +134,7 @@ For example, the union schema `["null","string","Foo"]`, where `Foo` is a record
134
134
- the string `"a"` as `{"string": "a"}`; and
135
135
- a `Foo` instance as `{"Foo": {...}}`, where `{...}` indicates the JSON encoding of a `Foo` instance.
136
136
137
-
However, it is possible to instead create documents in https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard/raw JSON format^] by setting the field <<avro_raw_json, `avro_raw_json`>> to `true`.
137
+
However, it is possible to instead create documents in standard/raw JSON format by setting the field <<avro_raw_json, `avro_raw_json`>> to `true`.
Copy file name to clipboardExpand all lines: docs/modules/components/pages/processors/schema_registry_encode.adoc
+1-5Lines changed: 1 addition & 5 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -116,14 +116,10 @@ For example, the union schema `["null","string","Foo"]`, where `Foo` is a record
116
116
- the string `"a"` as `\{"string": "a"}`; and
117
117
- a `Foo` instance as `\{"Foo": {...}}`, where `{...}` indicates the JSON encoding of a `Foo` instance.
118
118
119
-
However, it is possible to instead consume documents in https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard/raw JSON format^] by setting `avro.raw_json` to `true`. This is strongly recommended when using `schema_metadata` mode, as CDC sources emit standard JSON rather than Avro JSON.
119
+
However, it is possible to instead consume documents in standard/raw JSON format by setting `avro.raw_json` to `true`. This is strongly recommended when using `schema_metadata` mode, as CDC sources emit standard JSON rather than Avro JSON.
120
120
121
121
NOTE: The top-level `avro_raw_json` field is deprecated in favor of `avro.raw_json`.
122
122
123
-
=== Known issues
124
-
125
-
Important! There is an outstanding issue in the https://github.com/linkedin/goavro[avro serializing library^] that Redpanda Connect uses which means it https://github.com/linkedin/goavro/issues/252[doesn't encode logical types correctly^]. It's still possible to encode logical types that are in-line with the spec if `avro.raw_json` is set to true, though now of course non-logical types will not be in-line with the spec.
126
-
127
123
== Protobuf format
128
124
129
125
This processor encodes protobuf messages either from any format parsed within Redpanda Connect (encoded as JSON by default), or from raw JSON documents, you can read more about JSON mapping of protobuf messages here: https://developers.google.com/protocol-buffers/docs/proto3#json
Copy file name to clipboardExpand all lines: docs/modules/components/pages/scanners/avro.adoc
+2-2Lines changed: 2 additions & 2 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -63,7 +63,7 @@ For example, the union schema `["null","string","Foo"]`, where `Foo` is a record
63
63
- the string `"a"` as `{"string": "a"}`; and
64
64
- a `Foo` instance as `{"Foo": {...}}`, where `{...}` indicates the JSON encoding of a `Foo` instance.
65
65
66
-
However, it is possible to instead create documents in https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard/raw JSON format^] by setting the field <<avro_raw_json,`avro_raw_json`>> to `true`.
66
+
However, it is possible to instead create documents in standard/raw JSON format by setting the field <<avro_raw_json,`avro_raw_json`>> to `true`.
67
67
68
68
This scanner also emits the canonical Avro schema as `@avro_schema` metadata, along with the schema's fingerprint available via `@avro_schema_fingerprint`.
69
69
@@ -72,7 +72,7 @@ This scanner also emits the canonical Avro schema as `@avro_schema` metadata, al
72
72
73
73
=== `raw_json`
74
74
75
-
Whether messages should be decoded into normal JSON ("json that meets the expectations of regular internet json") rather than https://avro.apache.org/docs/current/specification/_print/#json-encoding[Avro JSON^]. If `true` the schema returned from the subject should be decoded as https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard json^] instead of as https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodec[avro json^]. There is a https://github.com/linkedin/goavro/blob/5ec5a5ee7ec82e16e6e2b438d610e1cab2588393/union.go#L224-L249[comment in goavro^], the https://github.com/linkedin/goavro[underlining library used for avro serialization^], that explains in more detail the difference between the standard json and avro json.
75
+
Whether messages should be decoded into normal JSON rather than https://avro.apache.org/docs/current/specification/_print/#json-encoding[Avro JSON^]. When true, union values are unwrapped (bare values instead of {"type": value} wrappers).
@@ -47,13 +49,13 @@ For example, the union schema ` + "`[\"null\",\"string\",\"Foo\"]`, where `Foo`"
47
49
- the string `+"`\"a\"` as `{\"string\": \"a\"}`"+`; and
48
50
- a `+"`Foo` instance as `{\"Foo\": {...}}`, where `{...}` indicates the JSON encoding of a `Foo`"+` instance.
49
51
50
-
However, it is possible to instead create documents in https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard/raw JSON format^] by setting the field `+"<<avro_raw_json,`avro_raw_json`>> to `true`"+`.
52
+
However, it is possible to instead create documents in standard/raw JSON format by setting the field `+"<<avro_raw_json,`avro_raw_json`>> to `true`"+`.
51
53
52
54
This scanner also emits the canonical Avro schema as `+"`@avro_schema`"+` metadata, along with the schema's fingerprint available via `+"`@avro_schema_fingerprint`"+`.
53
55
`).
54
56
Fields(
55
57
service.NewBoolField(sFieldRawJSON).
56
-
Description("Whether messages should be decoded into normal JSON (\"json that meets the expectations of regular internet json\") rather than https://avro.apache.org/docs/current/specification/_print/#json-encoding[Avro JSON^]. If `true` the schema returned from the subject should be decoded as https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodecForStandardJSONFull[standard json^] instead of as https://pkg.go.dev/github.com/linkedin/goavro/v2#NewCodec[avro json^]. There is a https://github.com/linkedin/goavro/blob/5ec5a5ee7ec82e16e6e2b438d610e1cab2588393/union.go#L224-L249[comment in goavro^], the https://github.com/linkedin/goavro[underlining library used for avro serialization^], that explains in more detail the difference between the standard json and avro json.").
58
+
Description("Whether messages should be decoded into normal JSON rather than https://avro.apache.org/docs/current/specification/_print/#json-encoding[Avro JSON^]. When true, union values are unwrapped (bare values instead of {\"type\": value} wrappers).").
57
59
Advanced().
58
60
Default(false),
59
61
)
@@ -79,24 +81,22 @@ type avroScannerCreator struct {
0 commit comments