[ https://issues.apache.org/jira/browse/FLINK-9444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16531044#comment-16531044 ]
ASF GitHub Bot commented on FLINK-9444: --------------------------------------- Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/6218#discussion_r199731327 --- Diff: flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java --- @@ -123,4 +125,24 @@ public void testNestedRowTypeInfo() { assertEquals("Short", typeInfo.getTypeAt("f1.f0").toString()); } + @Test + public void testSchemaEquals() { + final RowTypeInfo row1 = new RowTypeInfo( + new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"field1", "field2"}); + final RowTypeInfo row2 = new RowTypeInfo( + new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"field1", "field2"}); + assertTrue(row1.schemaEquals(row2)); --- End diff -- This is covered by the test base. But I added another test data entry with different field names. > KafkaAvroTableSource failed to work for map and array fields > ------------------------------------------------------------ > > Key: FLINK-9444 > URL: https://issues.apache.org/jira/browse/FLINK-9444 > Project: Flink > Issue Type: Bug > Components: Kafka Connector, Table API & SQL > Affects Versions: 1.6.0 > Reporter: Jun Zhang > Assignee: Jun Zhang > Priority: Blocker > Labels: patch, pull-request-available > Fix For: 1.6.0 > > Attachments: flink-9444.patch > > > When some Avro schema has map/array fields and the corresponding TableSchema > declares *MapTypeInfo/ListTypeInfo* for these fields, an exception will be > thrown when registering the *KafkaAvroTableSource*, complaining like: > Exception in thread "main" org.apache.flink.table.api.ValidationException: > Type Map<String, Integer> of table field 'event' does not match with type > GenericType<java.util.Map> of the field 'event' of the TableSource return > type. > at org.apache.flink.table.api.ValidationException$.apply(exceptions.scala:74) > at > org.apache.flink.table.sources.TableSourceUtil$$anonfun$validateTableSource$1.apply(TableSourceUtil.scala:92) > at > org.apache.flink.table.sources.TableSourceUtil$$anonfun$validateTableSource$1.apply(TableSourceUtil.scala:71) > at > scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) > at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) > at > org.apache.flink.table.sources.TableSourceUtil$.validateTableSource(TableSourceUtil.scala:71) > at > org.apache.flink.table.plan.schema.StreamTableSourceTable.<init>(StreamTableSourceTable.scala:33) > at > org.apache.flink.table.api.StreamTableEnvironment.registerTableSourceInternal(StreamTableEnvironment.scala:124) > at > org.apache.flink.table.api.TableEnvironment.registerTableSource(TableEnvironment.scala:438) -- This message was sent by Atlassian JIRA (v7.6.3#76005)