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

ASF subversion and git services commented on NIFI-13579:
--------------------------------------------------------

Commit cb2117f8e6c8a5bfff52ff43d5d47fb5a725546c in nifi's branch 
refs/heads/main from David Handermann
[ https://gitbox.apache.org/repos/asf?p=nifi.git;h=cb2117f8e6 ]

NIFI-13579 Improved Timestamp Zone Offset Formatting and Parsing

- Improved Timestamp to String formatting to support patterns with zone offsets
- Improved String to Timestamp parsing to support adjusted hours and minutes 
when zone offset is included


> Timezone issue with RecordReader and RecordSetWriter
> ----------------------------------------------------
>
>                 Key: NIFI-13579
>                 URL: https://issues.apache.org/jira/browse/NIFI-13579
>             Project: Apache NiFi
>          Issue Type: Bug
>    Affects Versions: 2.0.0-M4
>            Reporter: Julien G.
>            Assignee: Daniel Stieglitz
>            Priority: Major
>         Attachments: EXAMPLE_1.26.json, EXAMPLE_2_m4.json
>
>          Time Spent: 11h 50m
>  Remaining Estimate: 0h
>
> In NiFi 2 (in NiFi 1.X it works as expected), if you have a timestamp field 
> and you need to use a timestamp format that contains the timezone (Z) to 
> interpret it, as for example with a CSV or a JSON, you run into 2 problems:
> - When reading, the Z is accepted in the format but doesn't seem to be taken 
> into account. Indeed, if we have a timestamp with +0200 (plus 2 hours), on 
> output we have the timestamp on the NiFi timezone but without the time 
> offset. For example, with 15h in +0200 I'd have 15h in +0000 instead of 13h 
> in +0000.
> - When writing, if you want to write the timezone by indicating Z in the 
> format, the processor fails with the following error: 
> {code:java}
> nifi_1          | 2024-07-24 13:45:26,399 ERROR [Timer-Driven Process 
> Thread-8] o.a.n.processors.standard.ValidateRecord 
> ValidateRecord[id=e4dbf4ab-0190-1000-998b-343ac5255b0d] Failed to process 
> StandardFlowFileRecord[uuid=3225694e-
> 1d32-4c4c-8602-7dee663cab0f,claim=StandardContentClaim 
> [resourceClaim=StandardResourceClaim[id=1721827847690-1, container=default, 
> section=1], offset=888, 
> length=235],offset=0,name=3225694e-1d32-4c4c-8602-7dee663cab0f,size=235]; 
> will route to failure
> nifi_1          | java.time.temporal.UnsupportedTemporalTypeException: 
> Unsupported field: OffsetSeconds
> nifi_1          |       at 
> java.base/java.time.LocalDate.get0(LocalDate.java:698)
> nifi_1          |       at 
> java.base/java.time.LocalDate.getLong(LocalDate.java:678)
> nifi_1          |       at 
> java.base/java.time.LocalDateTime.getLong(LocalDateTime.java:718)
> nifi_1          |       at 
> java.base/java.time.format.DateTimePrintContext.getValue(DateTimePrintContext.java:308)
> nifi_1          |       at 
> java.base/java.time.format.DateTimeFormatterBuilder$OffsetIdPrinterParser.format(DateTimeFormatterBuilder.java:3984)
> nifi_1          |       at 
> java.base/java.time.format.DateTimeFormatterBuilder$CompositePrinterParser.format(DateTimeFormatterBuilder.java:2529)
> nifi_1          |       at 
> java.base/java.time.format.DateTimeFormatter.formatTo(DateTimeFormatter.java:1905)
> nifi_1          |       at 
> java.base/java.time.format.DateTimeFormatter.format(DateTimeFormatter.java:1879)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.field.ObjectStringFieldConverter.convertField(ObjectStringFieldConverter.java:62)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.field.ObjectStringFieldConverter.convertField(ObjectStringFieldConverter.java:36)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.util.DataTypeUtils.toString(DataTypeUtils.java:1105)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.util.DataTypeUtils.toString(DataTypeUtils.java:1081)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.MapRecord.convertToString(MapRecord.java:249)
> nifi_1          |       at 
> org.apache.nifi.serialization.record.MapRecord.getAsString(MapRecord.java:236)
> nifi_1          |       at 
> org.apache.nifi.csv.WriteCSVResult.writeRawRecord(WriteCSVResult.java:195)
> nifi_1          |       at 
> java.base/jdk.internal.reflect.DirectMethodHandleAccessor.invoke(DirectMethodHandleAccessor.java:103)
> nifi_1          |       at 
> java.base/java.lang.reflect.Method.invoke(Method.java:580)
> nifi_1          |       at 
> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:254)
> nifi_1          |       at 
> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler$ProxiedReturnObjectInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:240)
> nifi_1          |       at 
> jdk.proxy451/jdk.proxy451.$Proxy611.writeRawRecord(Unknown Source)
> nifi_1          |       at 
> org.apache.nifi.processors.standard.ValidateRecord.writeRecord(ValidateRecord.java:456)
> nifi_1          |       at 
> org.apache.nifi.processors.standard.ValidateRecord.onTrigger(ValidateRecord.java:335)
> nifi_1          |       at 
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> nifi_1          |       at 
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1274)
> nifi_1          |       at 
> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:244)
> nifi_1          |       at 
> org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:59)
> nifi_1          |       at 
> org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
> nifi_1          |       at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
> nifi_1          |       at 
> java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)
> nifi_1          |       at 
> java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
> nifi_1          |       at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
> nifi_1          |       at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
> nifi_1          |       at java.base/java.lang.Thread.run(Thread.java:1583)
> {code}
> You can find the same example in NiFi 1.26 and NiFi 2.0.0-M4 to see the 
> different behavior and the issue in NiFi 2.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to