You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "xzw0223 (Jira)" <ji...@apache.org> on 2023/03/03 08:45:00 UTC

[jira] [Commented] (FLINK-31312) EnableObjectReuse cause different behaviors

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

xzw0223 commented on FLINK-31312:
---------------------------------

[~Jiang Xin]  Because enableObjectReuse is enabled, the row generated by source is the same as the row object in map. Because objects are reused, the row constructed by source adds field data by location. Therefore, the specified field data needs to be retrieved by location in map,if enableObjectReuse is not enabled, a row object will be re-copied every time you emit. At this time, the row object will construct its fields according to RowTypeInfo, so there is no problem.

> EnableObjectReuse cause different behaviors
> -------------------------------------------
>
>                 Key: FLINK-31312
>                 URL: https://issues.apache.org/jira/browse/FLINK-31312
>             Project: Flink
>          Issue Type: Bug
>          Components: API / DataStream
>            Reporter: Jiang Xin
>            Priority: Major
>
> I have the following test code which fails with the exception `Accessing a field by name is not supported in position-based field mode`, however, if I remove the `enableObjectReuse`, it works.
> {code:java}
> public static void main(String[] args) throws Exception {
>     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
>     env.setParallelism(1);
>     // The test fails with enableObjectReuse
>     env.getConfig().enableObjectReuse();
>     final SourceFunction<Row> rowGenerator =
>             new SourceFunction<Row>() {
>                 @Override
>                 public final void run(SourceContext<Row> ctx) throws Exception {
>                     Row row = new Row(1);
>                     row.setField(0, "a");
>                     ctx.collect(row);
>                 }
>                 @Override
>                 public void cancel() {}
>             };
>     final RowTypeInfo typeInfo =
>             new RowTypeInfo(new TypeInformation[] {Types.STRING}, new String[] {"col1"});
>     DataStream<Row> dataStream = env.addSource(rowGenerator, typeInfo);
>     DataStream<Row> transformedDataStream =
>             dataStream.map(
>                     (MapFunction<Row, Row>) value -> Row.of(value.getField("col1")), typeInfo);
>     transformedDataStream.addSink(new PrintSinkFunction<>());
>     env.execute("Mini Test");
> } {code}
> The `SourceFunction` generates rows without field names, but the return type info is assigned by `env.addSource(rowGenerator, typeInfo)`.
> With object-reuse enabled, rows would be passed to the MapFunction directly, so the exception raises. While if the object-reuse is disabled,  rows would be reconstructed and given field names when passing to the next operator and the test works well.



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