Skip to content

Commit

Permalink
[FLINK-7251] [types] Remove the flink-java8 module and improve lambda…
Browse files Browse the repository at this point in the history
… type extraction

This commit removes the flink-java8 module and merges some tests into flink-core/flink-runtime. It ensures to have the possibility for passing explicit type information in DataStream API as a fallback. Since the tycho compiler approach was very hacky and seems not to work anymore, this commit also removes all references in the docs and quickstarts.

This closes apache#6120.
  • Loading branch information
twalthr committed Jul 20, 2018
1 parent 95eadfe commit ddba1b6
Show file tree
Hide file tree
Showing 52 changed files with 782 additions and 2,392 deletions.
198 changes: 0 additions & 198 deletions docs/dev/java8.md

This file was deleted.

138 changes: 138 additions & 0 deletions docs/dev/java_lambdas.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
---
title: "Java Lambda Expressions"
nav-parent_id: api-concepts
nav-pos: 20
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->

Java 8 introduced several new language features designed for faster and clearer coding. With the most important feature,
the so-called "Lambda Expressions", it opened the door to functional programming. Lambda expressions allow for implementing and
passing functions in a straightforward way without having to declare additional (anonymous) classes.

<span class="label label-danger">Attention</span> Flink supports the usage of lambda expressions for all operators of the Java API, however, whenever a lambda expression uses Java generics you need to declare type information *explicitly*.

This document shows how to use lambda expressions and describes current limitations. For a general introduction to the
Flink API, please refer to the [Programming Guide]({{ site.baseurl }}/dev/api_concepts.html)

### Examples and Limitations

The following example illustrates how to implement a simple, inline `map()` function that squares its input using a lambda expression.
The types of input `i` and output parameters of the `map()` function need not to be declared as they are inferred by the Java compiler.

{% highlight java %}
env.fromElements(1, 2, 3)
// returns the squared i
.map(i -> i*i)
.print();
{% endhighlight %}

Flink can automatically extract the result type information from the implementation of the method signature `OUT map(IN value)` because `OUT` is not generic but `Integer`.

Unfortunately, functions such as `flatMap()` with a signature `void flatMap(IN value, Collector<OUT> out)` are compiled into `void flatMap(IN value, Collector out)` by the Java compiler. This makes it impossible for Flink to infer the type information for the output type automatically.

Flink will most likely throw an exception similar to the following:

{% highlight plain%}
org.apache.flink.api.common.functions.InvalidTypesException: The generic type parameters of 'Collector' are missing.
In many cases lambda methods don't provide enough information for automatic type extraction when Java generics are involved.
An easy workaround is to use an (anonymous) class instead that implements the 'org.apache.flink.api.common.functions.FlatMapFunction' interface.
Otherwise the type has to be specified explicitly using type information.
{% endhighlight %}

In this case, the type information needs to be *specified explicitly*, otherwise the output will be treated as type `Object` which leads to unefficient serialization.

{% highlight java %}
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.util.Collector;

DataSet<Integer> input = env.fromElements(1, 2, 3);

// collector type must be declared
input.flatMap((Integer number, Collector<String> out) -> {
StringBuilder builder = new StringBuilder();
for(int i = 0; i < number; i++) {
builder.append("a");
out.collect(builder.toString());
}
})
// provide type information explicitly
.returns(Types.STRING)
// prints "a", "a", "aa", "a", "aa", "aaa"
.print();
{% endhighlight %}

Similar problems occur when using a `map()` function with a generic return type. A method signature `Tuple2<Integer, Integer> map(Integer value)` is erasured to `Tuple2 map(Integer value)` in the example below.

{% highlight java %}
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple2;

env.fromElements(1, 2, 3)
.map(i -> Tuple2.of(i, i)) // no information about fields of Tuple2
.print();
{% endhighlight %}

In general, those problems can be solved in multiple ways:

{% highlight java %}
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.tuple.Tuple2;

// use the explicit ".returns(...)"
env.fromElements(1, 2, 3)
.map(i -> Tuple2.of(i, i))
.returns(Types.TUPLE(Types.INT, Types.INT))
.print();

// use a class instead
env.fromElements(1, 2, 3)
.map(new MyTuple2Mapper())
.print();

public static class MyTuple2Mapper extends MapFunction<Integer, Integer> {
@Override
public Tuple2<Integer, Integer> map(Integer i) {
return Tuple2.of(i, i);
}
}

// use an anonymous class instead
env.fromElements(1, 2, 3)
.map(new MapFunction<Integer, Tuple2<Integer, Integer>> {
@Override
public Tuple2<Integer, Integer> map(Integer i) {
return Tuple2.of(i, i);
}
})
.print();

// or in this example use a tuple subclass instead
env.fromElements(1, 2, 3)
.map(i -> new DoubleTuple(i, i))
.print();

public static class DoubleTuple extends Tuple2<Integer, Integer> {
public DoubleTuple(int f0, int f1) {
this.f0 = f0;
this.f1 = f1;
}
}
{% endhighlight %}
Original file line number Diff line number Diff line change
Expand Up @@ -158,21 +158,25 @@ public static LambdaExecutable checkAndExtractLambda(Function function) throws T
/**
* Extracts type from given index from lambda. It supports nested types.
*
* @param baseClass SAM function that the lambda implements
* @param exec lambda function to extract the type from
* @param lambdaTypeArgumentIndices position of type to extract in type hierarchy
* @param paramLen count of total parameters of the lambda (including closure parameters)
* @param baseParametersLen count of lambda interface parameters (without closure parameters)
* @return extracted type
*/
public static Type extractTypeFromLambda(
Class<?> baseClass,
LambdaExecutable exec,
int[] lambdaTypeArgumentIndices,
int paramLen,
int baseParametersLen) {
Type output = exec.getParameterTypes()[paramLen - baseParametersLen + lambdaTypeArgumentIndices[0]];
for (int i = 1; i < lambdaTypeArgumentIndices.length; i++) {
validateLambdaType(baseClass, output);
output = extractTypeArgument(output, lambdaTypeArgumentIndices[i]);
}
validateLambdaType(baseClass, output);
return output;
}

Expand Down Expand Up @@ -328,4 +332,23 @@ public static Class<?> getRawClass(Type t) {
}
return Object.class;
}

/**
* Checks whether the given type has the generic parameters declared in the class definition.
*
* @param t type to be validated
*/
public static void validateLambdaType(Class<?> baseClass, Type t) {
if (!(t instanceof Class)) {
return;
}
final Class<?> clazz = (Class<?>) t;

if (clazz.getTypeParameters().length > 0) {
throw new InvalidTypesException("The generic type parameters of '" + clazz.getSimpleName() + "' are missing. "
+ "In many cases lambda methods don't provide enough information for automatic type extraction when Java generics are involved. "
+ "An easy workaround is to use an (anonymous) class instead that implements the '" + baseClass.getName() + "' interface. "
+ "Otherwise the type has to be specified explicitly using type information.");
}
}
}
Loading

0 comments on commit ddba1b6

Please sign in to comment.