GitHub user rmetzger opened a pull request:
https://github.com/apache/incubator-flink/pull/141
[FLINK-1032] Rework support for POJO types in the Java API
With this change, users can use string-based expressions to select keys on
pojos and tuples.
POJOs (=plain old java object) are bean-style java classes that have public
getter/setter for each non-public field and a constructor without arguments.
This change allows in particular to select nested pojos.
To illustrate it, imagine you have the following POJO
```java
public static class PojoContainingTupleAndWritable {
public int someInt;
public String someString;
public IntWritable hadoopFan;
public Tuple2<Long, Long> theTuple;
public FromTuple fromTuple;
public PojoContainingTupleAndWritable() {}
public PojoContainingTupleAndWritable(int i, long l1, long l2) {
hadoopFan = new IntWritable(i);
someInt = i;
theTuple = new Tuple2<Long, Long>(l1, l2);
}
}
// nested element
public static class FromTuple extends Tuple3<String, String, Long> {
private static final long serialVersionUID = 1L;
public int special;
}
```
You can now use these kinds of string-based expressions:
```java
// group by "hadoopFan"
DataSet<Integer> reduceDs = ds.groupBy("hadoopFan")
// include all fields of the Tuple2
DataSet<Integer> reduceDs = ds.groupBy("hadoopFan", "theTuple.*")
// use full pojo
DataSet<Integer> reduceDs = ds.groupBy("*")
// nested pojo access
DataSet<Integer> reduceDs = ds.groupBy("fromTuple.special")
```
Note that `FromTuple` is handled as a pojo, not a tuple.
The pull request contains changes from @aljoscha that I've included into my
own commit.
There are the following things missing:
- Documentation
- One TypeExtractor test case is disabled
- the partition operator has not support for the expressions yet. Ideally
its just an issue of changing the interfaces and adding a test
- Somehow, operator chaining (mapper and combiner) does not work with
pojos. I'll look into this.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/rmetzger/incubator-flink pojo-final
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/incubator-flink/pull/141.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 #141
----
commit c56e11bc82c4baebdd205b96b4500269a787b58c
Author: Robert Metzger <[email protected]>
Date: 2014-08-11T11:32:45Z
[FLINK-1032] Rework support for POJO types in the Java API
----
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---