flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jelmer Kuperus (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8828) Add collect method to DataStream / DataSet scala api
Date Sat, 03 Mar 2018 18:12:00 GMT

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

Jelmer Kuperus commented on FLINK-8828:

You are right. Infact the fact that there is already a method defined called collect() seems to
be biting me :(. TableSourceITCase is not compiling because the implicit conversions defined
in [org.apache.flink.table.api.scala._|https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala] now 
think the no args collect method is ambiguous.

And there does not seem to be a good way to make this class compile this without renaming method
I introduced to something else.

The problem is that scala developers expect this method to be called collect..

As an aside I think using implicit conversions in the way [org.apache.flink.table.api.scala._|https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala] does
is frowned upon a little in the scala world. A slightly more explicit that will give less
unexpected surprises like this one might look something like

implicit class RichTable[T](table: Table) {
  def asRowDataSet: DataSet[Row] = {
    val tableEnv = table.tableEnv.asInstanceOf[ScalaBatchTableEnv]

  def asRowDataStream: DataStream[Row] = {
    val tableEnv = table.tableEnv.asInstanceOf[ScalaStreamTableEnv]
Then this code
val results = tEnv.scan("T")
  .select('name, 'rtime, 'val)
Could be rewritten as
val results = tEnv.scan("T")
  .select('name, 'rtime, 'val)


> 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
> 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

View raw message