[DISCUSS] KIP-301 Schema Inferencing in JsonConverters
Hiya, I’m late to this thread, and was just referred to the KIP from the confluent mailing list. Nice idea! I like the idea of being able to provide a Schema fallback for JsonConverter. Perhaps an interface that could return the schema given the JsonNode?For my use case, I have JSONSchemas resolvable via a schema_uri field in my JSON data. I’d like to be able to implement an interface that would look up the JSONSchema for the given message, and then convert that JSONSchema to a Connect Schema. Then, instead of inferSchema being a simple method that is either enabled or not, we can set a schema.resolver.type. Pseudocode E.g. interface JsonSchemaResolver { public Schema apply(JsonNode value); } class InferSchemaResolver implements JsonSchemaResolver { public Schema apply(JsonNode value) { return inferSchema(value); } } Or for me and my custom resolver… class JsonSchemaURIResolver implements JsonSchemaResolver { public JsonSchemaURIResolver(schemaUriPath) { // … } public Schema apply(JsonNode value) { schemaURI = value.at(schemaUriPath); // cache JSON schema here. return asConnectSchema(getJsonSchemaAtURI(schemaURI)); } public Schema asConnectSchema(JsonNode schema) { // Recursively traverse schema and pull convert “type” fields to Connect Schema types. } } And in config: # perhaps the default: schema.resolver.type=org.apache.kafka.connect.json.InferSchemaResolver # Or for me and my custom resolver… schema.resolver.type=org.apache.kafka.connect.json.JsonSchemaURIResolver If we were really fancy, I suppose the Resolver interface wouldn't have to be JsonNode specific. Then any Converter could be configured with a schema resolver implementation that returned a Schema from an Object value. (I believe this is similar to how Confluent’s AvroConverter works, but that is Confluent Schema Registry specific.) Anyway, just a thought. - Andrew Otto Senior Systems Engineer Wikimedia Foundation
Re: [DISCUSS] KIP-301 Schema Inferencing in JsonConverters
There are lots of potential details/gotchas here, with no obvious defaults. Even if we did have optional configs for the type(s) to use for empty arrays and null values, how likely is it that those would apply for all occurrences? Two fields might have empty arrays, but the best schema for each might be different. Allowing the user to specify a schema makes sense as a powerful fallback, though it wouldn't be the most usable for cases where you might just want to correct one thing. I think we should pick one approach that works. If we can't find a simple way for a few configuration options to work in nearly all cases, then perhaps the explicit schema override is the least complex approach. I'd like to hear what other people think. Randall On Tue, May 15, 2018 at 4:37 PM, Allen Tang wrote: > I've went through several iterations of back-and-forth with @rhauch on the > PR and on Confluent's Slack Community. The current thinking is that > assuming > an empty array is a String array is not necessarily the best option, nor is > assuming that all null values in a JSON node is a String. > > We might be able to account for these potentially false > assumptions/inferences by introducing new task properties (with > value.converter prefix) that explicitly define overrides for either > specific json field keys, or give the option for Kafka Connect users to > provide a full immutabl schema they know are true for the topics impacted > by the Sink Connector. > > What do you think? > > - Allen > > > On Mon, May 14, 2018 at 2:58 PM, Allen Tang wrote: > > > Hi, > > > > I just opened a KIP to add Schema Inferencing in JsonConverters for > Kafka Connect. > > > > The details of the proposal can be found here: https://cwiki.apache.org/ > confluence/display/KAFKA/KIP-301%3A+Schema+Inferencing+for+JsonConverter > > > > Also, I have created a - > > > > 1.) JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-6895 > > > > 2.) Provisional PR with initial discussion: https://github.com/apache/ > kafka/pull/5001 > > > > Looking forward to the community's feedback! Cheers! > > > > -Allen > > > > >
Re: [DISCUSS] KIP-301 Schema Inferencing in JsonConverters
I've went through several iterations of back-and-forth with @rhauch on the PR and on Confluent's Slack Community. The current thinking is that assuming an empty array is a String array is not necessarily the best option, nor is assuming that all null values in a JSON node is a String. We might be able to account for these potentially false assumptions/inferences by introducing new task properties (with value.converter prefix) that explicitly define overrides for either specific json field keys, or give the option for Kafka Connect users to provide a full immutabl schema they know are true for the topics impacted by the Sink Connector. What do you think? - Allen On Mon, May 14, 2018 at 2:58 PM, Allen Tang wrote: > Hi, > > I just opened a KIP to add Schema Inferencing in JsonConverters for Kafka > Connect. > > The details of the proposal can be found here: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-301%3A+Schema+Inferencing+for+JsonConverter > > Also, I have created a - > > 1.) JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-6895 > > 2.) Provisional PR with initial discussion: > https://github.com/apache/kafka/pull/5001 > > Looking forward to the community's feedback! Cheers! > > -Allen > >
[DISCUSS] KIP-301 Schema Inferencing in JsonConverters
Hi, I just opened a KIP to add Schema Inferencing in JsonConverters for Kafka Connect. The details of the proposal can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-301%3A+Schema+Inferencing+for+JsonConverter Also, I have created a - 1.) JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-6895 2.) Provisional PR with initial discussion: https://github.com/apache/kafka/pull/5001 Looking forward to the community's feedback! Cheers! -Allen