Skip to content
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

fix: struct be converted to String for json_sr format #9678

Open
wants to merge 8 commits into
base: master
Choose a base branch
from

Conversation

aliehsaeedii
Copy link
Contributor

fixes issue #9580

Description

A topic that has a STRUCT column cannot be deserialized from KSQL if VARCHAR is used in the KSQL schema. This issue is found only on JSON_SR topics. This fix addresses this issue.

Testing done

manual testing
QTT

Reviewer checklist

  • Ensure docs are updated if necessary. (eg. if a user visible feature is being added or changed).
  • Ensure relevant issues are linked (description should include text like "Fixes #")

@aliehsaeedii aliehsaeedii requested a review from a team as a code owner October 31, 2022 19:39
// Then:
assertThat(e.getCause(), (hasMessage(containsString(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@spena wrote this code. Maybe he knows about this...

{"topic": "test_topic", "value": {"FOO": {"F0": 1}}}
],
"outputs": [
{"topic": "OUTPUT", "value": {"FOO": "Struct{F0=1}"}}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you check a plain JSON format to see how the Struct looks like? I think the plain JSON is only {F0=1} (withouth the Struct word).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you check a plain JSON format to see how the Struct looks like? I think the plain JSON is only {F0=1} (withouth the Struct word).

The plain json does not have the word Struct (As you told). I believe that the word Struct comes from the toString() method implemented by the Struct class: org.apache.kafka.connect.data.Struct.toString()

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but protobuf and avro show the word Struct as well.

@@ -231,7 +232,7 @@ private Object toKsqlValue(
case STRUCT:
return toKsqlStruct(schema, connectSchema, (Struct) convertedValue, pathStr);
case STRING:
// use String.valueOf to convert various int types and Boolean to string
// use String.valueOf to convert various int types, Struct and Boolean to string
return String.valueOf(convertedValue);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think something wrong is gonna happen if we done here. The ConnectDataTranslator is a generic class for all Connect types (avro, protobuf, json). The struct->string conversion should be in JSON string, i.e. {k1=v1}. You can verify this with a plain JSON format or with JSON_SR in version 0.26 or lower. I actually not sure what Avro/Protobuf should do wit Struct->String.

  1. Could you check in older ksql versions if Struct->String is supported with Avro and Protobuf?
  2. Could you check in older ksql versions what Struct->String wit JSON_SR and plain JSON display?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think something wrong is gonna happen if we done here.

The issue addresses exactly this line for change (please have a look at the issue explanation)

{"topic": "test_topic", "value": {"FOO": {"F0": 1, "F1": "bar"}}}
],
"outputs": [
{"topic": "OUTPUT", "value": {"FOO": "{F0=1,F1=bar}"}}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I run these cases and got the following JSON output in the old version:

{"F0":1,"F1":"2"}
{"F0":3,"F1":"this is a string"}
{"F0":3,"F1":"\"quotes\""}
{"F0":3,"F1":""}
null

Could you verify the JSON string will be formed like that?

I tried to add a ' single-quote escaped but KSQL didn't let me. Can you figure out how to add that in case KSQL finds a string with it?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, could you add more complex columns? say f2 struct<a1 int, a2 struct <b1 string>>. Include maps too. I think the code does not take into consideration structs inside a struct.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also add all the types we support to make sure everything is well covered and tested.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, could you add more complex columns? say f2 struct<a1 int, a2 struct <b1 string>>. Include maps too. I think the code does not take into consideration structs inside a struct.

You mean the whole f2 struct<a1 int, a2 struct <b1 string>> must be convertible to string as well? Currently the output is {a1=1,a2=Struct{b1=bar}}. Is it acceptable, or you think the Struct word must be removed?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Struct should be removed too. You could check the old JSON_SR deserialization to see how it works.

@@ -129,7 +129,8 @@ private static void validateType(
Schema.Type.FLOAT32,
Schema.Type.FLOAT64,
Schema.Type.BOOLEAN,
Schema.Type.STRING
Schema.Type.STRING,
Schema.Type.STRUCT
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This issue was created for STRUCT->STRING. But the issue also happens for ARRAY->STRING. Look at the KsqlJsonDeserializer class and processString method. That's where the old JSON_SR deserializer was doing.

Do you think you can add the ARRAY conversion to this PR or would you need another one?

@@ -244,6 +248,27 @@ private Object toKsqlValue(
}
}

private String structToString(final Struct input) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there an API we can use to generate a JSON string? Something that handle cases with String characters (special chars, nulls)? Not sure if we can re-use the ObjectMapper to generate a JSON string.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants