kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ewen Cheslack-Postava (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-6002) Kafka Connect Transform transforming JSON string into actual object
Date Fri, 23 Feb 2018 00:56:11 GMT

    [ https://issues.apache.org/jira/browse/KAFKA-6002?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16373755#comment-16373755
] 

Ewen Cheslack-Postava commented on KAFKA-6002:
----------------------------------------------

[~edvard.poliakov] Not sure if you made progress on this. The schema support in JsonConverter
doesn't use json-schema.org style, partly because it is quite complicated. To be honest, the
inline schema support with JSON is really more for demonstrative purposes – baking in and
supporting a ton of formats adds a lot of overhead to the project, so we wanted to stick to
shipping just one format with the framework and leave the rest to be community supported.
However, that meant we needed to include something that could have both schema and schemaless
modes in order to demonstrate both modes and ensure everything works with both modes. We ended
up doing this with JSON and an ad hoc schema format. But generally when using a schema, you
want something that doesn't need to ship the full schema inline with the message because that's
quite heavyweight – often times the schema ends up larger than the message data itself!

For a complete JSON w/ schemas solution, I would probably suggest implementing Converters
that look a lot like what Confluent has for Avro and using json-schema.org to express the
schemas. The one difference is that now that we have headers, I'd put the schema ID information
into a header instead and make the value just the JSON payload (whereas Avro has some additional
framing in the value itself).

For a transformation that does this you *could* just omit the schema entirely. That is an
option in Connect. Basically this would just mean that the transform only works when the user/connectors
expect schemaless data.

Regarding inference, you can also just do this on a per-message basis instead of continuously
updating a schema. There is a risk that you end up with lots of schemas because of this (since
each could be unique), but for a lot of cases that may not be expected. I also have an SMT
that infers schemas, so does something similar to what you'd need here [https://github.com/ewencp/kafka/commit/3abb54a8062fe727ddaabc4dd5a552dd0b465a03] I
didn't complete both modes, but the idea was to allow either inferring on a per-message basis
*or* specifying a schema (whether the JsonConverter variant or json-schema.org style) and
validating & add it to the record. I think offering those two options in your SMT would
give good flexibility as well.

> Kafka Connect Transform transforming JSON string into actual object
> -------------------------------------------------------------------
>
>                 Key: KAFKA-6002
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6002
>             Project: Kafka
>          Issue Type: Improvement
>          Components: KafkaConnect
>            Reporter: Edvard Poliakov
>            Priority: Minor
>
> My colleague and I have been working on a new Transform, that takes a JSON string and
transforms it into an actual object, like this:
> {code} 
> {
>   "a" : "{\"b\": 23}"
> }
> {code}
> into
> {code}
> {
>   "a" : {
>        "b" : 23
>   }
> }
> {code}
> There is no robust way of building a Schema from a JSON object itself, as it can be something
like an empty array or a null, that doesn't provide any info on the schema of the object.
So I see two options here.
> 1. For a transform to take in schema as a transform parameter. The problem I found with
this is that it is not clear what JSON schema specification should be used for this? I assume
it would be reasonable to use http://json-schema.org/, but it doesn't seem that Kafka Connect
supports it currently, moreover reading through JsonConverter class in Kafka Connect, I am
not able to understand what spec does the Json Schema have that is used in that class, for
example {{asConnectSchema}} method on {{JsonConverter}}, [see here|https://github.com/apache/kafka/blob/trunk/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java#L415].
> 2. On each object received, keep updating the schema, but I can't see a standard and
robust way of handling edge cases.
> I am happy to create a pull request for this transform, if we can agree on something
here. :)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message