[
https://issues.apache.org/jira/browse/NIFI-15548?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sven Van Kerrebroeck updated NIFI-15548:
----------------------------------------
Description:
It seems that since version 2.7 the ParquetReader is having problems with
timestamps in parquet data.
When reading a parquet-file with a timestamp column (timestamp_millis), an
error is thrown:
!image-2026-02-04-12-58-19-172.png!
{noformat}
ConvertRecord[id=59968c39-4fb7-324a-b1b7-bd2e4b93ccb7] Failed to process
FlowFile[filename=parquet_test.parquet]; will route to failure:
java.lang.ClassCastException: class java.time.Instant cannot be cast to class
java.lang.Long (java.time.Instant and java.lang.Long are in module java.base of
loader 'bootstrap')
java.lang.ClassCastException: class java.time.Instant cannot be cast to class
java.lang.Long (java.time.Instant and java.lang.Long are in module java.base of
loader 'bootstrap') at
org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1175) at
org.apache.nifi.avro.AvroTypeUtil.lambda$normalizeValue$3(AvroTypeUtil.java:1186)
at
org.apache.nifi.avro.AvroTypeUtil.convertUnionFieldValue(AvroTypeUtil.java:1016)
at
org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1186) at
org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:979)
at
org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:943)
at
org.apache.nifi.parquet.record.ParquetRecordReader.nextRecord(ParquetRecordReader.java:111)
at
org.apache.nifi.serialization.RecordReader.nextRecord(RecordReader.java:50) at
java.base/jdk.internal.reflect.DirectMethodHandleAccessor.invoke(DirectMethodHandleAccessor.java:103)
at java.base/java.lang.reflect.Method.invoke(Method.java:580) at
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:251)
at
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler$ProxiedReturnObjectInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:237)
at jdk.proxy196/jdk.proxy196.$Proxy501.nextRecord(Unknown Source)
at
org.apache.nifi.processors.standard.AbstractRecordProcessor.lambda$onTrigger$0(AbstractRecordProcessor.java:132)
at
org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3410)
at
org.apache.nifi.processors.standard.AbstractRecordProcessor.onTrigger(AbstractRecordProcessor.java:125)
at
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1274)
at
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:229)
at
org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:59)
at org.apache.nifi.engine.FlowEngine.lambda$wrap$1(FlowEngine.java:105) at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317) at
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1583)
{noformat}
Steps to reproduce:
# Take the attached parquet testfile ( [^parquet_test.parquet] ) and let nifi
take it in (via GetFile, GetHDFS, ....).
(this testfile was actually produced by an ExecuteSQLRecord processor with
standard parquet writer)
# after that, add for example a ConvertRecord processor with a +standaard
ParquetReader+ and another writer (for exapple Json writer, doesn't really
matter).
(but any other processor that can use a parquet reader is having the same issue)
# run the flow. An error like the one above will be shown.
We still have an older nifi v1 environment too (v1.23.2) and when trying the
exact same flow with the same file, it's working perfectly fine.
I found a similar mention of this problem on the Cloudera community also:
[https://community.cloudera.com/t5/Support-Questions/Nifi-2-7-x-seems-to-break-ParquetRecordReader-for-timestamps/m-p/413341]
The person there confirms that in v2.6.0 it still worked as expected, but not
in 2.7.
Reading parquet data is pretty important, so this is a rather blocking issue.
was:
It seems that since version 2.7 the ParquetReader is having problems with
timestamps in parquet data.
When reading a parquet-file with a timestamp column (timestamp_millis), an
error is thrown:
!image-2026-02-04-12-58-19-172.png!
{noformat}
ConvertRecord[id=59968c39-4fb7-324a-b1b7-bd2e4b93ccb7] Failed to process
FlowFile[filename=parquet_test.parquet]; will route to failure:
java.lang.ClassCastException: class java.time.Instant cannot be cast to class
java.lang.Long (java.time.Instant and java.lang.Long are in module java.base of
loader 'bootstrap')
java.lang.ClassCastException: class java.time.Instant cannot be cast to class
java.lang.Long (java.time.Instant and java.lang.Long are in module java.base of
loader 'bootstrap') at
org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1175) at
org.apache.nifi.avro.AvroTypeUtil.lambda$normalizeValue$3(AvroTypeUtil.java:1186)
at
org.apache.nifi.avro.AvroTypeUtil.convertUnionFieldValue(AvroTypeUtil.java:1016)
at
org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1186) at
org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:979)
at
org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:943)
at
org.apache.nifi.parquet.record.ParquetRecordReader.nextRecord(ParquetRecordReader.java:111)
at
org.apache.nifi.serialization.RecordReader.nextRecord(RecordReader.java:50) at
java.base/jdk.internal.reflect.DirectMethodHandleAccessor.invoke(DirectMethodHandleAccessor.java:103)
at java.base/java.lang.reflect.Method.invoke(Method.java:580) at
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:251)
at
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler$ProxiedReturnObjectInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:237)
at jdk.proxy196/jdk.proxy196.$Proxy501.nextRecord(Unknown Source)
at
org.apache.nifi.processors.standard.AbstractRecordProcessor.lambda$onTrigger$0(AbstractRecordProcessor.java:132)
at
org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3410)
at
org.apache.nifi.processors.standard.AbstractRecordProcessor.onTrigger(AbstractRecordProcessor.java:125)
at
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1274)
at
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:229)
at
org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:59)
at org.apache.nifi.engine.FlowEngine.lambda$wrap$1(FlowEngine.java:105) at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317) at
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1583)
{noformat}
Steps to reproduce:
# Take the attached parquet testfile ( [^parquet_test.parquet] ) and let nifi
take it in (via GetFile, GetHDFS, ....).
(this testfile was actually produced by an ExecuteSQLRecord processor with
standard parquet writer)
# after that, add for example a ConvertRecord processor with a +standaard
ParquetReader+ and another writer (for exapple Json writer, doesn't really
matter).
(but any other processor that can use a parquet reader is having the same issue)
# run the flow. An error like the one above will be shown.
We still have an older nifi v1 environment too (v1.23.2) and when trying the
exact same flow with the same file, it's working perfectly fine.
I found a similar mention of this problem on the Cloudera community also:
[https://community.cloudera.com/t5/Support-Questions/Nifi-2-7-x-seems-to-break-ParquetRecordReader-for-timestamps/m-p/413341]
The person there confirms that in v2.6.0 it still worked as expected, but not
in 2.7.
Reading parquet data is pretty important, so this is a rather blocking issue.
> ParquetReader fails on timestamps
> ---------------------------------
>
> Key: NIFI-15548
> URL: https://issues.apache.org/jira/browse/NIFI-15548
> Project: Apache NiFi
> Issue Type: Bug
> Affects Versions: 2.7.0, 2.7.1, 2.7.2
> Reporter: Sven Van Kerrebroeck
> Priority: Blocker
> Attachments: image-2026-02-04-12-58-19-172.png, parquet_test.parquet
>
>
> It seems that since version 2.7 the ParquetReader is having problems with
> timestamps in parquet data.
> When reading a parquet-file with a timestamp column (timestamp_millis), an
> error is thrown:
> !image-2026-02-04-12-58-19-172.png!
>
> {noformat}
> ConvertRecord[id=59968c39-4fb7-324a-b1b7-bd2e4b93ccb7] Failed to process
> FlowFile[filename=parquet_test.parquet]; will route to failure:
> java.lang.ClassCastException: class java.time.Instant cannot be cast to class
> java.lang.Long (java.time.Instant and java.lang.Long are in module java.base
> of loader 'bootstrap')
> java.lang.ClassCastException: class java.time.Instant cannot be cast to class
> java.lang.Long (java.time.Instant and java.lang.Long are in module java.base
> of loader 'bootstrap') at
> org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1175)
> at
> org.apache.nifi.avro.AvroTypeUtil.lambda$normalizeValue$3(AvroTypeUtil.java:1186)
> at
> org.apache.nifi.avro.AvroTypeUtil.convertUnionFieldValue(AvroTypeUtil.java:1016)
> at
> org.apache.nifi.avro.AvroTypeUtil.normalizeValue(AvroTypeUtil.java:1186)
> at
> org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:979)
> at
> org.apache.nifi.avro.AvroTypeUtil.convertAvroRecordToMap(AvroTypeUtil.java:943)
> at
> org.apache.nifi.parquet.record.ParquetRecordReader.nextRecord(ParquetRecordReader.java:111)
> at
> org.apache.nifi.serialization.RecordReader.nextRecord(RecordReader.java:50)
> at
> java.base/jdk.internal.reflect.DirectMethodHandleAccessor.invoke(DirectMethodHandleAccessor.java:103)
> at java.base/java.lang.reflect.Method.invoke(Method.java:580) at
> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:251)
> at
> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler$ProxiedReturnObjectInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:237)
> at jdk.proxy196/jdk.proxy196.$Proxy501.nextRecord(Unknown Source)
> at
> org.apache.nifi.processors.standard.AbstractRecordProcessor.lambda$onTrigger$0(AbstractRecordProcessor.java:132)
> at
> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3410)
> at
> org.apache.nifi.processors.standard.AbstractRecordProcessor.onTrigger(AbstractRecordProcessor.java:125)
> at
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> at
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1274)
> at
> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:229)
> at
> org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:59)
> at org.apache.nifi.engine.FlowEngine.lambda$wrap$1(FlowEngine.java:105)
> at
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
> at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)
> at
> java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
> at java.base/java.lang.Thread.run(Thread.java:1583)
> {noformat}
> Steps to reproduce:
> # Take the attached parquet testfile ( [^parquet_test.parquet] ) and let
> nifi take it in (via GetFile, GetHDFS, ....).
> (this testfile was actually produced by an ExecuteSQLRecord processor with
> standard parquet writer)
> # after that, add for example a ConvertRecord processor with a +standaard
> ParquetReader+ and another writer (for exapple Json writer, doesn't really
> matter).
> (but any other processor that can use a parquet reader is having the same
> issue)
> # run the flow. An error like the one above will be shown.
> We still have an older nifi v1 environment too (v1.23.2) and when trying the
> exact same flow with the same file, it's working perfectly fine.
> I found a similar mention of this problem on the Cloudera community also:
> [https://community.cloudera.com/t5/Support-Questions/Nifi-2-7-x-seems-to-break-ParquetRecordReader-for-timestamps/m-p/413341]
> The person there confirms that in v2.6.0 it still worked as expected, but not
> in 2.7.
> Reading parquet data is pretty important, so this is a rather blocking issue.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)