[ https://issues.apache.org/jira/browse/KAFKA-3910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15361212#comment-15361212 ]
ASF GitHub Bot commented on KAFKA-3910: --------------------------------------- GitHub user johnhofman opened a pull request: https://github.com/apache/kafka/pull/1582 KAFKA-3910: Cyclic schema support in ConnectSchema and SchemaBuilder This feature uses a FutureSchema as a placeholder to be resolved later. Resolution is attempted whenever a ConnectSchema is constructed, it attempts to resolve all its children (fields, keySchema, or valueSchema) and recurses until the end of the tree. A FutureSchema is resolved when it finds a parent schema that matches its name, and optional flag. If a FutureSchema is accessed before being resolved, it will throw a DataException. The SchemaBuilder constructs a FutureSchema if a field is added with only a type name. You can merge this pull request into a Git repository by running: $ git pull https://github.com/johnhofman/kafka cyclic Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/1582.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1582 ---- commit d932db4544cc2d20a46bf873fbd92f2c09450276 Author: John Hofman <johnhof...@users.noreply.github.com> Date: 2016-06-30T20:04:19Z Add FutureSchema to support cyclic schemas commit 7d95c487d4a5bf3cb751deb98425e5123abb461b Author: John Hofman <johnhof...@users.noreply.github.com> Date: 2016-07-04T07:32:47Z Fix resolution failure test commit 09f1b47c238ff10f58681e44816f2ba39ed95166 Author: John Hofman <johnhof...@users.noreply.github.com> Date: 2016-07-04T09:53:58Z Move cyclic comparison resolution to FutureSchema commit c1c632b51f80d81c29cb66a35c5aed867ad869e7 Author: John Hofman <johnhof...@users.noreply.github.com> Date: 2016-07-04T11:26:35Z Clean up unused tokens, minor spelling fixes ---- > Cyclic schema support in ConnectSchema and SchemaBuilder > -------------------------------------------------------- > > Key: KAFKA-3910 > URL: https://issues.apache.org/jira/browse/KAFKA-3910 > Project: Kafka > Issue Type: Improvement > Components: KafkaConnect > Affects Versions: 0.10.0.0 > Reporter: John Hofman > Assignee: Ewen Cheslack-Postava > Priority: Blocker > > Cyclic schema's are not supported by ConnectSchema or SchemaBuilder. > Subsequently the AvroConverter (confluentinc/schema-registry) hits a stack > overflow when converting a cyclic avro schema, e.g: > {code} > {"type":"record", > "name":"list","fields":[{"name":"value","type":"int"},{"name":"next","type":["null","list"]}]} > {code} > This is a blocking issue for all connectors running on the connect framework > with data containing cyclic references. The AvroConverter cannot support > cyclic schema's until the underlying ConnectSchema and SchemaBuilder do. > To reproduce the stack-overflow (Confluent-3.0.0): > Produce some cyclic data: > {code} > bin/kafka-avro-console-producer --broker-list localhost:9092 --topic test > --property value.schema='{"type":"record", > "name":"list","fields":[{"name":"value","type":"int"},{"name":"next","type":["null","list"]}]}' > {"value":1,"next":null} > {"value":1,"next":{"list":{"value":2,"next":null}}} > {code} > Then try to consume it with connect: > {code:title=connect-console-sink.properties} > name=local-console-sink > connector.class=org.apache.kafka.connect.file.FileStreamSinkConnector > tasks.max=1 > topics=test > {code} > {code} > ./bin/connect-standalone > ./etc/schema-registry/connect-avro-standalone.properties > connect-console-sink.properties > … start up logging … > java.lang.StackOverflowError > at org.apache.avro.JsonProperties.getJsonProp(JsonProperties.java:54) > at org.apache.avro.JsonProperties.getProp(JsonProperties.java:45) > at io.confluent.connect.avro.AvroData.toConnectSchema(AvroData.java:1055) > at io.confluent.connect.avro.AvroData.toConnectSchema(AvroData.java:1103) > at io.confluent.connect.avro.AvroData.toConnectSchema(AvroData.java:1137) > at io.confluent.connect.avro.AvroData.toConnectSchema(AvroData.java:1103) > at io.confluent.connect.avro.AvroData.toConnectSchema(AvroData.java:1137) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)