[ https://issues.apache.org/jira/browse/FLINK-947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14322816#comment-14322816 ]
ASF GitHub Bot commented on FLINK-947: -------------------------------------- GitHub user aljoscha opened a pull request: https://github.com/apache/flink/pull/405 [FLINK-947] Add a declarative expression API This one is quite big. So you should check out the documentation, skaldic, examples and test cases to see how the API works. You can merge this pull request into a Git repository by running: $ git pull https://github.com/aljoscha/flink linq Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/405.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 #405 ---- commit 147525ced43db6690a64fbae1395dbd258b8901d Author: Aljoscha Krettek <aljoscha.kret...@gmail.com> Date: 2014-10-03T16:25:15Z Change translateToDataflow to return Operator Before, translateToDataflow of SingleInputOperator could only return a single input operator of the lower layer, same for TwoInputOperator. This change allows translateToDataflow to return more kinds of operators. commit 58b5b9ec6e65855bfd71287deb6352dfc4498451 Author: Aljoscha Krettek <aljoscha.kret...@gmail.com> Date: 2014-10-23T16:09:38Z Add methods to CompositeType for querying field types and names commit ac29ee3ad36a72d7c41549f38da1a00e66d85041 Author: Aljoscha Krettek <aljoscha.kret...@gmail.com> Date: 2014-10-01T11:12:18Z [FLINK-947] Add a declarative expression API ---- > Add support for "Named Datasets" > -------------------------------- > > Key: FLINK-947 > URL: https://issues.apache.org/jira/browse/FLINK-947 > Project: Flink > Issue Type: New Feature > Components: Java API > Reporter: Aljoscha Krettek > Assignee: Aljoscha Krettek > Priority: Minor > > This would create an API that is a mix between SQL like declarativity and the > power of user defined functions. Example user code could look like this: > {code:Java} > NamedDataSet one = ... > NamedDataSet two = ... > NamedDataSet result = one.join(two).where("key").equalTo("otherKey") > .project("a", "b", "c") > .map( (UserTypeIn in) -> return new UserTypeOut(...) ) > .print(); > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)