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 F357817FEA for ; Fri, 6 Feb 2015 13:11:53 +0000 (UTC) Received: (qmail 92682 invoked by uid 500); 6 Feb 2015 13:11:53 -0000 Delivered-To: apmail-flink-dev-archive@flink.apache.org Received: (qmail 92628 invoked by uid 500); 6 Feb 2015 13:11:53 -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 92616 invoked by uid 99); 6 Feb 2015 13:11:53 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 06 Feb 2015 13:11:53 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of ewenstephan@gmail.com designates 209.85.223.169 as permitted sender) Received: from [209.85.223.169] (HELO mail-ie0-f169.google.com) (209.85.223.169) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 06 Feb 2015 13:11:49 +0000 Received: by iebtr6 with SMTP id tr6so1074362ieb.2 for ; Fri, 06 Feb 2015 05:11:28 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:sender:in-reply-to:references:date:message-id:subject :from:to:content-type; bh=1fcg/ly6T5zShzP3E8UI9j6e/LAQjIMkYwrQOsW4bIc=; b=GvzyaY/5mw5h+xB22PGi7TDi2gw/MeAvcUKXRKMQmeESIDn+uRxBCxDgvZ78J6W8vS W86dDKDAKgyGdmDD/hP/dDY0TyWQ5XkjXgjH6o6y0JpBpw5DyqmvjXASKfzvZKaOZISi wb3OYsCkvhlU26T17OhamRD5y72/zElu90NgBn0wKZgImViQ0WfJVJT2ipSGQ0QeTISV w6cm2oqwzfUTG0E+QWbXIEfO9tFI1yrxUaM/mWf7tUaI4GsnKyp3/dZ4dNpYz/mkfG1C JLys3me32Ib0Eyt4lLliVpqZNQD5LNy4tzJJsVT7aqI664OGiO8bHuY0feLB8hefUK/6 BQhQ== MIME-Version: 1.0 X-Received: by 10.107.17.9 with SMTP id z9mr3453295ioi.42.1423228288501; Fri, 06 Feb 2015 05:11:28 -0800 (PST) Sender: ewenstephan@gmail.com Received: by 10.64.5.229 with HTTP; Fri, 6 Feb 2015 05:11:28 -0800 (PST) In-Reply-To: References: <1423222074.54d4a53aa3840@euranova.eu> Date: Fri, 6 Feb 2015 14:11:28 +0100 X-Google-Sender-Auth: QQbgP0bpVf0Zj8OX4NFfm44cADM Message-ID: Subject: Re: Eclipse JDT, Java 8, lambdas From: Stephan Ewen To: "dev@flink.apache.org" Content-Type: multipart/alternative; boundary=001a113ed5e4921eed050e6b283a X-Virus-Checked: Checked by ClamAV on apache.org --001a113ed5e4921eed050e6b283a Content-Type: text/plain; charset=UTF-8 Hey! I just saw that the fix was ported to the Eclipse 4.5 M4 release, which can be downloaded here: https://www.eclipse.org/downloads/index-developer.php I am eager to try that out for myself later. If the fix is really included, this Eclipse version should support lambdas elegantly out of the box. Greetings, Stephan On Fri, Feb 6, 2015 at 2:09 PM, Stephan Ewen wrote: > Hi Tran Nam-Luc! > > Java 8 lambdas are a bit tricky right now, because of Java's generic type > erasure. Flink needs the type information of all Functions before sending > programs to the execution engine. > (for more details, see here: > https://github.com/apache/flink/blob/master/docs/internal_types_serialization.md > ) > > The type analysis is part of how we make program execution very robust, > even on really large data. By not just passing Java Objects and using > ad-hoc serialization frameworks, but > by understanding the types up front and parameterizing the runtime > accordingly, we get efficiency and reliability in the execution. We can > also do a better job in finding errors before the parallel execution. > > The JDT compiler accidentally dropped generic types on lambdas a while > back, but we submitted a patch to reintroduce them (see [1]) and it will be > part of the next major Eclipse release. > > Until then, to use Java 8 lambdas you need to follow these points: > > - If the function returns a simple type (like int, long, String) or any > non-generic class, it should work out of the box. > - If the function returns a generic type (like tuple), you need to use a > type hint to tell the system the return type. Below is an example for that. > > DataSet data = ... > DataSet> = data.map( value -> new Tuple2<>(value, > value * 0.5) ).returns("Tuple2"); > > The type hints are a workaround that will be obsolete once Java compiler > support is available. > > Greetings, > Stephan > > > [1] https://bugs.eclipse.org/bugs/show_bug.cgi?id=449063 > > > > > On Fri, Feb 6, 2015 at 12:27 PM, Nam-Luc Tran > wrote: > >> Hello, >> >> I am trying to use Java 8 lambdas in my project and hit the following >> error: >> >> Exception in thread "main" >> org.apache.flink.api.common.functions.InvalidTypesException: The >> generic type parameters of 'Tuple2' are missing. >> It seems that your compiler has not stored them into the .class >> file. >> Currently, only the Eclipse JDT compiler preserves the type >> information necessary to use the lambdas feature type-safely. >> See the documentation for more information about how to compile jobs >> containing lambda expressions. >> at >> >> org.apache.flink.api.java.typeutils.TypeExtractor.validateLambdaGenericParameter(TypeExtractor.java:779) >> at >> >> org.apache.flink.api.java.typeutils.TypeExtractor.validateLambdaGenericParameters(TypeExtractor.java:765) >> at >> >> org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:135) >> at >> >> org.apache.flink.api.java.typeutils.TypeExtractor.getMapReturnTypes(TypeExtractor.java:78) >> at org.apache.flink.api.java.DataSet.map(DataSet.java:160) >> at eu.euranova.flink.Axa.main(Axa.java:62) >> >> My very simple code is the following: >> >> File directory = new File( >> "PATH TO A DIRECTORY WITH CSV FILES"); >> DataSet set = env.fromElements(new Tuple3(0, 0.0, 0.0)); >> for (File file : directory.listFiles()) { >> int pathID = 0; >> String filePath = "file://" + file.getAbsolutePath(); >> DataSet set2 = >> >> env.readCsvFile(filePath).ignoreFirstLine().includeFields("11").types(Double.class,Double.class); >> DataSet set3 = set2.map(tuple -> new Tuple3(pathID, tuple.f0, >> tuple.f1)); >> set = set.union(set3); >> } >> >> I followed the steps in the Java 8 documentation section >> ( >> http://flink.apache.org/docs/0.8/java8_programming_guide.html#compiler-limitations >> ) >> and have applied the following to the pom.xml file created using the >> flink archetype: >> - Modified java 1.6 reference to 1.8 >> - Uncommented the section related to Java 8 lambdas >> - Installed Eclipse Java developer tools (JDT) >> - Installed m2e-jdt connector >> >> The pom.xml does not show any error and builds fine. >> >> Am I missing something? Do I need to explicity set up Eclipse JDT? The >> only installed environment shown in my preferences is the >> /usr/java/jdk-1.8.0_31 from oracle. >> >> Thanks and best regards, >> >> Tran Nam-Luc >> > > --001a113ed5e4921eed050e6b283a--