You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Rong Rong (JIRA)" <ji...@apache.org> on 2019/07/23 15:33:00 UTC

[jira] [Created] (FLINK-13389) Setting DataStream return type breaks some type conversion between Table and DataStream

Rong Rong created FLINK-13389:
---------------------------------

             Summary: Setting DataStream return type breaks some type conversion between Table and DataStream
                 Key: FLINK-13389
                 URL: https://issues.apache.org/jira/browse/FLINK-13389
             Project: Flink
          Issue Type: Bug
          Components: API / DataStream, Table SQL / API
            Reporter: Rong Rong


When converting between data stream and table, there are situations where only GenericTypeInfo can be successfully applied, but not directly setting the specific RowTypeInfo.
For example the following code doesn't work

{code:java}
		TypeInformation<?>[] types = {
			BasicTypeInfo.INT_TYPE_INFO,
			TimeIndicatorTypeInfo.ROWTIME_INDICATOR(),
			BasicTypeInfo.STRING_TYPE_INFO};
		String[] names = {"a", "b", "c"};
		RowTypeInfo typeInfo = new RowTypeInfo(types, names);
		DataStream<Row> ds = env.fromCollection(data).returns(typeInfo);
		Table sourceTable = tableEnv.fromDataStream(ds, "a,b,c");
		tableEnv.registerTable("MyTableRow", sourceTable);

		DataStream<Row> stream = tableEnv.toAppendStream(sourceTable, Row.class)
			.map(a -> a)
			// this line breaks the conversion, it sets the typeinfo to RowTypeInfo.
			// without this line the output type is GenericTypeInfo(Row)
			.returns(sourceTable.getSchema().toRowType());  
		stream.addSink(new StreamITCase.StringSink<Row>());
		env.execute();
{code}




--
This message was sent by Atlassian JIRA
(v7.6.14#76016)