Charlie Meyer created NIFI-4551:
-----------------------------------

             Summary: JSON to Avro conversion fails for records which have 
nested records
                 Key: NIFI-4551
                 URL: https://issues.apache.org/jira/browse/NIFI-4551
             Project: Apache NiFi
          Issue Type: Bug
          Components: Core Framework
    Affects Versions: 1.4.0
            Reporter: Charlie Meyer
         Attachments: ExampleObject.avsc, examplePayload.avro, 
examplePayload.json, example_object.avdl, nifi_json_avro_bug.xml, 
schema_registry_payload.json

JSON to Avro conversion fails for records which have nested records.

Given a confluent schema registry exists at some accessible address

Steps to recreate:

# register the schema:
{{$ curl -H "Content-Type: application/vnd.schemaregistry.v1+json" -d 
@schema_registry_payload.json 4.3.2.1:8081/subjects/nifiBug/versions | jq}}
# verify that we can use that schema to convert json to and from avro
{{$ avro-tools fromjson --schema-file ExampleObject.avsc examplePayload.json > 
examplePayload.avro
$ avro-tools tojson examplePayload.avro  | jq}}
# apply the attached template to nifi: nifi_avro_bug.xml
# start up the components that the template created in nifi
run the following command: 
{{$ curl -X POST -d @examplePayload.json http://localhost:5001/ | jq}}

The serialization to avro fails with the following stack trace:
{{
2017-10-30 11:41:02,199 ERROR [Timer-Driven Process Thread-5] 
o.a.n.p.k.pubsub.PublishKafkaRecord_0_10 
PublishKafkaRecord_0_10[id=19a933c0-f766-1221-4373-21c102ff71ab] Failed to send 
all message for 
StandardFlowFileRecord[uuid=4834f5cb-f513-49ee-8c3e-305a3acc64b6,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1509378326140-1, container=default, 
section=1], offset=4297, length=156],offset=0,name=75094273920075,size=156] to 
Kafka; routing to failure due to 
org.apache.avro.file.DataFileWriter$AppendWriteException: 
java.lang.NullPointerException: null of string in field name of 
com.example.SubtypeA of union in field payload of com.example.ExampleObject: {}
org.apache.avro.file.DataFileWriter$AppendWriteException: 
java.lang.NullPointerException: null of string in field name of 
com.example.SubtypeA of union in field payload of com.example.ExampleObject
        at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:308)
        at 
org.apache.nifi.avro.WriteAvroResultWithSchema.writeRecord(WriteAvroResultWithSchema.java:61)
        at 
org.apache.nifi.serialization.AbstractRecordSetWriter.write(AbstractRecordSetWriter.java:59)
        at 
org.apache.nifi.processors.kafka.pubsub.PublisherLease.publish(PublisherLease.java:114)
        at 
org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_10$1.process(PublishKafkaRecord_0_10.java:339)
        at 
org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2174)
        at 
org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2144)
        at 
org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_10.onTrigger(PublishKafkaRecord_0_10.java:331)
        at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
        at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1119)
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:147)
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47)
        at 
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:128)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NullPointerException: null of string in field name of 
com.example.SubtypeA of union in field payload of com.example.ExampleObject
        at 
org.apache.avro.generic.GenericDatumWriter.npe(GenericDatumWriter.java:132)
        at 
org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:126)
        at 
org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:73)
        at 
org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:60)
        at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:302)
        ... 19 common frames omitted
Caused by: java.lang.NullPointerException: null
        at org.apache.avro.io.Encoder.writeString(Encoder.java:121)
        at 
org.apache.avro.generic.GenericDatumWriter.writeString(GenericDatumWriter.java:254)
        at 
org.apache.avro.generic.GenericDatumWriter.writeString(GenericDatumWriter.java:249)
        at 
org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:115)
        at 
org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:73)
        at 
org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:153)
        at 
org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:143)
        at 
org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:105)
        at 
org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:73)
        at 
org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:112)
        at 
org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:73)
        at 
org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:153)
        at 
org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:143)
        at 
org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:105)
        ... 22 common frames omitted}}


I did a bit of digging on this one and had a few observations:

When writing to avro, the following code is run to generate the avro record: 
[https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java#L558]

Here it iterates over all the fields of the object. This same code appears to 
be excuted on thee nested record. When run on the nested record, the schema on 
it has an empty list of fields. Thus, when the avro is generated, it has null 
values for all fields on the nested record.

This appears to be being set here: 
[https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java#L162]





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

Reply via email to