(this, other);
}
+ // --------------------------------------------------------------------------------------------
+ // Partitioning
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Hash-partitions a DataSet on the specified key fields.
+ *
+ * Important:This operation shuffles the whole DataSet over the network and can take significant amount of time.
+ *
+ * @param fields The field indexes on which the DataSet is hash-partitioned.
+ * @return The partitioned DataSet.
+ */
+ public PartitionedDataSet partitionByHash(int... fields) {
+ return new PartitionedDataSet(this, PartitionMethod.HASH, new Keys.FieldPositionKeys(fields, getType(), false));
+ }
+
+ /**
+ * Partitions a DataSet using the specified KeySelector.
+ *
+ * Important:This operation shuffles the whole DataSet over the network and can take significant amount of time.
+ *
+ * @param keyExtractor The KeyExtractor with which the DataSet is hash-partitioned.
+ * @return The partitioned DataSet.
+ *
+ * @see KeySelector
+ */
+ public > PartitionedDataSet partitionByHash(KeySelector keyExtractor) {
+ final TypeInformation keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type);
+ return new PartitionedDataSet(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys(keyExtractor, this.getType(), keyType));
+ }
+
+ /**
+ * Enforces a rebalancing of the DataSet, i.e., the DataSet is evenly distributed over all parallel instances of the
+ * following task. This can help to improve performance in case of heavy data skew and compute intensive operations.
+ *
+ * Important:This operation shuffles the whole DataSet over the network and can take significant amount of time.
+ *
+ * @return The rebalanced DataSet.
+ */
+ public PartitionedDataSet rebalance() {
+ return new PartitionedDataSet(this, PartitionMethod.REBALANCE);
+ }
+
// --------------------------------------------------------------------------------------------
// Top-K
// --------------------------------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
index e9394381d7ee396a101e718d7dbcd8da4b4b5b8c..727820aa1a9ea95237a8578d17a527bac26b2074 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.java.operators.translation.PlanFilterOperator;
-
import org.apache.flink.api.java.DataSet;
/**
@@ -35,7 +34,8 @@ public class FilterOperator extends SingleInputUdfOperator function;
-
+ protected PartitionedDataSet partitionedDataSet;
+
public FilterOperator(DataSet input, FilterFunction function) {
super(input, input.getType());
@@ -43,9 +43,19 @@ public class FilterOperator extends SingleInputUdfOperator input, FilterFunction function) {
+ this(input.getDataSet(), function);
+ this.partitionedDataSet = input;
+ }
+
@Override
protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) {
+ // inject partition operator if necessary
+ if(this.partitionedDataSet != null) {
+ input = this.partitionedDataSet.translateToDataFlow(input, this.getParallelism());
+ }
+
String name = getName() != null ? getName() : function.getClass().getName();
// create operator
PlanFilterOperator po = new PlanFilterOperator(function, name, getInputType());
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
index ff9076b2caf8ca2bd4f971e2759d5a6990652792..0dc401ec3c78f62028f22ddd0f55e6e846c89d12 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
@@ -36,6 +36,7 @@ public class FlatMapOperator extends SingleInputUdfOperator function;
+ protected PartitionedDataSet partitionedDataSet;
public FlatMapOperator(DataSet input, TypeInformation resultType, FlatMapFunction function) {
super(input, resultType);
@@ -43,10 +44,20 @@ public class FlatMapOperator extends SingleInputUdfOperator input, TypeInformation resultType, FlatMapFunction function) {
+ this(input.getDataSet(), resultType, function);
+ this.partitionedDataSet = input;
+ }
@Override
protected org.apache.flink.api.common.operators.base.FlatMapOperatorBase> translateToDataFlow(Operator input) {
+ // inject partition operator if necessary
+ if(this.partitionedDataSet != null) {
+ input = this.partitionedDataSet.translateToDataFlow(input, this.getParallelism());
+ }
+
String name = getName() != null ? getName() : function.getClass().getName();
// create operator
FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name);
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
index 2db0b3b8e8fa5af1ae7b053a2ac6b7eb108d19aa..e45cc96afaa8614b778ef560a61793751b07d5ed 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
@@ -38,6 +38,8 @@ public class MapOperator extends SingleInputUdfOperator function;
+ protected PartitionedDataSet partitionedDataSet;
+
public MapOperator(DataSet input, TypeInformation resultType, MapFunction function) {
@@ -46,10 +48,20 @@ public class MapOperator extends SingleInputUdfOperator input, TypeInformation resultType, MapFunction function) {
+ this(input.getDataSet(), resultType, function);
+ this.partitionedDataSet = input;
+ }
@Override
protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) {
+ // inject partition operator if necessary
+ if(this.partitionedDataSet != null) {
+ input = this.partitionedDataSet.translateToDataFlow(input, this.getParallelism());
+ }
+
String name = getName() != null ? getName() : function.getClass().getName();
// create operator
MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name);
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
index fc0503631199c01fab0cb79a61c00d0ab5adf0fb..9c896f9644382cbbe61da9fef6a010f0db5a7f55 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
@@ -38,6 +38,7 @@ public class MapPartitionOperator extends SingleInputUdfOperator function;
+ protected PartitionedDataSet partitionedDataSet;
public MapPartitionOperator(DataSet input, TypeInformation resultType, MapPartitionFunction function) {
super(input, resultType);
@@ -45,10 +46,20 @@ public class MapPartitionOperator extends SingleInputUdfOperator input, TypeInformation resultType, MapPartitionFunction function) {
+ this(input.getDataSet(), resultType, function);
+ this.partitionedDataSet = input;
+ }
@Override
protected MapPartitionOperatorBase> translateToDataFlow(Operator input) {
+ // inject partition operator if necessary
+ if(this.partitionedDataSet != null) {
+ input = this.partitionedDataSet.translateToDataFlow(input, this.getParallelism());
+ }
+
String name = getName() != null ? getName() : function.getClass().getName();
// create operator
MapPartitionOperatorBase> po = new MapPartitionOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name);
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index 7030be59ddb8a378fb8d283ac0339b2b9b1f5e6c..eb17134717d75a59b452608422bf7b2c093f0518 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -67,6 +67,7 @@ public class OperatorTranslation {
private Operator translate(DataSet dataSet) {
+
// check if we have already translated that data set (operation or source)
Operator> previous = (Operator>) this.translated.get(dataSet);
if (previous != null) {
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionedDataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionedDataSet.java
new file mode 100644
index 0000000000000000000000000000000000000000..0ab984df03ff31a713d6f3949348d1b1ff7b88d2
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionedDataSet.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.UnaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.MapOperatorBase;
+import org.apache.flink.api.common.operators.base.PartitionOperatorBase;
+import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
+import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
+import org.apache.flink.api.java.operators.translation.KeyRemovingMapper;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+public class PartitionedDataSet {
+
+ private final DataSet dataSet;
+
+ private final Keys pKeys;
+ private final PartitionMethod pMethod;
+
+ public PartitionedDataSet(DataSet input, PartitionMethod pMethod, Keys pKeys) {
+ this.dataSet = input;
+
+ if(pMethod == PartitionMethod.HASH && pKeys == null) {
+ throw new IllegalArgumentException("Hash Partitioning requires keys");
+ } else if(pMethod == PartitionMethod.RANGE) {
+ throw new UnsupportedOperationException("Range Partitioning not yet supported");
+ }
+
+ if(pKeys instanceof Keys.FieldPositionKeys> && !input.getType().isTupleType()) {
+ throw new IllegalArgumentException("Hash Partitioning with key fields only possible on Tuple DataSets");
+ }
+
+ this.pMethod = pMethod;
+ this.pKeys = pKeys;
+ }
+
+ public PartitionedDataSet(DataSet input, PartitionMethod pMethod) {
+ this(input, pMethod, null);
+ }
+
+ public DataSet getDataSet() {
+ return this.dataSet;
+ }
+
+
+ /**
+ * Applies a Map transformation on a {@link DataSet}.
+ * The transformation calls a {@link org.apache.flink.api.java.functions.RichMapFunction} for each element of the DataSet.
+ * Each MapFunction call returns exactly one element.
+ *
+ * @param mapper The MapFunction that is called for each element of the DataSet.
+ * @return A MapOperator that represents the transformed DataSet.
+ *
+ * @see org.apache.flink.api.java.functions.RichMapFunction
+ * @see MapOperator
+ * @see DataSet
+ */
+ public MapOperator map(MapFunction mapper) {
+ if (mapper == null) {
+ throw new NullPointerException("Map function must not be null.");
+ }
+ if (FunctionUtils.isLambdaFunction(mapper)) {
+ throw new UnsupportedLambdaExpressionException();
+ }
+
+ final TypeInformation resultType = TypeExtractor.getMapReturnTypes(mapper, dataSet.getType());
+
+ return new MapOperator(this, resultType, mapper);
+ }
+
+ /**
+ * Applies a Map-style operation to the entire partition of the data.
+ * The function is called once per parallel partition of the data,
+ * and the entire partition is available through the given Iterator.
+ * The number of elements that each instance of the MapPartition function
+ * sees is non deterministic and depends on the degree of parallelism of the operation.
+ *
+ * This function is intended for operations that cannot transform individual elements,
+ * requires no grouping of elements. To transform individual elements,
+ * the use of {@code map()} and {@code flatMap()} is preferable.
+ *
+ * @param mapPartition The MapPartitionFunction that is called for the full DataSet.
+ * @return A MapPartitionOperator that represents the transformed DataSet.
+ *
+ * @see MapPartitionFunction
+ * @see MapPartitionOperator
+ * @see DataSet
+ */
+ public MapPartitionOperator mapPartition(MapPartitionFunction mapPartition ){
+ if (mapPartition == null) {
+ throw new NullPointerException("MapPartition function must not be null.");
+ }
+
+ final TypeInformation resultType = TypeExtractor.getMapPartitionReturnTypes(mapPartition, dataSet.getType());
+
+ return new MapPartitionOperator(this, resultType, mapPartition);
+ }
+
+ /**
+ * Applies a FlatMap transformation on a {@link DataSet}.
+ * The transformation calls a {@link org.apache.flink.api.java.functions.RichFlatMapFunction} for each element of the DataSet.
+ * Each FlatMapFunction call can return any number of elements including none.
+ *
+ * @param flatMapper The FlatMapFunction that is called for each element of the DataSet.
+ * @return A FlatMapOperator that represents the transformed DataSet.
+ *
+ * @see org.apache.flink.api.java.functions.RichFlatMapFunction
+ * @see FlatMapOperator
+ * @see DataSet
+ */
+ public FlatMapOperator flatMap(FlatMapFunction flatMapper) {
+ if (flatMapper == null) {
+ throw new NullPointerException("FlatMap function must not be null.");
+ }
+ if (FunctionUtils.isLambdaFunction(flatMapper)) {
+ throw new UnsupportedLambdaExpressionException();
+ }
+
+ TypeInformation resultType = TypeExtractor.getFlatMapReturnTypes(flatMapper, dataSet.getType());
+
+ return new FlatMapOperator(this, resultType, flatMapper);
+ }
+
+ /**
+ * Applies a Filter transformation on a {@link DataSet}.
+ * The transformation calls a {@link org.apache.flink.api.java.functions.RichFilterFunction} for each element of the DataSet
+ * and retains only those element for which the function returns true. Elements for
+ * which the function returns false are filtered.
+ *
+ * @param filter The FilterFunction that is called for each element of the DataSet.
+ * @return A FilterOperator that represents the filtered DataSet.
+ *
+ * @see org.apache.flink.api.java.functions.RichFilterFunction
+ * @see FilterOperator
+ * @see DataSet
+ */
+ public FilterOperator filter(FilterFunction filter) {
+ if (filter == null) {
+ throw new NullPointerException("Filter function must not be null.");
+ }
+ return new FilterOperator(this, filter);
+ }
+
+
+ /*
+ * Translation of partitioning
+ */
+
+ protected org.apache.flink.api.common.operators.SingleInputOperator, IN, ?> translateToDataFlow(Operator input, int partitionDop) {
+
+ String name = "Partition";
+
+ // distinguish between partition types
+ if (pMethod == PartitionMethod.REBALANCE) {
+
+ UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(dataSet.getType(), dataSet.getType());
+ PartitionOperatorBase noop = new PartitionOperatorBase(operatorInfo, pMethod, name);
+ // set input
+ noop.setInput(input);
+ // set DOP
+ noop.setDegreeOfParallelism(partitionDop);
+
+ return noop;
+ }
+ else if (pMethod == PartitionMethod.HASH) {
+
+ if (pKeys instanceof Keys.FieldPositionKeys) {
+
+ int[] logicalKeyPositions = pKeys.computeLogicalKeyPositions();
+ UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation(dataSet.getType(), dataSet.getType());
+ PartitionOperatorBase noop = new PartitionOperatorBase(operatorInfo, pMethod, logicalKeyPositions, name);
+ // set input
+ noop.setInput(input);
+ // set DOP
+ noop.setDegreeOfParallelism(partitionDop);
+
+ return noop;
+ } else if (pKeys instanceof Keys.SelectorFunctionKeys) {
+
+ @SuppressWarnings("unchecked")
+ Keys.SelectorFunctionKeys selectorKeys = (Keys.SelectorFunctionKeys) pKeys;
+ MapOperatorBase, IN, ?> po = translateSelectorFunctionReducer(selectorKeys, pMethod, dataSet.getType(), name, input, partitionDop);
+ return po;
+ }
+ else {
+ throw new UnsupportedOperationException("Unrecognized key type.");
+ }
+
+ }
+ else if (pMethod == PartitionMethod.RANGE) {
+ throw new UnsupportedOperationException("Range partitioning not yet supported");
+ }
+
+ return null;
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ private static MapOperatorBase, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys rawKeys,
+ PartitionMethod pMethod, TypeInformation inputType, String name, Operator input, int partitionDop)
+ {
+ @SuppressWarnings("unchecked")
+ final Keys.SelectorFunctionKeys keys = (Keys.SelectorFunctionKeys) rawKeys;
+
+ TypeInformation> typeInfoWithKey = new TupleTypeInfo>(keys.getKeyType(), inputType);
+ UnaryOperatorInformation, Tuple2> operatorInfo = new UnaryOperatorInformation, Tuple2>(typeInfoWithKey, typeInfoWithKey);
+
+ KeyExtractingMapper extractor = new KeyExtractingMapper(keys.getKeyExtractor());
+
+ MapOperatorBase, MapFunction>> keyExtractingMap = new MapOperatorBase, MapFunction>>(extractor, new UnaryOperatorInformation>(inputType, typeInfoWithKey), "Key Extractor");
+ PartitionOperatorBase> noop = new PartitionOperatorBase>(operatorInfo, pMethod, new int[]{0}, name);
+ MapOperatorBase, T, MapFunction, T>> keyRemovingMap = new MapOperatorBase, T, MapFunction, T>>(new KeyRemovingMapper(), new UnaryOperatorInformation, T>(typeInfoWithKey, inputType), "Key Extractor");
+
+ keyExtractingMap.setInput(input);
+ noop.setInput(keyExtractingMap);
+ keyRemovingMap.setInput(noop);
+
+ // set dop
+ keyExtractingMap.setDegreeOfParallelism(input.getDegreeOfParallelism());
+ noop.setDegreeOfParallelism(partitionDop);
+ keyRemovingMap.setDegreeOfParallelism(partitionDop);
+
+ return keyRemovingMap;
+ }
+
+
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java
index 202a95197df66bdecb21e4ff1ca6baee5b8d675f..4f297b001f650c988e978ba98bb9c7ced07f045f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java
@@ -86,6 +86,7 @@ public class OutputEmitter implements ChannelSelector implements ChannelSelector getConfigurations() throws FileNotFoundException, IOException {
+
+ LinkedList tConfigs = new LinkedList();
+
+ for(int i=1; i <= NUM_PROGRAMS; i++) {
+ Configuration config = new Configuration();
+ config.setInteger("ProgramId", i);
+ tConfigs.add(config);
+ }
+
+ return toParameterList(tConfigs);
+ }
+
+ private static class PartitionProgs {
+
+ public static String runProgram(int progId, String resultPath) throws Exception {
+
+ switch(progId) {
+ case 1: {
+ /*
+ * Test hash partition by key field
+ */
+
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet> ds = CollectionDataSets.get3TupleDataSet(env);
+ DataSet uniqLongs = ds
+ .partitionByHash(1)
+ .mapPartition(new UniqueLongMapper());
+ uniqLongs.writeAsText(resultPath);
+ env.execute();
+
+ // return expected result
+ return "1\n" +
+ "2\n" +
+ "3\n" +
+ "4\n" +
+ "5\n" +
+ "6\n";
+ }
+ case 2: {
+ /*
+ * Test hash partition by key selector
+ */
+
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet> ds = CollectionDataSets.get3TupleDataSet(env);
+ DataSet uniqLongs = ds
+ .partitionByHash(new KeySelector, Long>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Long getKey(Tuple3 value) throws Exception {
+ return value.f1;
+ }
+
+ })
+ .mapPartition(new UniqueLongMapper());
+ uniqLongs.writeAsText(resultPath);
+ env.execute();
+
+ // return expected result
+ return "1\n" +
+ "2\n" +
+ "3\n" +
+ "4\n" +
+ "5\n" +
+ "6\n";
+ }
+ case 3: {
+ /*
+ * Test forced rebalancing
+ */
+
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ // generate some number in parallel
+ DataSet ds = env.generateSequence(1,3000);
+ DataSet> uniqLongs = ds
+ // introduce some partition skew by filtering
+ .filter(new FilterFunction() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean filter(Long value) throws Exception {
+ if (value <= 780) {
+ return false;
+ } else {
+ return true;
+ }
+ }
+ })
+ // rebalance
+ .rebalance()
+ // count values in each partition
+ .map(new PartitionIndexMapper())
+ .groupBy(0)
+ .reduce(new ReduceFunction>() {
+ private static final long serialVersionUID = 1L;
+
+ public Tuple2 reduce(Tuple2 v1, Tuple2 v2) {
+ return new Tuple2(v1.f0, v1.f1+v2.f1);
+ }
+ })
+ // round counts to mitigate runtime scheduling effects (lazy split assignment)
+ .map(new MapFunction, Tuple2>(){
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple2 map(Tuple2 value) throws Exception {
+ value.f1 = (value.f1 / 10);
+ return value;
+ }
+
+ });
+
+ uniqLongs.writeAsText(resultPath);
+
+ env.execute();
+
+ // return expected result
+ return "(0,55)\n" +
+ "(1,55)\n" +
+ "(2,55)\n" +
+ "(3,55)\n";
+ }
+ case 4: {
+ /*
+ * Test hash partition by key field and different DOP
+ */
+
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ env.setDegreeOfParallelism(3);
+
+ DataSet> ds = CollectionDataSets.get3TupleDataSet(env);
+ DataSet uniqLongs = ds
+ .partitionByHash(1)
+ .mapPartition(new UniqueLongMapper()).setParallelism(4);
+ uniqLongs.writeAsText(resultPath);
+
+ env.execute();
+
+ // return expected result
+ return "1\n" +
+ "2\n" +
+ "3\n" +
+ "4\n" +
+ "5\n" +
+ "6\n";
+ }
+
+ default:
+ throw new IllegalArgumentException("Invalid program id");
+ }
+
+ }
+
+ }
+
+ public static class UniqueLongMapper implements MapPartitionFunction, Long> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void mapPartition(Iterable> records, Collector out) throws Exception {
+ HashSet uniq = new HashSet();
+ for(Tuple3 t : records) {
+ uniq.add(t.f1);
+ }
+ for(Long l : uniq) {
+ out.collect(l);
+ }
+ }
+ }
+
+ public static class PartitionIndexMapper extends RichMapFunction> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public Tuple2 map(Long value) throws Exception {
+ return new Tuple2(this.getRuntimeContext().getIndexOfThisSubtask(), 1);
+ }
+
+ }
+
+}