Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-7204] [core] CombineHint.NONE #4350

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
[FLINK-7204] [core] CombineHint.NONE
FLINK-3477 added a hash-combine preceding the reducer configured with
CombineHint.HASH or CombineHint.SORT (default). In some cases it may be
useful to disable the combiner in ReduceNode by specifying a new
CombineHint.NONE value.
  • Loading branch information
greghogan committed Jul 17, 2017
commit 1adbce836013440a5bd96a5e38fec24a8c0928a7
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,13 @@ public enum CombineHint {
* Use a hash-based strategy. This should be faster in most cases, especially if the number
* of different keys is small compared to the number of input elements (eg. 1/10).
*/
HASH
HASH,

/**
* Disable the use of a combiner. This can be faster in cases when the number of different keys
* is very small compared to the number of input elements (eg. 1/100).
*/
NONE
}

private CombineHint hint;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.graph.asm.simple.directed;

import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.graph.Edge;
import org.apache.flink.graph.Graph;
Expand All @@ -44,6 +45,7 @@ public Graph<K, VV, EV> runInternal(Graph<K, VV, EV> input)
.setParallelism(parallelism)
.name("Remove self-loops")
.distinct(0, 1)
.setCombineHint(CombineHint.NONE)
.setParallelism(parallelism)
.name("Remove duplicate edges");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.graph.asm.simple.undirected;

import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.graph.Edge;
import org.apache.flink.graph.Graph;
Expand Down Expand Up @@ -74,6 +75,7 @@ public Graph<K, VV, EV> runInternal(Graph<K, VV, EV> input)
.setParallelism(parallelism)
.name("Remove self-loops")
.distinct(0, 1)
.setCombineHint(CombineHint.NONE)
.setParallelism(parallelism)
.name("Remove duplicate edges");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ public ReduceNode(ReduceOperatorBase<?, ?> operator) {
case HASH:
combinerStrategy = DriverStrategy.HASHED_PARTIAL_REDUCE;
break;
case NONE:
combinerStrategy = DriverStrategy.SORTED_REDUCE;
break;
default:
throw new RuntimeException("Unknown CombineHint");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
in.setLocalStrategy(LocalStrategy.COMBININGSORT, in.getLocalStrategyKeys(),
in.getLocalStrategySortOrder());
}
return new SingleInputPlanNode(node, "Reduce("+node.getOperator().getName()+")", in,
return new SingleInputPlanNode(node, "Reduce ("+node.getOperator().getName()+")", in,
DriverStrategy.SORTED_GROUP_REDUCE, this.keyList);
} else {
// non forward case. all local properties are killed anyways, so we can safely plug in a combiner
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
in.getShipStrategySortOrder(), in.getDataExchangeMode());
toReducer.setLocalStrategy(LocalStrategy.SORT, in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());

return new SingleInputPlanNode(node, "Reduce("+node.getOperator().getName()+")", toReducer,
return new SingleInputPlanNode(node, "Reduce ("+node.getOperator().getName()+")", toReducer,
DriverStrategy.SORTED_REDUCE, this.keyList);
}
}
Expand Down