You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/03/02 09:28:00 UTC

[jira] [Commented] (FLINK-8828) Add collect method to DataStream / DataSet scala api

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

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

GitHub user jelmerk opened a pull request:

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

    [FLINK-8828] [stream, dataset, scala] Introduce collect method

    ## What is the purpose of the change
    
    A collect function is a method that takes a Partial Function as its parameter and applies it to all the elements in the collection to create a new collection which satisfies the Partial Function. It makes certain things nicer to express 
    
    ## Brief change log
    
    - added collect method on scala dataset and datastream api
    
    ## Verifying this change
    
    It seems to be hard to find a place where this could be tested in isolation, suggestions welcome
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): no
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes
      - The serializers: don't know
      - The runtime per-record code paths (performance sensitive): yes
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper:don't know
      - The S3 file system connector: no
    
    ## Documentation
    
      - Does this pull request introduce a new feature? yes
      - If yes, how is the feature documented? JavaDocs


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jelmerk/flink collect_support

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5616.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 #5616
    
----
commit 61b4bcb7c941950d62d0db0aa2041f1796fdbaaf
Author: Jelmer Kuperus <jk...@...>
Date:   2018-03-02T09:17:54Z

    [FLINK-8828] [stream, dataset, scala] Introduce collect method

----


> Add collect method to DataStream / DataSet scala api
> ----------------------------------------------------
>
>                 Key: FLINK-8828
>                 URL: https://issues.apache.org/jira/browse/FLINK-8828
>             Project: Flink
>          Issue Type: Improvement
>          Components: Core, DataSet API, DataStream API, Scala API
>    Affects Versions: 1.4.0
>            Reporter: Jelmer Kuperus
>            Priority: Major
>
> A collect function is a method that takes a Partial Function as its parameter and applies it to all the elements in the collection to create a new collection which satisfies the Partial Function.
> It can be found on all [core scala collection classes|http://www.scala-lang.org/api/2.9.2/scala/collection/TraversableLike.html] as well as on spark's [rdd interface|https://spark.apache.org/docs/2.2.0/api/scala/index.html#org.apache.spark.rdd.RDD]
> To understand its utility imagine the following scenario :
> Given a DataStream that produces events of type _Purchase_ and _View_ 
> Transform this stream into a stream of purchase amounts over 1000 euros.
> Currently an implementation might look like
> {noformat}
> val x = dataStream
>   .filter(_.isInstanceOf[Purchase])
>   .map(_.asInstanceOf[Purchase])
>   .filter(_.amount > 1000)
>   .map(_.amount){noformat}
> Or alternatively you could do this
> {noformat}
> dataStream.flatMap(_ match {
>   case p: Purchase if p.amount > 1000 => Some(p.amount)
>   case _ => None
> }){noformat}
> But with collect implemented it could look like
> {noformat}
> dataStream.collect {
>   case p: Purchase if p.amount > 1000 => p.amount
> }{noformat}
>  
> Which is a lot nicer to both read and write



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)