Skip to content

RisingWave fails to read Avro schemas that have multiple Debezium VariableScaleDecimals in a single record #14702

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
james-johnston-thumbtack opened this issue Jan 21, 2024 · 3 comments · Fixed by #14781
Assignees
Labels
BY-user Issue found by: User. type/bug Type: Bug. Only for issues.
Milestone

Comments

@james-johnston-thumbtack
Copy link

james-johnston-thumbtack commented Jan 21, 2024

Describe the bug

RisingWave recently added support for VariableScaleDecimal: #10247

However, if a Debezium connector is used to write records with multiple variable scale decimal columns to Kafka with Avro encoding, RisingWave can't ingest the schema without failing. It seems like if a record references a data type created elsewhere, then RisingWave can't parse it.

Error message/log

Failed to run the query

Caused by:
  unsupported type in Avro: Ref { name: Name { name: "VariableScaleDecimal", namespace: Some("io.debezium.data") } }

To Reproduce

Create an Avro schema with a record that contains fields like this, which is a copy from what Debezium 2.5.0 outputs. Then try to create a table with a connector to read from it.

            {
              "name": "unconstrained_decimal",
              "type": [
                "null",
                {
                  "type": "record",
                  "name": "VariableScaleDecimal",
                  "namespace": "io.debezium.data",
                  "fields": [
                    {
                      "name": "scale",
                      "type": "int"
                    },
                    {
                      "name": "value",
                      "type": "bytes"
                    }
                  ],
                  "connect.doc": "Variable scaled decimal",
                  "connect.name": "io.debezium.data.VariableScaleDecimal",
                  "connect.version": 1
                }
              ],
              "default": null
            },
            {
              "name": "unconstrained_numeric",
              "type": [
                "null",
                "io.debezium.data.VariableScaleDecimal"
              ],
              "default": null
            },

Note that if I delete the second field that references the type defined in the first one, then RisingWave can successfully parse the schema.

Expected behavior

RisingWave should successfully read any number of VariableScaleDecimals, not just the one column.

How did you deploy RisingWave?

Docker compose, with the included basic docker compose file.

The version of RisingWave

 PostgreSQL 9.5-RisingWave-1.6.0 (71898b4873a32b9df911a8157864100c0dfab27d)

Additional context

No response

@james-johnston-thumbtack james-johnston-thumbtack added the type/bug Type: Bug. Only for issues. label Jan 21, 2024
@github-actions github-actions bot added this to the release-1.7 milestone Jan 21, 2024
@neverchanje neverchanje added the BY-user Issue found by: User. label Jan 22, 2024
@fuyufjh
Copy link
Collaborator

fuyufjh commented Jan 24, 2024

@StrikeW Can you please take a look?

@xuefengze
Copy link
Contributor

Can the issue be solved by adding the code below in https://github.com/risingwavelabs/risingwave/blob/main/src/connector/src/parser/avro/util.rs#L141?

Schema::Ref { name } => {
    if name.name == DBZ_VARIABLE_SCALE_DECIMAL_NAME
        && name.namespace == Some(DBZ_VARIABLE_SCALE_DECIMAL_NAMESPACE.into())
    {
        return Ok(DataType::Decimal);
    }
    return Err(anyhow::format_err!(
        "unsupported type in Avro: {:?}",
        schema
    ));
}

@StrikeW
Copy link
Contributor

StrikeW commented Jan 24, 2024

Can the issue be solved by adding the code below in https://github.com/risingwavelabs/risingwave/blob/main/src/connector/src/parser/avro/util.rs#L141?

Schema::Ref { name } => {

    if name.name == DBZ_VARIABLE_SCALE_DECIMAL_NAME

        && name.namespace == Some(DBZ_VARIABLE_SCALE_DECIMAL_NAMESPACE.into())

    {

        return Ok(DataType::Decimal);

    }

    return Err(anyhow::format_err!(

        "unsupported type in Avro: {:?}",

        schema

    ));

}

You can verify it with a unit test. I will have a try later.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
BY-user Issue found by: User. type/bug Type: Bug. Only for issues.
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants