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

Pierre Villard commented on NIFI-7790:
--------------------------------------

 
{noformat}
2020-10-27 19:18:07,954 ERROR [Event-Driven Process Thread-1] 
o.a.n.processors.standard.ConvertRecord 
ConvertRecord[id=a2eb82b4-ac5c-32f9-062a-194ed4057ecb] Failed to process 
StandardFlowFileRecord[uuid=080f5df4-a920-4a82-b99e-8776db105df7,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1603822546154-1, container=default, 
section=1], offset=2956, 
length=227],offset=0,name=080f5df4-a920-4a82-b99e-8776db105df7,size=227]; will 
route to failure: org.apache.avro.SchemaParseException: Can't redefine: 
org.apache.nifi.itemType
org.apache.avro.SchemaParseException: Can't redefine: org.apache.nifi.itemType
        at org.apache.avro.Schema$Names.put(Schema.java:1128)
        at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:562)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:690)
        at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
        at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
        at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
        at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
        at org.apache.avro.Schema.toString(Schema.java:324)
        at org.apache.avro.Schema.toString(Schema.java:314)
        at org.apache.avro.file.DataFileWriter.create(DataFileWriter.java:144)
        at org.apache.avro.file.DataFileWriter.create(DataFileWriter.java:135)
        at 
org.apache.nifi.avro.WriteAvroResultWithSchema.<init>(WriteAvroResultWithSchema.java:45)
        at 
org.apache.nifi.avro.AvroRecordSetWriter.createWriter(AvroRecordSetWriter.java:149)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:254)
        at 
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:105)
        at com.sun.proxy.$Proxy376.createWriter(Unknown Source)
        at 
org.apache.nifi.processors.standard.AbstractRecordProcessor$1.process(AbstractRecordProcessor.java:150)
        at 
org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2988)
        at 
org.apache.nifi.controller.repository.BatchingSessionFactory$HighThroughputSession.write(BatchingSessionFactory.java:222)
        at 
org.apache.nifi.processors.standard.AbstractRecordProcessor.onTrigger(AbstractRecordProcessor.java:122)
        at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
        at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1174)
        at 
org.apache.nifi.controller.scheduling.EventDrivenSchedulingAgent$EventDrivenTask.trigger(EventDrivenSchedulingAgent.java:354)
        at 
org.apache.nifi.controller.scheduling.EventDrivenSchedulingAgent$EventDrivenTask.run(EventDrivenSchedulingAgent.java:233)
        at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
        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:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748){noformat}
Your first example is going to generate this schema:

 

 
{noformat}
{
   "type":"record",
   "name":"nifiRecord",
   "namespace":"org.apache.nifi",
   "fields":[
      {
         "name":"authors",
         "type":[
            "null",
            {
               "type":"record",
               "name":"authorsType",
               "fields":[
                  {
                     "name":"item",
                     "type":[
                        "null",
                        {
                           "type":"record",
                           "name":"itemType",
                           "fields":[
                              {
                                 "name":"name",
                                 "type":[
                                    "null",
                                    "string"
                                 ]
                              }
                           ]
                        }
                     ]
                  }
               ]
            }
         ]
      },
      {
         "name":"editors",
         "type":[
            "null",
            {
               "type":"record",
               "name":"editorsType",
               "fields":[
                  {
                     "name":"item",
                     "type":[
                        "null",
                        "itemType"
                     ]
                  }
               ]
            }
         ]
      }
   ]
}{noformat}
In your second example the "item" record is changing and this is not allowed in 
the current form of the processor (because we use the same "name" field). What 
I would recommend is to provide a schema instead using the schema inference. 
The below schema would work with your data for the first example you gave:

 

 
{noformat}
{
   "type":"record",
   "name":"nifiRecord",
   "namespace":"org.apache.nifi",
   "fields":[
      {
         "name":"authors",
         "type":[
            "null",
            {
               "type":"record",
               "name":"authorsType",
               "fields":[
                  {
                     "name":"item",
                     "type":[
                        "null",
                        {
                           "type":"record",
                           "name":"itemType1",
                           "fields":[
                              {
                                 "name":"name",
                                 "type":[
                                    "null",
                                    "string"
                                 ]
                              }
                           ]
                        }
                     ]
                  }
               ]
            }
         ]
      },
      {
         "name":"editors",
         "type":[
            "null",
            {
               "type":"record",
               "name":"editorsType",
               "fields":[
                  {
                     "name":"item",
                     "type":[
                        "null",
                        {
                           "type":"record",
                           "name":"itemType2",
                           "fields":[
                              {
                                 "name":"commercialName",
                                 "type":[
                                    "null",
                                    "string"
                                 ]
                              }
                           ]
                        }
                     ]
                  }
               ]
            }
         ]
      }
   ]
}{noformat}
We could definitely improve things though to support such cases but overall 
it's always better to provide a schema.

 

> XML record reader - failure on well-formed XML
> ----------------------------------------------
>
>                 Key: NIFI-7790
>                 URL: https://issues.apache.org/jira/browse/NIFI-7790
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 1.11.4
>            Reporter: Pierre Gramme
>            Priority: Major
>              Labels: records, xml
>         Attachments: bug-parse-xml.xml
>
>
> I am using ConvertRecord in order to parse XML flowfiles to Avro, with the 
> Infer Schema strategy. Some input flowfiles are sent to the failure output 
> queue whereas they are well-formed: 
> {code:java}
> <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
> <root>
>       <authors>
>               <item>
>                       <name>Neil Gaiman</name>
>               </item>
>       </authors>
>       <editors>
>               <item>
>                       <commercialName>Hachette</commercialName>
>               </item>
>       </editors>
> </root>
> {code}
> Note the use of authors/item/name on one side, and 
> editors/item/commercialName on the other side.
> On the other hand, this gets correctly parsed: 
> {code:java}
> <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
> <root>
>       <authors>
>               <item>
>                       <name>Neil Gaiman</name>
>               </item>
>       </authors>
>       <editors>
>               <item>
>                       <name>Hachette</name>
>               </item>
>       </editors>
> </root>
> {code}
> See the attached template for minimal reproducible example.
>  
> My interpretation is that the failure in the first case is due to 2 
> independent XML node types having the same name (<item> in this case) but 
> having different types and occurring in different parents with different 
> types. In the second case, both <item>'s actually have the same node type. I 
> didn't use any Schema Inference Cache, so both item types should be inferred 
> independently. 
> Since the first document is legal XML (an XSD could be written for it) and 
> can also be represented in Avro, its conversion shouldn't fail.
> I'll be happy to provide more details if needed.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to