Joe,

There was a similar issue I worked on recently that had to do with array 
handling:
https://issues.apache.org/jira/browse/NIFI-5316 
<https://issues.apache.org/jira/browse/NIFI-5316>

From a quick glance this may be a different variation of a similar issue.

We are using Parquet’s Java API that allows you to read Parquet into Avro, and 
some of the resulting conversions they perform are not lining up with what NiFi 
is expecting in terms of Avro types.

Feel free to create a JIRA with the stack trace.

Thanks,

Bryan

> On Jun 28, 2018, at 7:49 PM, Joe Trite <[email protected]> wrote:
> 
> Greetings, I'm using NiFi 1.6 trying to ListHDFS -> FetchParquet.  I'm seeing 
> the below error on the fetch side.  Are there any limitations with 
> FetchParquet data type conversions?  I am able to cat/dump the file with 
> parquet-tools and load/query the data in Spark so believe the files are 
> solid/not corrupt.  Any help would be appreciated.
> 
> Here is the data type of the field throwing the error:  optional 
> fixed_len_byte_array(16) amt (DECIMAL(38,10));
> 
> Here is the error:
> 2018-06-28 22:56:57,030 ERROR [Timer-Driven Process Thread-3] 
> o.a.nifi.processors.parquet.FetchParquet 
> FetchParquet[id=9280c518-6a92-363f-893a-5a03feb45ebc] Failed to retrieve 
> content from /data/000157_0 for 
> StandardFlowFileRecord[uuid=112b6e45-011a-44ac-b258-97f52a505597,claim=,offset=0,name=000157_0,size=0]
>  due to 
> org.apache.nifi.serialization.record.util.IllegalTypeConversionException: 
> Cannot convert value [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 90, -13, 16, 122, 64, 0] 
> of type class org.apache.avro.generic.GenericData$Fixed because no compatible 
> types exist in the UNION for field amt; routing to failure: 
> org.apache.nifi.serialization.record.util.IllegalTypeConversionException: 
> Cannot convert value [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 90, -13, 16, 122, 64, 0] 
> of type class org.apache.avro.generic.GenericData$Fixed because no compatible 
> types exist in the UNION for field amt
> org.apache.nifi.serialization.record.util.IllegalTypeConversionException: 
> Cannot convert value [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 90, -13, 16, 122, 64, 0] 
> of type class org.apache.avro.generic.GenericData$Fixed because no compatible 
> types exist in the UNION for field amt
>         at 
> org.apache.nifi.avro.AvroTypeUtil.convertUnionFieldValue(AvroTypeUtil.java:759)
>         at 
> org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:844)
>         at 
> org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:710)
>         at 
> org.apache.nifi.processors.parquet.record.AvroParquetHDFSRecordReader.nextRecord(AvroParquetHDFSRecordReader.java:62)
>         at 
> org.apache.nifi.processors.hadoop.AbstractFetchHDFSRecord.lambda$null$0(AbstractFetchHDFSRecord.java:197)
>         at 
> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2621)
>         at 
> org.apache.nifi.processors.hadoop.AbstractFetchHDFSRecord.lambda$onTrigger$1(AbstractFetchHDFSRecord.java:193)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:360)
>         at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1678)
>         at 
> org.apache.nifi.processors.hadoop.AbstractFetchHDFSRecord.onTrigger(AbstractFetchHDFSRecord.java:177)
>         at 
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
>         at 
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1147)
>         at 
> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:175)
>         at 
> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117)
>         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)
> 
> Thanks
> Joe

Attachment: signature.asc
Description: Message signed with OpenPGP

Reply via email to