flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-1159) Case style anonymous functions not supported by Scala API
Date Fri, 25 Mar 2016 15:07:25 GMT

    [ https://issues.apache.org/jira/browse/FLINK-1159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15211923#comment-15211923
] 

ASF GitHub Bot commented on FLINK-1159:
---------------------------------------

Github user stefanobaghino commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1704#discussion_r57452214
  
    --- Diff: flink-scala/src/main/scala/org/apache/flink/api/scala/extensions/package.scala
---
    @@ -0,0 +1,201 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.extensions.acceptPartialFunctions._
    +
    +import scala.reflect.ClassTag
    +
    +package object extensions {
    +
    +  /**
    +    * acceptPartialFunctions extends the original DataSet with methods with unique names
    +    * that delegate to core higher-order functions (e.g. `map`) so that we can work around
    +    * the fact that overloaded methods taking functions as parameters can't accept partial
    +    * functions as well. This enables the possibility to directly apply pattern matching
    +    * to decompose inputs such as tuples, case classes and collections.
    +    *
    +    * e.g.
    +    * {{{
    +    *   object Main {
    +    *     import org.apache.flink.api.scala.extensions._
    +    *     case class Point(x: Double, y: Double)
    +    *     def main(args: Array[String]): Unit = {
    +    *       val env = ExecutionEnvironment.getExecutionEnvironment
    +    *       val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6))
    +    *       ds.filterWith {
    +    *         case Point(x, _) => x > 1
    +    *       }.reduceWith {
    +    *         case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2)
    +    *       }.mapWith {
    +    *         case Point(x, y) => (x, y)
    +    *       }.flatMapWith {
    +    *         case (x, y) => Seq('x' -> x, 'y' -> y)
    +    *       }.groupingBy {
    +    *         case (id, value) => id
    +    *       }
    +    *     }
    +    *   }
    +    * }}}
    +    *
    +    */
    +  implicit def acceptPartialFunctionsOnDataSet[T: TypeInformation](ds: DataSet[T]): OnDataSet[T]
=
    --- End diff --
    
    Ok, in the end the problem was a clash between the `acceptPartialFunctions` package and
implicit conversions. I've solved it by moving the actual implementations under the `impl`
package under `acceptPartialFunctions`; now the whole set of conversions can be imported with
the `acceptPartialFunctions` name. I was wrong regarding the issue on the resolution of overloaded
implicit conversions.


> Case style anonymous functions not supported by Scala API
> ---------------------------------------------------------
>
>                 Key: FLINK-1159
>                 URL: https://issues.apache.org/jira/browse/FLINK-1159
>             Project: Flink
>          Issue Type: Bug
>          Components: Scala API
>            Reporter: Till Rohrmann
>            Assignee: Stefano Baghino
>
> In Scala it is very common to define anonymous functions of the following form
> {code}
> {
> case foo: Bar => foobar(foo)
> case _ => throw new RuntimeException()
> }
> {code}
> These case style anonymous functions are not supported yet by the Scala API. Thus, one
has to write redundant code to name the function parameter.
> What works is the following pattern, but it is not intuitive for someone coming from
Scala:
> {code}
> dataset.map{
>   _ match{
>     case foo:Bar => ...
>   }
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message