You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "aloyszhang (JIRA)" <ji...@apache.org> on 2019/06/20 09:26:00 UTC

[jira] [Comment Edited] (FLINK-12848) Method equals() in RowTypeInfo should consider fieldsNames

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

aloyszhang edited comment on FLINK-12848 at 6/20/19 9:25 AM:
-------------------------------------------------------------

Hi Enrico, 

Simple add the fieldNames to the equals method of RowTypeInfo is not safe. It will cause test failed in `ExternalCatalogInsertTest` because of some operator like `union` use equals in RowTypeInfo to determine whether the two input are the of same type.  So I did not find a way to meet both tableEnv.scan() and union operator.

-And more , this problem does not appear in flink-1.9.-

Actually, this problem also appeared in flink 1.9, test case like this : 

{{{code:title=Test.java|borderStyle=solid}}}

@Test
 public void  test001(){
  String [] fields = new String []\{"first", "second"};
  TypeInformation<?>[] types = new TypeInformation[]{
   Types.ROW_NAMED(new String[]\{"a0001"}, Types.INT),
   Types.ROW_NAMED(new String[]\{"b0002"}, Types.INT)
  };
  for(TypeInformation type: types){
   System.out.println(type);
  }
  //build flink program
  StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
  execEnv.setParallelism(1);
  StreamTableEnvironment env = StreamTableEnvironment.create(execEnv);

  SimpleProcessionTimeSource streamTableSource = new SimpleProcessionTimeSource(fields, types);
  env.registerTableSource("testSource", streamTableSource);
  Table sourceTable = env.scan("testSource");
  System.out.println("Source table schema : ");
  sourceTable.printSchema();

  MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink();
  env.registerTableSink("targetTable", sink.configure(fields, types));

  sourceTable.select("first,second").insertInto("targetTable");

  try {
   execEnv.execute();
  } catch (Exception e) {
   e.printStackTrace();
  }
 }

{code}

 


was (Author: aloyszhang):
Hi Enrico, 

Simple add the fieldNames to the equals method of RowTypeInfo is not safe. It will cause test failed in `ExternalCatalogInsertTest` because of some operator like `union` use equals in RowTypeInfo to determine whether the two input are the of same type.  So I did not find a way to meet both tableEnv.scan() and union operator.

And more , this problem does not appear in flink-1.9.

 

> Method equals() in RowTypeInfo should consider fieldsNames
> ----------------------------------------------------------
>
>                 Key: FLINK-12848
>                 URL: https://issues.apache.org/jira/browse/FLINK-12848
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.7.2
>            Reporter: aloyszhang
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.9.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Since the `RowTypeInfo#equals()` does not consider the fieldNames , when process data with RowTypeInfo type there may comes an error of the field name.  
> {code:java}
> String [] fields = new String []{"first", "second"};
> TypeInformation<?>[] types = new TypeInformation[]{
> Types.ROW_NAMED(new String[]{"first001"}, Types.INT),
> Types.ROW_NAMED(new String[]{"second002"}, Types.INT) }; StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
> StreamTableEnvironment env = StreamTableEnvironment.getTableEnvironment(execEnv);
> SimpleProcessionTimeSource streamTableSource = new SimpleProcessionTimeSource(fields, types);
> env.registerTableSource("testSource", streamTableSource);
> Table sourceTable = env.scan("testSource");
> System.out.println("Source table schema : ");
> sourceTable.printSchema();
> {code}
> The table shcema will be 
> {code:java}
> Source table schema : 
> root 
> |-- first: Row(first001: Integer) 
> |-- second: Row(first001: Integer) 
> |-- timestamp: TimeIndicatorTypeInfo(proctime)
> {code}
> the second field has the same name with the first field.
> So, we should consider the fieldnames in RowTypeInfo#equals()
>  
>  
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)