Hi Jon,

Thank you for this email!

On Mon, Nov 13, 2023 at 7:56 PM Jonathan Slusher <jonslus...@gmail.com>
wrote:

> Hi,
>
> I opened an issue in the AVRO project in Jira <
> https://issues.apache.org/jira/browse/AVRO-3900> and I’ve been asked to
> submit a topic for discussion to this email group.
>
> See this issue in the rust schema_registry_converter repo <
> https://github.com/gklijs/schema_registry_converter/issues/100> for
> details specific to the crate in rust that we’re having trouble with:
>
> A couple of things to point out here:
>
> 1. I understand that at this time the Avro spec does not allow hyphens in
> its namespaces, but somehow our registry is allowing them to be created
> from our Debezium connectors. We have been using the confluent_python <
> https://github.com/confluentinc/confluent-kafka-python> module and since
> version 1.9.2, its deserializer seems to handle these hyphens without
> error. We also have several JDBC sink connectors with consumer groups that
> are able to use these topics.
>
> 2. We recently attempted to implement a consumer written in rust and the
> crate <https://github.com/gklijs/schema_registry_converter> above, which
> is used for deserialization, throws an exception when attempting to connect
> to these topics.
>
> ```
> thread 'main' panicked at /app/src/utils/kafka.rs:67:35:
> Error decoding value: Error: Supplied raw value
> "{\"type\":\"record\",\"name\":\"Envelope\",\"namespace\":\"debezium.abc-123-efg-20231005.table.u_table_dbz\",\"fields\":[{\"name\":\"before\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Value\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"uid\",\"type\":{\"type\":\"long\",\"connect.default\":0},\"default\":0},{\"name\":\"release_id\",\"type\":{\"type\":\"long\",\"connect.default\":0},\"default\":0},{\"name\":\"notes\",\"type\":\"string\"},{\"name\":\"notes_public\",\"type\":{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"Y,N\"},\"connect.default\":\"N\",\"
> connect.name
> \":\"io.debezium.data.Enum\"},\"default\":\"N\"},{\"name\":\"added_ts\",\"type\":{\"type\":\"long\",\"connect.version\":1,\"
> connect.name\":\"io.debezium.time.Timestamp\"}}],\"connect.name
> \":\"debezium.abc-123-efg-20231005.table.u_table_dbz.Value\"}],\"default\":null},{\"name\":\"after\",\"type\":[\"null\",\"Value\"],\"default\":null},{\"name\":\"source\",\"type\":{\"type\":\"record\",\"name\":\"Source\",\"namespace\":\"io.debezium.connector.mysql\",\"fields\":[{\"name\":\"version\",\"type\":\"string\"},{\"name\":\"connector\",\"type\":\"string\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":\"long\"},{\"name\":\"snapshot\",\"type\":[{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"true,last,false,incremental\"},\"connect.default\":\"false\",\"
> connect.name
> \":\"io.debezium.data.Enum\"},\"null\"],\"default\":\"false\"},{\"name\":\"db\",\"type\":\"string\"},{\"name\":\"sequence\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"table\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"server_id\",\"type\":\"long\"},{\"name\":\"gtid\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file\",\"type\":\"string\"},{\"name\":\"pos\",\"type\":\"long\"},{\"name\":\"row\",\"type\":\"int\"},{\"name\":\"thread\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"query\",\"type\":[\"null\",\"string\"],\"default\":null}],\"
> connect.name
> \":\"io.debezium.connector.mysql.Source\"}},{\"name\":\"op\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"transaction\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"block\",\"namespace\":\"event\",\"fields\":[{\"name\":\"id\",\"type\":\"string\"},{\"name\":\"total_order\",\"type\":\"long\"},{\"name\":\"data_collection_order\",\"type\":\"long\"}],\"connect.version\":1,\"
> connect.name
> \":\"event.block\"}],\"default\":null}],\"connect.version\":1,\"
> connect.name\":\"debezium.abc-123-efg-20231005.table.u_table_dbz.Envelope\"}"
> cant be turned into a Schema, was cause by Invalid namespace
> debezium.abc-123-efg-20231005.table.u_table_dbz. It must match the regex
> '^([A-Za-z_][A-Za-z0-9_]*(\.[A-Za-z_][A-Za-z0-9_]*)*)?$', it's retriable:
> false, it's cached: false
> ```
>
> Ideally, the Avro spec would just accept hyphens since they’re a pretty
> common character and unavoidable in certain circumstances. If this is
> easier said than done, I think at the least any library used for Avro
> deserialization should account for them, including the rust library. If
> this works in Java and Python, shouldn’t it also work in rust?
>

This is exactly the reason I asked you to raise this question here in the
mailing list!
I also agree that if most/all of the SDKs will allow hyphens in the
name[space] then it is a better idea to add it to the list of allowed
characters in the specification instead of adding logic to disable the
validation.

@Avro devs: What is your opinion ?

Martin


>
> Here’s a generic example of a schema created by a Debezium connector:
>
> ```
> {
>   "type": "record",
>   "name": "Envelope",
>   "namespace": "abc-123-efg-20231005.table.u_table_dbz",
>   "fields": [
>     {
>       "name": "before",
>       "type": [
>         "null",
>         {
> ...
> ```
>
> Please let me know if you need any more details, and thank you!
>
> Jon Slusher
>
>
>
>

Reply via email to