[
https://issues.apache.org/jira/browse/FLINK-28275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17563049#comment-17563049
]
Yao Zhang commented on FLINK-28275:
-----------------------------------
Hi all,
I tested in both Flink 1.14.3 and 1.15.0. It came out that the problem only
exists with Flink 1.15.x.
I investigated the code in
org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java::init(),
and found:
{code:java}
this.columnConverters[index] =
row -> {
if (row.isNullAt(index)) {
return PrintStyle.NULL_VALUE;
}
return castExecutor.cast(getter.getFieldOrNull(row)).toString();
}; {code}
The generated castExecutor here calls
org/apache/flink/table/data/binary/BinaryRawValueData.java::toObject() method,
with the serializer of RawValueDataSerializer.
The generated code for castExecutor is as below:
{code:java}
public final class GeneratedCastExecutor$0 implements
org.apache.flink.table.data.utils.CastExecutor {
private final
org.apache.flink.table.runtime.typeutils.RawValueDataSerializer
typeSerializer$2;
public
GeneratedCastExecutor$0(org.apache.flink.table.runtime.typeutils.RawValueDataSerializer
typeSerializer$2) {
this.typeSerializer$2 = typeSerializer$2;
}
@Override public Object cast(Object _myInputObj) throws
org.apache.flink.table.api.TableException {
org.apache.flink.table.data.binary.BinaryRawValueData _myInput =
((org.apache.flink.table.data.binary.BinaryRawValueData)(_myInputObj));
boolean _myInputIsNull = _myInputObj == null;
boolean isNull$0;
org.apache.flink.table.data.binary.BinaryStringData result$1;
isNull$0 = _myInputIsNull;
if (!isNull$0) {
java.lang.Object deserializedObj$0 =
_myInput.toObject(typeSerializer$2);
if (deserializedObj$0 != null) {
java.lang.String resultString$1;
resultString$1 = deserializedObj$0.toString().toString();
result$1 =
org.apache.flink.table.data.binary.BinaryStringData.fromString(resultString$1);
} else {
result$1 = null;
}
isNull$0 = result$1 == null;
} else {
result$1 =
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
}
return result$1;
}
} {code}
Via debug I found that the contructor arg is a RowValueDataserializer instance
holding a DenseVectorSerializer.
In RowValueDataserializer::deserialize, it does not use the wrapped serializer
to deserialize the object. This is where I think the bug is.
I would like to fix this. Could anyone assign me this ticket please?
> Table.execute().print() prints SqlRawValue
> ------------------------------------------
>
> Key: FLINK-28275
> URL: https://issues.apache.org/jira/browse/FLINK-28275
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / API
> Affects Versions: 1.15.0
> Environment: Flink 1.15
> Flink ML 2.1
> Reporter: Yunfeng Zhou
> Priority: Major
>
> If the following code is executed with Flink 1.15.0 and Flink ML 2.1
>
> {code:java}
> import org.apache.flink.ml.linalg.Vectors;
> import org.apache.flink.streaming.api.datastream.DataStream;
> import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
> import org.apache.flink.table.api.Table;
> import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> import org.apache.flink.types.Row;
> public class SqlRawValueExample {
> public static void main(String[] args) {
> StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
> DataStream<Row> dataStream = env.fromElements(Row.of(Vectors.dense(1.0,
> 2.0)));
> Table table = tEnv.fromDataStream(dataStream);
> table.execute().print();
> }
> }{code}
>
> The following result would be printed out
>
> {code:java}
> +----+--------------------------------+
> | op | f0 |
> +----+--------------------------------+
> | +I | SqlRawValue{?} |
> +----+--------------------------------+{code}
>
> while the expected result is
> {code:java}
> +----+--------------------------------+
> | op | f0 |
> +----+--------------------------------+
> | +I | [1.0, 2.0] |
> +----+--------------------------------+{code}
> This behavior affects the readability of the generated results.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)