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

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

GitHub user uce opened a pull request:

    https://github.com/apache/incubator-flink/pull/63

    [FLINK-758] Add count operator to DataSet

    - Adds a count operator to DataSet:
    
    ```java
    DataSet<String> text = env.fromElements(
        "Who's there?",
        "I think I hear them. Stand, ho! Who's there?");
    DataSet<Long> count = text.count(); // 2
    ```
    - Adds an all (ungrouped) reduce variant, which allows to specify an 
initial value to the reduce function
    - Adds an utility method to InstantiationUtil to serialize a record to a 
byte array
    - Fixes some javadocs warnings in DataSet
    
    I've tested the count operator on a cluster with DOP 1, 200 and verified 
the results with `cat | wc -l`.

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

    $ git pull https://github.com/uce/incubator-flink FLINK-758-count_operator

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

    https://github.com/apache/incubator-flink/pull/63.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 #63
    
----
commit 7ae6ffb8f25525a1633fe27a08af0604faa5b3b6
Author: uce <[email protected]>
Date:   2014-07-03T14:35:48Z

    Add byte array serialization to InstantiationUtil

commit 3d6b1fe9c8efd1ab31cc260a29663c60e3fb1116
Author: uce <[email protected]>
Date:   2014-07-07T13:57:45Z

    Add initial value to all (ungrouped) reduce
    
    Adds an all (ungrouped) reduce variant to DataSet, which allows to
    specify an initial value for the ReduceFunction. The initial value is
    an extra input element to the ReduceFuntion and needs to be of the same
    type as the ReduceFunction input type. The initial value is not used
    with the combiners.
    
    In cases, where there is no input to the reduce function, for example
    after a filter operator, which filters all elements, the ReduceFunction
    will be called with the initial value only.

commit dfbe9ed6557fae798ba82ffff79809e93b779164
Author: uce <[email protected]>
Date:   2014-07-07T13:59:26Z

    [FLINK-758] Add count operator to DataSet
    
    Adds a count method to DataSet, which translates to a map-reduce. The
    map operator maps each element to a 1 and the reduce operator sums up
    all the 1s, resulting in the total count of elements.

----


> Add count method to DataSet and implement CountOperator
> -------------------------------------------------------
>
>                 Key: FLINK-758
>                 URL: https://issues.apache.org/jira/browse/FLINK-758
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: GitHub Import
>              Labels: github-import
>             Fix For: pre-apache
>
>         Attachments: pull-request-758-7518001488867571817.patch
>
>
> At the request of @twalthr. This is the count operator I've implemented some 
> time ago to get the to know the new Java API. It introduces 
> `DataSet.count()`, which is executed as a map (to ones) and reduce (sum up 
> the ones). I initially didn't do the PR, because of the following problem: 
> empty DataSets don't work as the first map won't have any input to operate on.
> If more people think that we should include this operator we can think about 
> a possible solution to the problem.
> ---------------- Imported from GitHub ----------------
> Url: https://github.com/stratosphere/stratosphere/pull/758
> Created by: [uce|https://github.com/uce]
> Labels: enhancement, java api, 
> Milestone: Release 0.6 (unplanned)
> Created at: Tue May 06 10:42:33 CEST 2014
> State: open



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to