-
Notifications
You must be signed in to change notification settings - Fork 1k
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
base: master
Are you sure you want to change the base?
Changes from all commits
2f3a042
89700f5
33f232d
3bddc47
abe597f
938210c
f1b0101
139e3ec
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -15,6 +15,8 @@ | |
|
||
package io.confluent.ksql.serde.connect; | ||
|
||
import com.fasterxml.jackson.core.JsonProcessingException; | ||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import io.confluent.ksql.serde.SerdeUtils; | ||
import java.nio.ByteBuffer; | ||
import java.util.ArrayList; | ||
|
@@ -129,7 +131,10 @@ private static void validateType( | |
Schema.Type.FLOAT32, | ||
Schema.Type.FLOAT64, | ||
Schema.Type.BOOLEAN, | ||
Schema.Type.STRING | ||
Schema.Type.STRING, | ||
Schema.Type.STRUCT, | ||
Schema.Type.ARRAY, | ||
Schema.Type.MAP | ||
}; | ||
|
||
private static void validateSchema( | ||
|
@@ -170,7 +175,7 @@ private static Object maybeConvertLogicalType( | |
if (connectSchema.name() == null) { | ||
return connectValue; | ||
} | ||
switch (connectSchema.name()) { | ||
switch (connectSchema.name()) { | ||
case Date.LOGICAL_NAME: | ||
return Date.fromLogical(connectSchema, (java.util.Date) connectValue); | ||
case Time.LOGICAL_NAME: | ||
|
@@ -231,7 +236,10 @@ 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 | ||
if (convertedValue instanceof Struct) { | ||
return structToString((Struct) convertedValue); | ||
} | ||
// use String.valueOf to convert various int types, and Boolean to string | ||
return String.valueOf(convertedValue); | ||
case BYTES: | ||
if (convertedValue instanceof byte[]) { | ||
|
@@ -244,6 +252,47 @@ private Object toKsqlValue( | |
} | ||
} | ||
|
||
private String structToString(final Struct input) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. |
||
final StringBuilder sb = new StringBuilder("{"); | ||
boolean first = true; | ||
final List<Field> fields = input.schema().fields(); | ||
|
||
for (Field field : fields) { | ||
final Object value = input.get(field); | ||
if (value != null) { | ||
if (first) { | ||
first = false; | ||
} else { | ||
sb.append(","); | ||
} | ||
sb.append("\"").append(field.name()).append("\"").append(":"); | ||
if (value instanceof Struct) { | ||
sb.append(structToString((Struct) value)); | ||
} else if (value instanceof Map || value instanceof ArrayList) { | ||
sb.append(objectToString(value)); | ||
} else if (value instanceof String) { | ||
sb.append("\"").append(value).append("\""); | ||
} else { | ||
sb.append(value); | ||
} | ||
} | ||
} | ||
|
||
return sb.append("}").toString(); | ||
} | ||
|
||
private <T> String objectToString(final T input) { | ||
final ObjectMapper mapper = new ObjectMapper(); | ||
final String json; | ||
try { | ||
json = mapper.writeValueAsString(input); | ||
} catch (JsonProcessingException e) { | ||
throw new RuntimeException(e); | ||
} | ||
return json; | ||
} | ||
|
||
|
||
private List<?> toKsqlArray( | ||
final Schema valueSchema, | ||
final Schema connectValueSchema, | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -66,6 +66,7 @@ | |
import org.apache.kafka.common.serialization.Deserializer; | ||
import org.apache.kafka.connect.data.ConnectSchema; | ||
import org.apache.kafka.connect.data.Date; | ||
import org.apache.kafka.connect.data.Field; | ||
import org.apache.kafka.connect.data.Schema; | ||
import org.apache.kafka.connect.data.SchemaBuilder; | ||
import org.apache.kafka.connect.data.Struct; | ||
|
@@ -815,14 +816,9 @@ public void shouldThrowIfCanNotCoerceToString() { | |
final byte[] bytes = givenAvroSerialized(AN_ORDER, ORDER_AVRO_SCHEMA); | ||
|
||
// When: | ||
final Exception e = assertThrows( | ||
SerializationException.class, | ||
() -> deserializer.deserialize(SOME_TOPIC, bytes) | ||
); | ||
|
||
final Object result = deserializer.deserialize(SOME_TOPIC, bytes); | ||
// Then: | ||
assertThat(e.getCause(), (hasMessage(containsString( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @spena wrote this code. Maybe he knows about this... |
||
"Cannot deserialize type struct as type string")))); | ||
assertThat(result, is(toStructString(AN_ORDER))); | ||
} | ||
|
||
@Test | ||
|
@@ -1790,4 +1786,20 @@ private static List<?> buildEntriesForMapWithOptionalKey( | |
|
||
return ImmutableList.of(e1, e2); | ||
} | ||
|
||
private static String toStructString (Map<String, Object> map) { | ||
StringBuilder sb = new StringBuilder("{"); | ||
boolean first = true; | ||
|
||
for (Map.Entry<String, Object> entry : map.entrySet()) { | ||
if (first) { | ||
first = false; | ||
} else { | ||
sb.append(","); | ||
} | ||
sb.append(entry.getKey()).append("=").append(entry.getValue()); | ||
} | ||
|
||
return sb.append("}").toString(); | ||
} | ||
} |
There was a problem hiding this comment.
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.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The issue addresses exactly this line for change (please have a look at the issue explanation)