From issues-return-164472-archive-asf-public=cust-asf.ponee.io@flink.apache.org Sun Apr 29 22:42:10 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id B446D18063A for ; Sun, 29 Apr 2018 22:42:09 +0200 (CEST) Received: (qmail 34933 invoked by uid 500); 29 Apr 2018 20:42:08 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 34924 invoked by uid 99); 29 Apr 2018 20:42:08 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 29 Apr 2018 20:42:08 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 361F31A015B for ; Sun, 29 Apr 2018 20:42:08 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -101.511 X-Spam-Level: X-Spam-Status: No, score=-101.511 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id FHPGvG7sF84r for ; Sun, 29 Apr 2018 20:42:07 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 2D89B5F178 for ; Sun, 29 Apr 2018 20:42:06 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 609F2E012E for ; Sun, 29 Apr 2018 20:42:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 1E2F621292 for ; Sun, 29 Apr 2018 20:42:00 +0000 (UTC) Date: Sun, 29 Apr 2018 20:42:00 +0000 (UTC) From: "Sergey Nuyanzin (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (FLINK-8255) Key expressions on named row types do not work MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-8255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16458179#comment-16458179 ] Sergey Nuyanzin edited comment on FLINK-8255 at 4/29/18 8:41 PM: ----------------------------------------------------------------- A little bit research shows that it is something related to class Hierarchy: RowTypeInfo, TupleTypeInfoBase, TupleTypeInfo. Both RowTypeInfo and TupleTypeInfo are ancent of TupleTypeInfoBase. At the same time in e.g. org.apache.flink.streaming.util.typeutils.FieldAccessorFactory there are some checks with casting {code:java} ... else if (typeInfo.isTupleType()) { TupleTypeInfoBase tupleTypeInfo = (TupleTypeInfoBase) typeInfo; ...{code} As RowTypeInfo and TupleTypeInfo are in parallel hierarchy branches => casting will fail for RowTypeInfo. At the same time it looks like there is nothing special related to TupleTypeInfo => casting to TupleTypeInfoBase is enough. Based on finding usages of FieldAccessorFactory's methods with specified casting there could be added 2 more test-case which are also fails with the similar ClassCastException {code:java} final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); TypeInformation[] types = new TypeInformation[]{Types.INT, Types.INT}; String[] fieldNames = new String[]{"id", "value"}; RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0); groupDs.maxBy(1); {code} and one more almost the same however with .minBy in the last line one of the possible fixes: usage casting to TupleTypeInfoBase rther than to TupleTypeInfo (I'm not sure that changing hierarchy could be an option). such fix for the mentioned 3 cases is available here https://github.com/apache/flink/compare/master...snuyanzin:FLINK-8255_Key_expressions_on_named_row_types_do_not_work at the same time it looks like there still could be issues, e.g. org.apache.flink.api.java.DataSet#minBy org.apache.flink.api.java.DataSet#maxBy org.apache.flink.streaming.util.typeutils.FieldAccessor.RecursiveTupleFieldAccessor#RecursiveTupleFieldAccessor also have such casting however at the moment I do not have any idea about test where it could fail was (Author: sergey nuyanzin): A little bit research shows that it is something related to class Hierarchy: RowTypeInfo, TupleTypeInfoBase, TupleTypeInfo. Both RowTypeInfo and TupleTypeInfo are ancent of TupleTypeInfoBase. At the same time in e.g. org.apache.flink.streaming.util.typeutils.FieldAccessorFactory there are some checks with casting {code:java} ... else if (typeInfo.isTupleType()) { TupleTypeInfoBase tupleTypeInfo = (TupleTypeInfoBase) typeInfo; ...{code} As RowTypeInfo and TupleTypeInfo are in parallel hierarchy branches => casting will fail for RowTypeInfo. At the same time it looks like there is nothing special related to TupleTypeInfo => casting to TupleTypeInfoBase is enough. Based on finding usages of FieldAccessorFactory's methods with specified casting there could be added 2 more test-case which are also fails with the similar ClastCastException {code:java} final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); TypeInformation[] types = new TypeInformation[]{Types.INT, Types.INT}; String[] fieldNames = new String[]{"id", "value"}; RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0); groupDs.maxBy(1); {code} and one more almost the same however with .minBy in the last line one of the possible fixes: usage casting to TupleTypeInfoBase rther than to TupleTypeInfo (I'm not sure that changing hierarchy could be an option). such fix for the mentioned 3 cases is available here https://github.com/apache/flink/compare/master...snuyanzin:FLINK-8255_Key_expressions_on_named_row_types_do_not_work at the same time it looks like there still could be issues, e.g. org.apache.flink.api.java.DataSet#minBy org.apache.flink.api.java.DataSet#maxBy org.apache.flink.streaming.util.typeutils.FieldAccessor.RecursiveTupleFieldAccessor#RecursiveTupleFieldAccessor also have such casting however at the moment I do not have any idea about test where it could fail > Key expressions on named row types do not work > ---------------------------------------------- > > Key: FLINK-8255 > URL: https://issues.apache.org/jira/browse/FLINK-8255 > Project: Flink > Issue Type: Bug > Components: DataSet API, DataStream API > Affects Versions: 1.4.0, 1.5.0 > Reporter: Timo Walther > Priority: Major > > The following program fails with a {{ClassCastException}}. It seems that key expressions and rows are not tested well. We should add more tests for them. > {code} > final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); > TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; > String[] fieldNames = new String[]{"id", "value"}; > RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); > env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo) > .keyBy("id").sum("value").print(); > env.execute("Streaming WordCount"); > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)