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

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

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

    https://github.com/apache/flink/pull/1517#discussion_r70182792
  
    --- Diff: 
flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java ---
    @@ -45,10 +46,28 @@ public ReduceNode(ReduceOperatorBase<?, ?> operator) {
                        // case of a key-less reducer. force a parallelism of 1
                        setParallelism(1);
                }
    -           
    -           OperatorDescriptorSingle props = this.keys == null ?
    -                   new AllReduceProperties() :
    -                   new ReduceProperties(this.keys, 
operator.getCustomPartitioner());
    +
    +           OperatorDescriptorSingle props;
    +
    +           if (this.keys == null) {
    +                   props = new AllReduceProperties();
    +           } else {
    +                   DriverStrategy combinerStrategy;
    +                   switch(operator.getCombineHint()) {
    +                           case OPTIMIZER_CHOOSES:
    +                                   combinerStrategy = 
DriverStrategy.SORTED_PARTIAL_REDUCE;
    --- End diff --
    
    I included the `OPTIMIZER_CHOOSES` case because `JoinHint` and `CrossHint` 
also have this case. I guess it is there because we might modify what the 
default is later.


> Add hash-based combine strategy for ReduceFunction
> --------------------------------------------------
>
>                 Key: FLINK-3477
>                 URL: https://issues.apache.org/jira/browse/FLINK-3477
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Local Runtime
>            Reporter: Fabian Hueske
>            Assignee: Gabor Gevay
>
> This issue is about adding a hash-based combine strategy for ReduceFunctions.
> The interface of the {{reduce()}} method is as follows:
> {code}
> public T reduce(T v1, T v2)
> {code}
> Input type and output type are identical and the function returns only a 
> single value. A Reduce function is incrementally applied to compute a final 
> aggregated value. This allows to hold the preaggregated value in a hash-table 
> and update it with each function call. 
> The hash-based strategy requires special implementation of an in-memory hash 
> table. The hash table should support in place updates of elements (if the 
> updated value has the same size as the new value) but also appending updates 
> with invalidation of the old value (if the binary length of the new value 
> differs). The hash table needs to be able to evict and emit all elements if 
> it runs out-of-memory.
> We should also add {{HASH}} and {{SORT}} compiler hints to 
> {{DataSet.reduce()}} and {{Grouping.reduce()}} to allow users to pick the 
> execution strategy.



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

Reply via email to