[ 
https://issues.apache.org/jira/browse/FLINK-4872?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15673716#comment-15673716
 ] 

ASF GitHub Bot commented on FLINK-4872:
---------------------------------------

GitHub user twalthr opened a pull request:

    https://github.com/apache/flink/pull/2823

    [FLINK-4872] [types] Type erasure problem exclusively on cluster execution

    Thanks for contributing to Apache Flink. Before you open your pull request, 
please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your 
pull request. For more information and/or questions please refer to the [How To 
Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful 
description of your changes.
    
    - [x] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira 
title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the 
JIRA id)
    
    - [x] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis 
build has passed
    
    This PR solves a classloader issue in the TypeExtractor that only happens 
when using object arrays on a cluster. A test for this change would be 
difficult but in general this feature is already tested in the existing tests.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/twalthr/flink FLINK-4872

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2823.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2823
    
----
commit f8b17aaa1b79e8ffe9428e3d57b44cbae002bfdd
Author: twalthr <[email protected]>
Date:   2016-11-17T13:17:23Z

    [FLINK-4872] [types] Type erasure problem exclusively on cluster execution

----


> 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<T> extends Tuple1<T> {
>   }
>   public static class Bar<T> extends Tuple1<T[]> {
>   }
>   public static class UDF<T> implements MapFunction<Foo<T>, Bar<T>> {
>     private final Class<T> clazz;
>     public UDF(Class<T> clazz) {
>       this.clazz = clazz;
>     }
>     @Override
>     public Bar<T> map(Foo<T> value) throws Exception {
>       Bar<T> 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<Long> foo = new Foo<>();
>     foo.f0 = 42L;
>     DataSet<Foo<Long>> barDataSource = env.fromElements(foo);
>     DataSet<Bar<Long>> map = barDataSource.map(new UDF<>(Long.class));
>     map.print();
>   }
>   public static void withPojo() throws Exception {
>     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
>     Foo<Pojo> foo = new Foo<>();
>     foo.f0 = new Pojo();
>     DataSet<Foo<Pojo>> barDataSource = env.fromElements(foo);
>     DataSet<Bar<Pojo>> 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)

Reply via email to