twalthr commented on code in PR #23829: URL: https://github.com/apache/flink/pull/23829#discussion_r1409397810
########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/expressions/LiteralExpressionsSerialisationITCase.java: ########## @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.expressions; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.operations.ProjectQueryOperation; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.math.BigDecimal; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.Period; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.Expressions.lit; +import static org.apache.flink.table.api.Expressions.nullOf; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ValueLiteralExpression#asSerializableString()}. */ +@ExtendWith(MiniClusterExtension.class) +public class LiteralExpressionsSerialisationITCase { Review Comment: nit: use American English `Serialization` everywhere ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java: ########## @@ -219,6 +222,83 @@ public String asSummaryString() { return stringifyValue(value); } + @Override + public String asSerializableString() { + if (value == null && !dataType.getLogicalType().is(LogicalTypeRoot.NULL)) { + return String.format( + "CAST(NULL AS %s)", + // casting does not support nullability + dataType.getLogicalType().copy(true).asSerializableString()); + } + final LogicalType logicalType = dataType.getLogicalType(); + switch (logicalType.getTypeRoot()) { + case TINYINT: + return String.format("CAST(%s AS TINYINT)", value); + case SMALLINT: + return String.format("CAST(%s AS SMALLINT)", value); + case BIGINT: + return String.format("CAST(%s AS BIGINT)", value); + case FLOAT: + return String.format("CAST(%s AS FLOAT)", value); + case DOUBLE: + return String.format("CAST(%s AS DOUBLE)", value); + case CHAR: + case VARCHAR: + case DECIMAL: + case INTEGER: + return stringifyValue(value); + case BOOLEAN: + case SYMBOL: + case NULL: + return stringifyValue(value).toUpperCase(Locale.ROOT); + case BINARY: + case VARBINARY: + return String.format("X'%s'", StringUtils.byteToHexString((byte[]) value)); + case DATE: + return String.format("DATE '%s'", getValueAs(LocalDate.class).get()); + case TIME_WITHOUT_TIME_ZONE: + return String.format("TIME '%s'", getValueAs(LocalTime.class).get()); + case TIMESTAMP_WITHOUT_TIME_ZONE: + final LocalDateTime localDateTime = getValueAs(LocalDateTime.class).get(); + return String.format( + "TIMESTAMP '%s %s'", + localDateTime.toLocalDate(), localDateTime.toLocalTime()); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final Instant instant = getValueAs(Instant.class).get(); + return String.format("TO_TIMESTAMP_LTZ(%d, %d)", instant.toEpochMilli(), 3); Review Comment: we can support more than milliseconds here ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java: ########## @@ -219,6 +222,83 @@ public String asSummaryString() { return stringifyValue(value); } + @Override + public String asSerializableString() { + if (value == null && !dataType.getLogicalType().is(LogicalTypeRoot.NULL)) { + return String.format( + "CAST(NULL AS %s)", + // casting does not support nullability + dataType.getLogicalType().copy(true).asSerializableString()); + } + final LogicalType logicalType = dataType.getLogicalType(); + switch (logicalType.getTypeRoot()) { + case TINYINT: + return String.format("CAST(%s AS TINYINT)", value); + case SMALLINT: + return String.format("CAST(%s AS SMALLINT)", value); + case BIGINT: + return String.format("CAST(%s AS BIGINT)", value); + case FLOAT: + return String.format("CAST(%s AS FLOAT)", value); + case DOUBLE: + return String.format("CAST(%s AS DOUBLE)", value); + case CHAR: + case VARCHAR: + case DECIMAL: + case INTEGER: + return stringifyValue(value); Review Comment: in general `stringifyValue` was meant for logging, maybe it makes sense to implement the functionality in this line directly ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java: ########## @@ -219,6 +222,83 @@ public String asSummaryString() { return stringifyValue(value); } + @Override + public String asSerializableString() { + if (value == null && !dataType.getLogicalType().is(LogicalTypeRoot.NULL)) { + return String.format( + "CAST(NULL AS %s)", + // casting does not support nullability + dataType.getLogicalType().copy(true).asSerializableString()); + } + final LogicalType logicalType = dataType.getLogicalType(); + switch (logicalType.getTypeRoot()) { + case TINYINT: + return String.format("CAST(%s AS TINYINT)", value); + case SMALLINT: + return String.format("CAST(%s AS SMALLINT)", value); + case BIGINT: + return String.format("CAST(%s AS BIGINT)", value); + case FLOAT: + return String.format("CAST(%s AS FLOAT)", value); + case DOUBLE: + return String.format("CAST(%s AS DOUBLE)", value); + case CHAR: + case VARCHAR: + case DECIMAL: + case INTEGER: + return stringifyValue(value); + case BOOLEAN: + case SYMBOL: Review Comment: add a test for symbol ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java: ########## @@ -219,6 +222,83 @@ public String asSummaryString() { return stringifyValue(value); } + @Override + public String asSerializableString() { + if (value == null && !dataType.getLogicalType().is(LogicalTypeRoot.NULL)) { + return String.format( + "CAST(NULL AS %s)", + // casting does not support nullability + dataType.getLogicalType().copy(true).asSerializableString()); + } + final LogicalType logicalType = dataType.getLogicalType(); + switch (logicalType.getTypeRoot()) { + case TINYINT: + return String.format("CAST(%s AS TINYINT)", value); + case SMALLINT: + return String.format("CAST(%s AS SMALLINT)", value); + case BIGINT: + return String.format("CAST(%s AS BIGINT)", value); + case FLOAT: + return String.format("CAST(%s AS FLOAT)", value); + case DOUBLE: + return String.format("CAST(%s AS DOUBLE)", value); + case CHAR: + case VARCHAR: + case DECIMAL: + case INTEGER: + return stringifyValue(value); Review Comment: `stringifyValue` might not be correct for DECIMAL if there is a scale and precision defined. -- 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]
