[ 
https://issues.apache.org/jira/browse/FLINK-24790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sergey Nuyanzin updated FLINK-24790:
------------------------------------
    Affects Version/s: 1.13.3
                       1.14.0

> DATE_FORMAT: some formats are handled in a wrong way or fail
> ------------------------------------------------------------
>
>                 Key: FLINK-24790
>                 URL: https://issues.apache.org/jira/browse/FLINK-24790
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / API
>    Affects Versions: 1.14.0, 1.13.3
>            Reporter: Sergey Nuyanzin
>            Priority: Major
>
> In documentation it is mentioned that 
> {quote}
> Converts timestamp to a value of string in the format specified by the date 
> format string. The format string is compatible with Java's SimpleDateFormat. 
> {quote}
> At the same time 
> {code:sql}
> select date_format(current_timestamp, 'YYYY-''W''ww-u');
> {code}
> gives
> {noformat}
> 2021-W45-2021
> {noformat}
> while in javadoc for SimpleDateFormat it is mentioned:
> {quote}
>  *         <td><code>u</code>
>  *         <td>Day number of week (1 = Monday, ..., 7 = Sunday)
>  *         <td><a href="#number">Number</a>
>  *         <td><code>1</code>
> {quote}
> also any dateformat with timezone fails 
> e.g.
> {code:sql}
> select date_format(current_timestamp, 'yyyy.MM.dd G ''at'' HH:mm:ss z');
> {code}
> {noformat}
> [ERROR] Could not execute SQL statement. Reason:
> java.time.DateTimeException: Unable to extract ZoneId from temporal 
> 2021-11-05T10:28:44.663
>       at 
> java.base/java.time.format.DateTimePrintContext.getValue(DateTimePrintContext.java:289)
>       at 
> java.base/java.time.format.DateTimeFormatterBuilder$ZoneTextPrinterParser.format(DateTimeFormatterBuilder.java:4072)
>       at 
> java.base/java.time.format.DateTimeFormatterBuilder$CompositePrinterParser.format(DateTimeFormatterBuilder.java:2341)
>       at 
> java.base/java.time.format.DateTimeFormatter.formatTo(DateTimeFormatter.java:1843)
>       at 
> java.base/java.time.format.DateTimeFormatter.format(DateTimeFormatter.java:1817)
>       at java.base/java.time.LocalDateTime.format(LocalDateTime.java:1752)
>       at 
> org.apache.flink.table.utils.DateTimeUtils.dateFormat(DateTimeUtils.java:475)
>       at 
> org.apache.flink.table.utils.DateTimeUtils.dateFormat(DateTimeUtils.java:483)
>       at StreamExecCalc$27.processElement(Unknown Source)
>       at 
> org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
>       at 
> org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
>       at 
> org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
>       at 
> org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
>       at 
> org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103)
>       at 
> org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:92)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67)
>       at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:330)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to