Return-Path: X-Original-To: apmail-flink-dev-archive@www.apache.org Delivered-To: apmail-flink-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 232B11846C for ; Wed, 2 Mar 2016 14:42:41 +0000 (UTC) Received: (qmail 59983 invoked by uid 500); 2 Mar 2016 14:42:40 -0000 Delivered-To: apmail-flink-dev-archive@flink.apache.org Received: (qmail 59912 invoked by uid 500); 2 Mar 2016 14:42:40 -0000 Mailing-List: contact dev-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 dev@flink.apache.org Received: (qmail 59901 invoked by uid 99); 2 Mar 2016 14:42:40 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Mar 2016 14:42:40 +0000 Received: from [141.23.94.232] (wlan-141-23-94-232.tubit.tu-berlin.de [141.23.94.232]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id 2C2151A003F for ; Wed, 2 Mar 2016 14:42:39 +0000 (UTC) Subject: Re: Input type validation is killing me To: dev@flink.apache.org References: <56D6E464.6090104@apache.org> <56D6F70F.8060408@apache.org> From: Timo Walther Message-ID: <56D6FBE2.8090605@apache.org> Date: Wed, 2 Mar 2016 15:42:42 +0100 User-Agent: Mozilla/5.0 (Windows NT 10.0; WOW64; rv:38.0) Gecko/20100101 Thunderbird/38.6.0 MIME-Version: 1.0 In-Reply-To: Content-Type: text/plain; charset=utf-8; format=flowed Content-Transfer-Encoding: 8bit Can you open an issue with an example of your custom TypeInfo? I will then open a suitable PR for it. On 02.03.2016 15:33, Gyula Fóra wrote: > Would that work with generic classes? > > Timo Walther ezt írta (időpont: 2016. márc. 2., Sze, > 15:22): > >> After thinking about it, I think an even better solution is to provide >> an interface for the TypeExtractor where the user can register mappings >> from class to TypeInformation. >> So that the TypeExctractor is more extensible. This would also solve you >> problem. What do you think? >> >> On 02.03.2016 15:00, Gyula Fóra wrote: >>> Hi! >>> >>> Yes I think, that sounds good :) We just need to make sure that this >> works >>> with things like the TupleTypeInfo which is built-on but I can still mix >> in >>> new Types for the fields. >>> >>> Thanks, >>> Gyula >>> >>> Timo Walther ezt írta (időpont: 2016. márc. 2., >> Sze, >>> 14:02): >>> >>>> The TypeExtractor's input type validation was designed for the built-in >>>> TypeInformation classes. >>>> >>>> In your case of a new, unknown TypeInformation, the validation should >>>> simply skipped, because we can assume that you user knows what he is >> doing. >>>> I can open a PR for that. >>>> >>>> >>>> On 02.03.2016 11:34, Aljoscha Krettek wrote: >>>>> I think you have a point. Another user also just ran into problems with >>>> the TypeExtractor. (The “Java Maps and TypeInformation” email). >>>>> So let’s figure out what needs to be changed to make it work for all >>>> people. >>>>> Cheers, >>>>> Aljoscha >>>>>> On 02 Mar 2016, at 11:15, Gyula Fóra wrote: >>>>>> >>>>>> Hey, >>>>>> >>>>>> I have brought up this issue a couple months back but I would like to >>>> do it >>>>>> again. >>>>>> >>>>>> I think the current way of validating the input type of udfs against >> the >>>>>> out type of the preceeding operators is too aggressive and breaks a >> lot >>>> of >>>>>> code that should otherwise work. >>>>>> >>>>>> This issue appears all the time when I want to use my own >>>>>> TypeInformations<> for operators such as creating my own Tuple >> typeinfos >>>>>> with custom types for the different fields and so. >>>>>> >>>>>> I have a more complex streaming job which would not run if I have the >>>> input >>>>>> type validation. Replacing the Exceptions with logging my Job runs >>>>>> perfectly (making my point) but you can see the errors that would have >>>> been >>>>>> reported as exceptions in the logs: >>>>>> >>>>>> 2016-03-02 11:06:03,447 ERROR >>>>>> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: >>>> Generic >>>>>> object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’. >>>>>> 2016-03-02 11:06:03,450 ERROR >>>>>> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: >>>> Unknown >>>>>> Error. Type is null. >>>>>> 2016-03-02 11:06:03,466 ERROR >>>>>> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: >>>> Basic >>>>>> type expected. >>>>>> 2016-03-02 11:06:03,470 ERROR >>>>>> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: >>>> Basic >>>>>> type expected. >>>>>> >>>>>> Clearly all these errors where not valid in my case as my job runs >>>>>> perfectly. >>>>>> >>>>>> Would it make sense to change the current behaviour or am I just >> abusing >>>>>> the .returns(..) and ResultTypeQueryable interfaces in unintended >> ways. >>>>>> Cheers, >>>>>> Gyula >>