twalthr commented on code in PR #27757:
URL: https://github.com/apache/flink/pull/27757#discussion_r2939435650
##########
docs/data/sql_functions.yml:
##########
@@ -687,7 +687,7 @@ temporal:
description: Converts a date string string1 with format string2 (by
default 'yyyy-MM-dd') to a date.
- sql: TO_TIMESTAMP_LTZ(numeric[, precision])
table: toTimestampLtz(NUMERIC, PRECISION)
- description: Converts an epoch seconds or epoch milliseconds to a
TIMESTAMP_LTZ, the valid precision is 0 or 3, the 0 represents
TO_TIMESTAMP_LTZ(epochSeconds, 0), the 3 represents
TO_TIMESTAMP_LTZ(epochMilliseconds, 3). If no precision is provided, the
default precision is 3. If any input is null, the function will return null.
+ description: Converts a numeric epoch value to a TIMESTAMP_LTZ. The
precision parameter (0-9) specifies the unit of the epoch value, where 0
represents seconds, 3 represents milliseconds, 6 represents microseconds, and 9
represents nanoseconds. Other values between 0 and 9 are also supported,
representing 10^(-precision) seconds. If no precision is provided, the default
precision is 3 (milliseconds). If any input is null, the function will return
null.
Review Comment:
I actually preferred some parts of the previous description.
`TO_TIMESTAMP_LTZ(epochSeconds, 0), the 3 represents
TO_TIMESTAMP_LTZ(epochMilliseconds, 3)`. So depending on precision parameter
the meaning of a input long value changes, correct?
##########
docs/data/sql_functions.yml:
##########
@@ -687,7 +687,7 @@ temporal:
description: Converts a date string string1 with format string2 (by
default 'yyyy-MM-dd') to a date.
- sql: TO_TIMESTAMP_LTZ(numeric[, precision])
table: toTimestampLtz(NUMERIC, PRECISION)
- description: Converts an epoch seconds or epoch milliseconds to a
TIMESTAMP_LTZ, the valid precision is 0 or 3, the 0 represents
TO_TIMESTAMP_LTZ(epochSeconds, 0), the 3 represents
TO_TIMESTAMP_LTZ(epochMilliseconds, 3). If no precision is provided, the
default precision is 3. If any input is null, the function will return null.
+ description: Converts a numeric epoch value to a TIMESTAMP_LTZ. The
precision parameter (0-9) specifies the unit of the epoch value, where 0
represents seconds, 3 represents milliseconds, 6 represents microseconds, and 9
represents nanoseconds. Other values between 0 and 9 are also supported,
representing 10^(-precision) seconds. If no precision is provided, the default
precision is 3 (milliseconds). If any input is null, the function will return
null.
- sql: TO_TIMESTAMP_LTZ(string1[, string2[, string3]])
table: toTimestampLtz(STRING1[, STRING2[, STRING3]])
description: Converts a timestamp string string1 with format string2 (by
default 'yyyy-MM-dd HH:mm:ss.SSS') in time zone string3 (by default 'UTC') to a
TIMESTAMP_LTZ. If any input is null, the function will return null.
Review Comment:
Do we also support `yyyy-MM-dd HH:mm:ss.SSSSSSSSS` now? Does this doc need
an update as well?
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java:
##########
@@ -283,11 +285,10 @@ public String asSerializableString(SqlFactory sqlFactory)
{
localDateTime.toLocalTime().format(DateTimeFormatter.ISO_LOCAL_TIME));
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
final Instant instant = getValueAs(Instant.class).get();
- if (instant.getNano() % 1_000_000 != 0) {
- throw new TableException(
- "Maximum precision for
TIMESTAMP_WITH_LOCAL_TIME_ZONE literals is '3'");
- }
- return String.format("TO_TIMESTAMP_LTZ(%d, %d)",
instant.toEpochMilli(), 3);
+ final int precision =
+ ((LocalZonedTimestampType)
dataType.getLogicalType()).getPrecision();
+ long epochValue = DateTimeUtils.toEpochValue(instant,
precision);
+ return String.format("TO_TIMESTAMP_LTZ(%d, %d)", epochValue,
precision);
Review Comment:
This is not correct, it would limit us to `April 11, 2262`. We need to use
the string-based variant `yyyy-MM-dd HH:mm:ss.SSSSSSSSS` here to represent the
full precision.
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/TimeFunctionsITCase.java:
##########
@@ -850,12 +850,17 @@ private Stream<TestSetSpec> toTimestampLtzTestCases() {
LocalDateTime.of(1969, 12, 31, 23, 58, 20)
.atZone(ZoneOffset.UTC)
.toInstant(),
- TIMESTAMP_LTZ(3).nullable())
+ TIMESTAMP_LTZ(0).nullable())
Review Comment:
I guess this is a breaking change for existing pipelines?
##########
docs/data/sql_functions.yml:
##########
@@ -687,7 +687,7 @@ temporal:
description: Converts a date string string1 with format string2 (by
default 'yyyy-MM-dd') to a date.
- sql: TO_TIMESTAMP_LTZ(numeric[, precision])
table: toTimestampLtz(NUMERIC, PRECISION)
- description: Converts an epoch seconds or epoch milliseconds to a
TIMESTAMP_LTZ, the valid precision is 0 or 3, the 0 represents
TO_TIMESTAMP_LTZ(epochSeconds, 0), the 3 represents
TO_TIMESTAMP_LTZ(epochMilliseconds, 3). If no precision is provided, the
default precision is 3. If any input is null, the function will return null.
+ description: Converts a numeric epoch value to a TIMESTAMP_LTZ. The
precision parameter (0-9) specifies the unit of the epoch value, where 0
represents seconds, 3 represents milliseconds, 6 represents microseconds, and 9
represents nanoseconds. Other values between 0 and 9 are also supported,
representing 10^(-precision) seconds. If no precision is provided, the default
precision is 3 (milliseconds). If any input is null, the function will return
null.
Review Comment:
Maybe an example would also be beneficial here.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]