You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by aljoscha <gi...@git.apache.org> on 2015/02/16 15:35:17 UTC

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

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 <al...@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 <al...@gmail.com>
Date:   2014-10-23T16:09:38Z

    Add methods to CompositeType for querying field types and names

commit ac29ee3ad36a72d7c41549f38da1a00e66d85041
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2014-10-01T11:12:18Z

    [FLINK-947] Add a declarative expression 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by mbalassi <gi...@git.apache.org>.
Github user mbalassi commented on the pull request:

    https://github.com/apache/flink/pull/405#issuecomment-74579541
  
    Great additions! Really looking to trying it out. :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by mbalassi <gi...@git.apache.org>.
Github user mbalassi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/405#discussion_r24778932
  
    --- Diff: flink-staging/flink-linq/pom.xml ---
    @@ -0,0 +1,227 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +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.
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
    +
    +	<modelVersion>4.0.0</modelVersion>
    +
    +	<parent>
    +		<groupId>org.apache.flink</groupId>
    +		<artifactId>flink-staging</artifactId>
    +		<version>0.9-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-linq</artifactId>
    +	<name>flink-linq</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-scala</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-scala</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-scala-examples</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-reflect</artifactId>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-library</artifactId>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-compiler</artifactId>
    +		</dependency>
    +
    --- End diff --
    
    You transitively depend on the scala stuff through flink-scala, so you could omit these.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/405#discussion_r24862039
  
    --- Diff: flink-staging/flink-linq/src/main/scala/org/apache/flink/api/scala/expressions/package.scala ---
    @@ -0,0 +1,104 @@
    +/*
    + * 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.
    + */
    +package org.apache.flink.api.scala
    +
    +import com.google.common.base.Preconditions
    +import org.apache.flink.api.expressions.{Row, ExpressionOperation}
    +import org.apache.flink.api.common.typeutils.CompositeType
    +import org.apache.flink.streaming.api.scala.DataStream
    +
    +import scala.language.implicitConversions
    +
    +/**
    + * == Language Integrated Queries (aka Expression Operations) ==
    + *
    + * Importing this package with:
    + *
    + * {{{
    + *   import org.apache.flink.api.scala.expressions._
    + * }}}
    + *
    + * imports implicit conversions for converting a [[DataSet]] or [[DataStream]] to an
    + * [[ExpressionOperation]]. This can be used to perform SQL-like queries on data. Please have
    + * a look at [[ExpressionOperation]] to see which operations are supported and
    + * [[org.apache.flink.api.scala.expressions.ImplicitExpressionOperations]] to see how an
    + * expression can be specified.
    + *
    + * Inside an expression operation you can use Scala Symbols to refer to field names. One would
    + * refer to field `a` by writing `'a`. Sometimes it is necessary to manually confert a
    + * Scala literal to an Expression Literal, in those cases use `Literal`, as in `Literal(3)`.
    + *
    + * Example:
    + *
    + * {{{
    + *   import org.apache.flink.api.scala._
    + *   import org.apache.flink.api.scala.expressions._
    + *
    + *   val env = ExecutionEnvironment.getExecutionEnvironment
    + *   val input = env.fromElements(("Hello", 2), ("Hello", 5), ("Ciao", 3))
    + *   val result = input.as('word, 'count).groupBy('word).select('word, 'count.avg)
    + *   result.print()
    + *
    + *   env.execute()
    + * }}}
    + *
    + * The result of an [[ExpressionOperation]] can be converted back to the underlying API
    + * representation using `as`:
    + *
    + * {{{
    + *   case class Word(word: String, count: Int)
    + *
    + *   val result = in.select(...).as('word, 'count)
    + *   val set = result.as[Word]
    + * }}}
    + */
    +package object expressions extends ImplicitExpressionConversions {
    +
    +  implicit def dataSet2DataSetConversions[T](set: DataSet[T]): DataSetConversions[T] = {
    +    Preconditions.checkArgument(set.getType.isInstanceOf[CompositeType[T]])
    --- End diff --
    
    I think we need a good error message here, telling the user why the type cannot be used with the expr lang.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the pull request:

    https://github.com/apache/flink/pull/405#issuecomment-74842606
  
    Yeah, I'm not sure about linq as well. I like the name but realise that it might be problematic. What do the others think. I could call it "flink-expressions".
    
    I will add documentation about which types are supported and a good error message for unsupported types as @rmetzger mentioned.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/405#discussion_r24857326
  
    --- Diff: flink-staging/flink-linq/pom.xml ---
    @@ -0,0 +1,227 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +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.
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
    +
    +	<modelVersion>4.0.0</modelVersion>
    +
    +	<parent>
    +		<groupId>org.apache.flink</groupId>
    +		<artifactId>flink-staging</artifactId>
    +		<version>0.9-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-linq</artifactId>
    +	<name>flink-linq</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-scala</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-scala</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-scala-examples</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-reflect</artifactId>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-library</artifactId>
    +		</dependency>
    +
    +		<dependency>
    +			<groupId>org.scala-lang</groupId>
    +			<artifactId>scala-compiler</artifactId>
    +		</dependency>
    +
    --- End diff --
    
    I think its really not an issue to directly add your dependencies to the pom.
    Imagine we change something in the `flink-scala` module.
    I was actually thinking about adding a check to maven that every dependency has to be added directly. I'm pretty sure there are cases in the project where we use stuff Apache Commons libraries which come from external dependencies.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/405#issuecomment-74773124
  
    Very very nice work! I've played around a bit with it and the first impression is very good.
    
    +1 to merge the pull request. The change is very big, but stable enough to be merged to master. Not merging it soon would probably cause a lot of work on @aljoscha side.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the pull request:

    https://github.com/apache/flink/pull/405#issuecomment-74845397
  
    To me, `flink-expressions` sounds much better than `linq` and it mitigates the risk of law suites :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/405#discussion_r24858527
  
    --- Diff: docs/linq.md ---
    @@ -0,0 +1,65 @@
    +---
    +title: "Language-Integrated Queries"
    +---
    +<!--
    +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.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +**Language-Integrated Queries are an experimental feature and can currently only be used with
    --- End diff --
    
    Good to see some documentation as well!
    
    Which types are supported by the expression API? Only scala case-classes? POJOs ? Even more? 
    Would be could if you could add that to the documentation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-947] Add a declarative expression API

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/405#issuecomment-74832989
  
    One more thing, the maven module is called "flink-linq". Are we certain that we can use the name LINQ without problems here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---