Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id AA0EB200BC7 for ; Thu, 20 Oct 2016 19:37:00 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A8FD6160AE0; Thu, 20 Oct 2016 17:37:00 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 101CC160AFF for ; Thu, 20 Oct 2016 19:36:59 +0200 (CEST) Received: (qmail 26192 invoked by uid 500); 20 Oct 2016 17:36:58 -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 25830 invoked by uid 99); 20 Oct 2016 17:36:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 20 Oct 2016 17:36:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 710902C2A67 for ; Thu, 20 Oct 2016 17:36:58 +0000 (UTC) Date: Thu, 20 Oct 2016 17:36:58 +0000 (UTC) From: "Martin Junghanns (JIRA)" To: dev@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (FLINK-4872) Type erasure problem exclusively on cluster execution MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 20 Oct 2016 17:37:00 -0000 Martin Junghanns created FLINK-4872: --------------------------------------- Summary: Type erasure problem exclusively on cluster execution Key: FLINK-4872 URL: https://issues.apache.org/jira/browse/FLINK-4872 Project: Flink Issue Type: Bug Components: Core Affects Versions: 1.1.2 Reporter: Martin Junghanns The following codes runs fine on local and collection execution environment but fails when executed on a cluster. {code:title=Problem.java} import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; import java.lang.reflect.Array; public class Problem { public static class Pojo { } public static class Foo extends Tuple1 { } public static class Bar extends Tuple1 { } public static class UDF implements MapFunction, Bar> { private final Class clazz; public UDF(Class clazz) { this.clazz = clazz; } @Override public Bar map(Foo value) throws Exception { Bar bar = new Bar<>(); //noinspection unchecked bar.f0 = (T[]) Array.newInstance(clazz, 10); return bar; } } public static void main(String[] args) throws Exception { // runs in local, collection and cluster execution withLong(); // runs in local and collection execution, fails on cluster execution withPojo(); } public static void withLong() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Foo foo = new Foo<>(); foo.f0 = 42L; DataSet> barDataSource = env.fromElements(foo); DataSet> map = barDataSource.map(new UDF<>(Long.class)); map.print(); } public static void withPojo() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Foo foo = new Foo<>(); foo.f0 = new Pojo(); DataSet> barDataSource = env.fromElements(foo); DataSet> map = barDataSource.map(new UDF<>(Pojo.class)); map.print(); } } {code} {code:title=ProblemTest.java} import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @RunWith(Parameterized.class) public class ProblemTest extends MultipleProgramsTestBase { public ProblemTest(TestExecutionMode mode) { super(mode); } @Test public void testWithLong() throws Exception { Problem.withLong(); } @Test public void testWithPOJO() throws Exception { Problem.withPojo(); } } {code} Exception: {code} The return type of function 'withPojo(Problem.java:58)' could not be determined automatically, due to type erasure. You can give type information hints by using the returns(...) method on the result of the transformation call, or by letting your function implement the 'ResultTypeQueryable' interface. org.apache.flink.api.java.DataSet.getType(DataSet.java:178) org.apache.flink.api.java.DataSet.collect(DataSet.java:407) org.apache.flink.api.java.DataSet.print(DataSet.java:1605) Problem.withPojo(Problem.java:60) Problem.main(Problem.java:38) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)