[
https://issues.apache.org/jira/browse/FLINK-4793?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15565439#comment-15565439
]
ASF GitHub Bot commented on FLINK-4793:
---------------------------------------
GitHub user twalthr opened a pull request:
https://github.com/apache/flink/pull/2621
[FLINK-4793] [types] Improve lambda constructor reference handling
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 further improves the handling of Java 8 lambdas. It improves the
handling of constructor references. The following is now possible:
`env.fromElements("xx", "xx").map(Trade::new).map(Trade::toString).print();`
Unfortunately, I had to add the ASM dependency to `flink-core`. I hope this
is not a problem.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/twalthr/flink FLINK-4793
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/2621.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 #2621
----
commit 0e85f7273dab446f71d832bf453d3dcc96c41599
Author: twalthr <[email protected]>
Date: 2016-10-11T13:33:20Z
[FLINK-4793] [types] Improve lambda constructor reference handling
----
> Using a local method with :: notation in Java 8 causes index out of bounds
> --------------------------------------------------------------------------
>
> Key: FLINK-4793
> URL: https://issues.apache.org/jira/browse/FLINK-4793
> Project: Flink
> Issue Type: Bug
> Reporter: Ted Dunning
> Assignee: Timo Walther
>
> I tried to use the toString method on an object as a map function:
> {code}
> .<String>map(Trade::toString)
> {code}
> This caused an index out of bounds error:
> {code}
> java.lang.ArrayIndexOutOfBoundsException: -1
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:351)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:305)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.getMapReturnTypes(TypeExtractor.java:120)
> at
> org.apache.flink.streaming.api.datastream.DataStream.map(DataStream.java:506)
> at
> com.mapr.aggregate.AggregateTest.testAggregateTrades(AggregateTest.java:81)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
> at
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> at
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
> at
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
> at
> com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:68)
> {code}
> On the other hand, if I use a public static method, like this:
> {code}
> .<Trade>map(Trade::fromString)
> {code}
> All is good. fromString and toString are defined like this:
> {code}
> public static Trade fromString(String s) throws IOException {
> return mapper.readValue(s, Trade.class);
> }
> @Override
> public String toString() {
> return String.format("{\"%s\", %d, %d, %.2f}", symbol, time, volume,
> price);
> }
> {code}
> This might be a viable restriction on what functions I can use, but there
> certainly should be a better error message, if so.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)