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

Randall Hauch commented on KAFKA-3832:
--------------------------------------

No, the problem is actually [here|]:

{code:java}
    private JsonNode convertToJsonWithEnvelope(Schema schema, Object value) {
        return new JsonSchema.Envelope(asJsonSchema(schema), 
convertToJson(schema, value)).toJsonNode();
    }
{code}

This method _always_ returns an envelope, regardless of the schema and value. 
It is called whenever {{enableSchemas=true}} because of this code:

{code:java}
    public byte[] fromConnectData(String topic, Schema schema, Object value) {
        JsonNode jsonValue = enableSchemas ? convertToJsonWithEnvelope(schema, 
value) : convertToJsonWithoutEnvelope(schema, value);
        try {
            return serializer.serialize(topic, jsonValue);
        } catch (SerializationException e) {
            throw new DataException("Converting Kafka Connect data to byte[] 
failed due to serialization error: ", e);
        }
    }
{code}

I think the problem is that the {{toConnect}} method currently throws an 
exception when schemas are enabled and the JSON key or value is null. So, if 
the JSON converter is changed to output null keys/values rather than an empty 
envelope, then the {{toConnect}} will also have to be changed.

Note that this is not a problem when {{enableSchemas=false}}, which is probably 
how many people will use the converter to store JSON data in Kafka topics as 
the schemas can provide significant overhead to the size of the messages. If 
that is the case, then this change may not be appropriate or valuable.

> Kafka Connect's JSON Converter never outputs a null value
> ---------------------------------------------------------
>
>                 Key: KAFKA-3832
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3832
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 0.9.0.1
>            Reporter: Randall Hauch
>              Labels: newbie
>
> Kafka Connect's JSON Converter will never output a null value when 
> {{enableSchemas=true}}. This means that when a connector outputs a 
> {{SourceRecord}} with a null value, the JSON Converter will always produce a 
> message value with:
> {code:javascript}
> {
>   "schema": null,
>   "payload": null
> }
> {code}
> And, this means that while Kafka log compaction will always be able to remove 
> earlier messages with the same key, log compaction will never remove _all_ of 
> the messages with the same key. 
> The JSON Connector's {{fromConnectData(...)}} should always return null when 
> it is supplied a null value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to