diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index 26cf054525384..05bf9094b1266 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -18,9 +18,6 @@ package org.apache.flink.api.java.operators; -import java.util.ArrayList; -import java.util.List; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.InvalidProgramException; @@ -42,56 +39,57 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; +import java.util.ArrayList; +import java.util.List; + /** * This operator represents the application of a "aggregate" operation on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set aggregated by the operator. */ @Public public class AggregateOperator extends SingleInputOperator> { - + private final List> aggregationFunctions = new ArrayList<>(4); - + private final List fields = new ArrayList<>(4); - + private final Grouping grouping; - + private final String aggregateLocationName; - + /** - *

- * Non grouped aggregation + * Non grouped aggregation. */ public AggregateOperator(DataSet input, Aggregations function, int field, String aggregateLocationName) { super(Preconditions.checkNotNull(input), input.getType()); Preconditions.checkNotNull(function); - + this.aggregateLocationName = aggregateLocationName; - + if (!input.getType().isTupleType()) { throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types."); } - + TupleTypeInfoBase inType = (TupleTypeInfoBase) input.getType(); - + if (field < 0 || field >= inType.getArity()) { throw new IllegalArgumentException("Aggregation field position is out of range."); } - + AggregationFunctionFactory factory = function.getFactory(); AggregationFunction aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass()); - + // this is the first aggregation operator after a regular data set (non grouped aggregation) this.aggregationFunctions.add(aggFunct); this.fields.add(field); this.grouping = null; } - + /** - * - * Grouped aggregation - * + * Grouped aggregation. + * * @param input * @param function * @param field @@ -99,49 +97,46 @@ public AggregateOperator(DataSet input, Aggregations function, int field, St public AggregateOperator(Grouping input, Aggregations function, int field, String aggregateLocationName) { super(Preconditions.checkNotNull(input).getInputDataSet(), input.getInputDataSet().getType()); Preconditions.checkNotNull(function); - + this.aggregateLocationName = aggregateLocationName; - + if (!input.getInputDataSet().getType().isTupleType()) { throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types."); } - + TupleTypeInfoBase inType = (TupleTypeInfoBase) input.getInputDataSet().getType(); - + if (field < 0 || field >= inType.getArity()) { throw new IllegalArgumentException("Aggregation field position is out of range."); } - + AggregationFunctionFactory factory = function.getFactory(); AggregationFunction aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass()); - + // set the aggregation fields this.aggregationFunctions.add(aggFunct); this.fields.add(field); this.grouping = input; } - - + public AggregateOperator and(Aggregations function, int field) { Preconditions.checkNotNull(function); - + TupleTypeInfoBase inType = (TupleTypeInfoBase) getType(); - + if (field < 0 || field >= inType.getArity()) { throw new IllegalArgumentException("Aggregation field position is out of range."); } - - + AggregationFunctionFactory factory = function.getFactory(); AggregationFunction aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass()); - + this.aggregationFunctions.add(aggFunct); this.fields.add(field); return this; } - public AggregateOperator andSum (int field) { return this.and(Aggregations.SUM, field); } @@ -154,68 +149,65 @@ public AggregateOperator andMax (int field) { return this.and(Aggregations.MAX, field); } - @SuppressWarnings("unchecked") @Override @Internal protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase> translateToDataFlow(Operator input) { - + // sanity check if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) { throw new IllegalStateException(); } - + // construct the aggregation function AggregationFunction[] aggFunctions = new AggregationFunction[this.aggregationFunctions.size()]; int[] fields = new int[this.fields.size()]; StringBuilder genName = new StringBuilder(); - + for (int i = 0; i < fields.length; i++) { aggFunctions[i] = (AggregationFunction) this.aggregationFunctions.get(i); fields[i] = this.fields.get(i); - + genName.append(aggFunctions[i].toString()).append('(').append(fields[i]).append(')').append(','); } genName.append(" at ").append(aggregateLocationName); - genName.setLength(genName.length()-1); - - + genName.setLength(genName.length() - 1); + @SuppressWarnings("rawtypes") RichGroupReduceFunction function = new AggregatingUdf(aggFunctions, fields); - - + String name = getName() != null ? getName() : genName.toString(); - + // distinguish between grouped reduce and non-grouped reduce if (this.grouping == null) { // non grouped aggregation UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getInputType(), getResultType()); GroupReduceOperatorBase> po = new GroupReduceOperatorBase>(function, operatorInfo, new int[0], name); - + po.setCombinable(true); - + // set input po.setInput(input); // set parallelism po.setParallelism(this.getParallelism()); - + return po; } - + if (this.grouping.getKeys() instanceof Keys.ExpressionKeys) { // grouped aggregation int[] logicalKeyPositions = this.grouping.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getInputType(), getResultType()); GroupReduceOperatorBase> po = new GroupReduceOperatorBase>(function, operatorInfo, logicalKeyPositions, name); - + po.setCombinable(true); - + po.setInput(input); po.setParallelism(this.getParallelism()); po.setCustomPartitioner(grouping.getCustomPartitioner()); - + SingleInputSemanticProperties props = new SingleInputSemanticProperties(); for (int keyField : logicalKeyPositions) { @@ -232,9 +224,9 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase + private static final class AggregatingUdf extends RichGroupReduceFunction implements GroupCombineFunction { private static final long serialVersionUID = 1L; - + private final int[] fieldPositions; - + private final AggregationFunction[] aggFunctions; - - + public AggregatingUdf(AggregationFunction[] aggFunctions, int[] fieldPositions) { Preconditions.checkNotNull(aggFunctions); Preconditions.checkNotNull(aggFunctions); Preconditions.checkArgument(aggFunctions.length == fieldPositions.length); - + this.aggFunctions = aggFunctions; this.fieldPositions = fieldPositions; } - @Override public void open(Configuration parameters) throws Exception { @@ -276,14 +266,14 @@ public void open(Configuration parameters) throws Exception { aggFunction.initializeAggregate(); } } - + @Override public void reduce(Iterable records, Collector out) { final AggregationFunction[] aggFunctions = this.aggFunctions; final int[] fieldPositions = this.fieldPositions; // aggregators are initialized from before - + T outT = null; for (T record : records) { outT = record; @@ -293,13 +283,13 @@ public void reduce(Iterable records, Collector out) { aggFunctions[i].aggregate(val); } } - + for (int i = 0; i < fieldPositions.length; i++) { Object aggVal = aggFunctions[i].getAggregate(); outT.setField(aggVal, fieldPositions[i]); aggFunctions[i].initializeAggregate(); } - + out.collect(outT); } @@ -307,6 +297,6 @@ public void reduce(Iterable records, Collector out) { public void combine(Iterable records, Collector out) { reduce(records, out); } - + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java index e6a46273883c7..d1ca742516d06 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java @@ -23,6 +23,10 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +/** + * Resulting {@link DataSet} of bulk iterations. + * @param + */ @Internal public class BulkIterationResultSet extends DataSet { @@ -41,8 +45,7 @@ public class BulkIterationResultSet extends DataSet { BulkIterationResultSet(ExecutionEnvironment context, TypeInformation type, IterativeDataSet iterationHead, - DataSet nextPartialSolution, DataSet terminationCriterion) - { + DataSet nextPartialSolution, DataSet terminationCriterion) { super(context, type); this.iterationHead = iterationHead; this.nextPartialSolution = nextPartialSolution; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 3c838cc45f75a..1d1ea8fd3a602 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -18,13 +18,6 @@ package org.apache.flink.api.java.operators; -import java.security.InvalidParameterException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.InvalidProgramException; @@ -33,6 +26,9 @@ import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -40,12 +36,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.common.operators.Keys.ExpressionKeys; -import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; -import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.translation.PlanBothUnwrappingCoGroupOperator; import org.apache.flink.api.java.operators.translation.PlanLeftUnwrappingCoGroupOperator; import org.apache.flink.api.java.operators.translation.PlanRightUnwrappingCoGroupOperator; @@ -53,13 +46,21 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + /** - * A {@link DataSet} that is the result of a CoGroup transformation. - * + * A {@link DataSet} that is the result of a CoGroup transformation. + * * @param The type of the first input DataSet of the CoGroup transformation. * @param The type of the second input DataSet of the CoGroup transformation. * @param The type of the result of the CoGroup transformation. - * + * * @see DataSet */ @Public @@ -69,33 +70,30 @@ public class CoGroupOperator extends TwoInputUdfOperator keys1; private final Keys keys2; - + private final String defaultName; - + private final List> groupSortKeyOrderFirst; private final List> groupSortKeyOrderSecond; - - private Partitioner customPartitioner; + private Partitioner customPartitioner; public CoGroupOperator(DataSet input1, DataSet input2, Keys keys1, Keys keys2, CoGroupFunction function, TypeInformation returnType, - Partitioner customPartitioner, String defaultName) - { + Partitioner customPartitioner, String defaultName) { this(input1, input2, keys1, keys2, function, returnType, null, null, customPartitioner, defaultName); } - + public CoGroupOperator(DataSet input1, DataSet input2, Keys keys1, Keys keys2, CoGroupFunction function, TypeInformation returnType, List> groupSortKeyOrderFirst, List> groupSortKeyOrderSecond, - Partitioner customPartitioner, String defaultName) - { + Partitioner customPartitioner, String defaultName) { super(input1, input2, returnType); this.function = function; this.customPartitioner = customPartitioner; this.defaultName = defaultName; - + this.groupSortKeyOrderFirst = groupSortKeyOrderFirst == null ? Collections.>emptyList() : groupSortKeyOrderFirst; this.groupSortKeyOrderSecond = groupSortKeyOrderSecond == null ? Collections.>emptyList() : groupSortKeyOrderSecond; @@ -126,7 +124,7 @@ public CoGroupOperator(DataSet input1, DataSet input2, Keys keys1, K UdfOperatorUtils.analyzeDualInputUdf(this, CoGroupFunction.class, defaultName, function, keys1, keys2); } - + @Override protected CoGroupFunction getFunction() { return function; @@ -138,16 +136,16 @@ public DualInputSemanticProperties getSemanticProperties() { DualInputSemanticProperties props = super.getSemanticProperties(); // offset semantic information by extracted key fields - if(props != null && + if (props != null && (this.keys1 instanceof SelectorFunctionKeys || this.keys2 instanceof SelectorFunctionKeys)) { int numFields1 = this.getInput1Type().getTotalFields(); int numFields2 = this.getInput2Type().getTotalFields(); int offset1 = (this.keys1 instanceof SelectorFunctionKeys) ? - ((SelectorFunctionKeys) this.keys1).getKeyType().getTotalFields() : 0; + ((SelectorFunctionKeys) this.keys1).getKeyType().getTotalFields() : 0; int offset2 = (this.keys2 instanceof SelectorFunctionKeys) ? - ((SelectorFunctionKeys) this.keys2).getKeyType().getTotalFields() : 0; + ((SelectorFunctionKeys) this.keys2).getKeyType().getTotalFields() : 0; props = SemanticPropUtil.addSourceFieldOffsets(props, numFields1, numFields2, offset1, offset2); } @@ -164,14 +162,14 @@ protected Keys getKeys1() { protected Keys getKeys2() { return this.keys2; } - + /** * Sets a custom partitioner for the CoGroup operation. The partitioner will be called on the join keys to determine * the partition a key should be assigned to. The partitioner is evaluated on both inputs in the * same way. - *

- * NOTE: A custom partitioner can only be used with single-field CoGroup keys, not with composite CoGroup keys. - * + * + *

NOTE: A custom partitioner can only be used with single-field CoGroup keys, not with composite CoGroup keys. + * * @param partitioner The custom partitioner to be used. * @return This CoGroup operator, to allow for function chaining. */ @@ -183,10 +181,10 @@ public CoGroupOperator withPartitioner(Partitioner partitioner) this.customPartitioner = getInput1().clean(partitioner); return this; } - + /** * Gets the custom partitioner used by this join, or {@code null}, if none is set. - * + * * @return The custom partitioner used by this join; */ public Partitioner getPartitioner() { @@ -196,7 +194,7 @@ public Partitioner getPartitioner() { @Override @Internal protected org.apache.flink.api.common.operators.base.CoGroupOperatorBase translateToDataFlow(Operator input1, Operator input2) { - + String name = getName() != null ? getName() : "CoGroup at " + defaultName; try { keys1.areCompatible(keys2); @@ -204,7 +202,7 @@ public Partitioner getPartitioner() { catch (IncompatibleKeysException e) { throw new InvalidProgramException("The types of the key fields do not match.", e); } - + final org.apache.flink.api.common.operators.base.CoGroupOperatorBase po; if (keys1 instanceof SelectorFunctionKeys @@ -246,8 +244,7 @@ else if (keys1 instanceof SelectorFunctionKeys) { po = translateSelectorFunctionCoGroupLeft(selectorKeys1, logicalKeyPositions2, function, getInput2Type(), getResultType(), name, input1, input2); } - else if ( keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.ExpressionKeys) - { + else if (keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.ExpressionKeys) { try { keys1.areCompatible(keys2); } catch (IncompatibleKeysException e) { @@ -256,12 +253,12 @@ else if ( keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.Expressi int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - + CoGroupOperatorBase> op = new CoGroupOperatorBase<>( function, new BinaryOperatorInformation<>(getInput1Type(), getInput2Type(), getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); - + op.setFirstInput(input1); op.setSecondInput(input2); po = op; @@ -269,11 +266,11 @@ else if ( keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.Expressi else { throw new UnsupportedOperationException("Unrecognized or incompatible key types."); } - + // configure shared characteristics po.setParallelism(getParallelism()); po.setCustomPartitioner(customPartitioner); - + if (groupSortKeyOrderFirst.size() > 0) { Ordering o = new Ordering(); for (Pair entry : groupSortKeyOrderFirst) { @@ -288,17 +285,15 @@ else if ( keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.Expressi } po.setGroupOrderForInputTwo(o); } - + return po; } - private static PlanBothUnwrappingCoGroupOperator translateSelectorFunctionCoGroup( SelectorFunctionKeys rawKeys1, SelectorFunctionKeys rawKeys2, CoGroupFunction function, TypeInformation outputType, String name, - Operator input1, Operator input2) - { + Operator input1, Operator input2) { @SuppressWarnings("unchecked") final SelectorFunctionKeys keys1 = (SelectorFunctionKeys) rawKeys1; @SuppressWarnings("unchecked") @@ -323,9 +318,8 @@ private static PlanRightUnwrappingCoGroupOperator rawKeys2, CoGroupFunction function, TypeInformation inputType1, TypeInformation outputType, String name, - Operator input1, Operator input2) - { - if(!inputType1.isTupleType()) { + Operator input1, Operator input2) { + if (!inputType1.isTupleType()) { throw new InvalidParameterException("Should not happen."); } @@ -333,7 +327,7 @@ private static PlanRightUnwrappingCoGroupOperator keys2 = (SelectorFunctionKeys) rawKeys2; final TypeInformation> typeInfoWithKey2 = KeyFunctions.createTypeWithKey(keys2); final Operator> keyedInput2 = KeyFunctions.appendKeyExtractor(input2, keys2); - + final PlanRightUnwrappingCoGroupOperator cogroup = new PlanRightUnwrappingCoGroupOperator<>( function, @@ -354,9 +348,8 @@ private static PlanLeftUnwrappingCoGroupOperator rawKeys1, int[] logicalKeyPositions2, CoGroupFunction function, TypeInformation inputType2, TypeInformation outputType, String name, - Operator input1, Operator input2) - { - if(!inputType2.isTupleType()) { + Operator input1, Operator input2) { + if (!inputType2.isTupleType()) { throw new InvalidParameterException("Should not happen."); } @@ -386,8 +379,9 @@ private static PlanLeftUnwrappingCoGroupOperator - * To continue the CoGroup transformation, select the grouping key of the first input {@link DataSet} by calling + * Intermediate step of a CoGroup transformation. + * + *

To continue the CoGroup transformation, select the grouping key of the first input {@link DataSet} by calling * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(KeySelector)}. * * @param The type of the first input DataSet of the CoGroup transformation. @@ -408,15 +402,17 @@ public CoGroupOperatorSets(DataSet input1, DataSet input2) { } /** - * Continues a CoGroup transformation.
- * Defines the {@link Tuple} fields of the first co-grouped {@link DataSet} that should be used as grouping keys.
- * Note: Fields can only be selected as grouping keys on Tuple DataSets.
+ * Continues a CoGroup transformation. + * + *

Defines the {@link Tuple} fields of the first co-grouped {@link DataSet} that should be used as grouping keys. + * + *

Note: Fields can only be selected as grouping keys on Tuple DataSets. * * * @param fields The indexes of the Tuple fields of the first co-grouped DataSets that should be used as keys. * @return An incomplete CoGroup transformation. * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int...)} to continue the CoGroup. - * + * * @see Tuple * @see DataSet */ @@ -425,10 +421,10 @@ public CoGroupOperatorSetsPredicate where(int... fields) { } /** - * Continues a CoGroup transformation.
- * Defines the fields of the first co-grouped {@link DataSet} that should be used as grouping keys. Fields - * are the names of member fields of the underlying type of the data set. + * Continues a CoGroup transformation. * + *

Defines the fields of the first co-grouped {@link DataSet} that should be used as grouping keys. Fields + * are the names of member fields of the underlying type of the data set. * * @param fields The fields of the first co-grouped DataSets that should be used as keys. * @return An incomplete CoGroup transformation. @@ -442,14 +438,15 @@ public CoGroupOperatorSetsPredicate where(String... fields) { } /** - * Continues a CoGroup transformation and defines a {@link KeySelector} function for the first co-grouped {@link DataSet}.
- * The KeySelector function is called for each element of the first DataSet and extracts a single - * key value on which the DataSet is grouped.
- * + * Continues a CoGroup transformation and defines a {@link KeySelector} function for the first co-grouped {@link DataSet}. + * + *

The KeySelector function is called for each element of the first DataSet and extracts a single + * key value on which the DataSet is grouped. + * * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is grouped. - * @return An incomplete CoGroup transformation. + * @return An incomplete CoGroup transformation. * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int...)} to continue the CoGroup. - * + * * @see KeySelector * @see DataSet */ @@ -461,8 +458,9 @@ public CoGroupOperatorSetsPredicate where(KeySelector keyExtractor) { // ---------------------------------------------------------------------------------------- /** - * Intermediate step of a CoGroup transformation.
- * To continue the CoGroup transformation, select the grouping key of the second input {@link DataSet} by calling + * Intermediate step of a CoGroup transformation. + * + *

To continue the CoGroup transformation, select the grouping key of the second input {@link DataSet} by calling * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(KeySelector)}. * */ @@ -483,10 +481,10 @@ private CoGroupOperatorSetsPredicate(Keys keys1) { } /** - * Continues a CoGroup transformation and defines the {@link Tuple} fields of the second co-grouped - * {@link DataSet} that should be used as grouping keys.
- * Note: Fields can only be selected as grouping keys on Tuple DataSets.
+ * Continues a CoGroup transformation and defines the {@link Tuple} fields of the second co-grouped + * {@link DataSet} that should be used as grouping keys. * + *

Note: Fields can only be selected as grouping keys on Tuple DataSets. * * @param fields The indexes of the Tuple fields of the second co-grouped DataSet that should be used as keys. * @return An incomplete CoGroup transformation. @@ -498,8 +496,7 @@ public CoGroupOperatorWithoutFunction equalTo(int... fields) { /** * Continues a CoGroup transformation and defines the fields of the second co-grouped - * {@link DataSet} that should be used as grouping keys.
- * + * {@link DataSet} that should be used as grouping keys. * * @param fields The fields of the first co-grouped DataSets that should be used as keys. * @return An incomplete CoGroup transformation. @@ -510,12 +507,13 @@ public CoGroupOperatorWithoutFunction equalTo(String... fields) { } /** - * Continues a CoGroup transformation and defines a {@link KeySelector} function for the second co-grouped {@link DataSet}.
- * The KeySelector function is called for each element of the second DataSet and extracts a single - * key value on which the DataSet is grouped.
- * + * Continues a CoGroup transformation and defines a {@link KeySelector} function for the second co-grouped {@link DataSet}. + * + *

The KeySelector function is called for each element of the second DataSet and extracts a single + * key value on which the DataSet is grouped. + * * @param keyExtractor The KeySelector function which extracts the key values from the second DataSet on which it is grouped. - * @return An incomplete CoGroup transformation. + * @return An incomplete CoGroup transformation. * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.common.functions.CoGroupFunction)} to finalize the CoGroup transformation. */ public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtractor) { @@ -524,8 +522,9 @@ public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtracto } /** - * Intermediate step of a CoGroup transformation.
- * To continue the CoGroup transformation, provide a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} by calling + * Intermediate step of a CoGroup transformation. + * + *

To continue the CoGroup transformation, provide a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} by calling * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.common.functions.CoGroupFunction)}. * */ @@ -539,22 +538,25 @@ private CoGroupOperatorWithoutFunction createCoGroupOperator(Keys keys2) { } try { keys1.areCompatible(keys2); - } catch(IncompatibleKeysException ike) { + } catch (IncompatibleKeysException ike) { throw new InvalidProgramException("The pair of co-group keys are not compatible with each other.", ike); } return new CoGroupOperatorWithoutFunction(keys2); } - + // ------------------------------------------------------------------------------------ + /** + * An intermediate {@link CoGroupOperator} without {@link CoGroupFunction} applied yet. + */ public final class CoGroupOperatorWithoutFunction { - + private final Keys keys2; - + private final List> groupSortKeyOrderFirst; private final List> groupSortKeyOrderSecond; - + private Partitioner customPartitioner; private CoGroupOperatorWithoutFunction(Keys keys2) { @@ -566,18 +568,18 @@ private CoGroupOperatorWithoutFunction(Keys keys2) { } this.keys2 = keys2; - + this.groupSortKeyOrderFirst = new ArrayList<>(); this.groupSortKeyOrderSecond = new ArrayList<>(); } - + /** * Sets a custom partitioner for the CoGroup operation. The partitioner will be called on the join keys to determine * the partition a key should be assigned to. The partitioner is evaluated on both inputs in the * same way. - *

- * NOTE: A custom partitioner can only be used with single-field CoGroup keys, not with composite CoGroup keys. - * + * + *

NOTE: A custom partitioner can only be used with single-field CoGroup keys, not with composite CoGroup keys. + * * @param partitioner The custom partitioner to be used. * @return This CoGroup operator, to allow for function chaining. */ @@ -589,10 +591,10 @@ public CoGroupOperatorWithoutFunction withPartitioner(Partitioner partitioner this.customPartitioner = input1.clean(partitioner); return this; } - + /** * Gets the custom partitioner used by this join, or {@code null}, if none is set. - * + * * @return The custom partitioner used by this join; */ public Partitioner getPartitioner() { @@ -600,12 +602,13 @@ public Partitioner getPartitioner() { } /** - * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with identical keys.
- * Each CoGroupFunction call returns an arbitrary number of keys. - * + * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with identical keys. + * + *

Each CoGroupFunction call returns an arbitrary number of keys. + * * @param function The CoGroupFunction that is called for all groups of elements with identical keys. * @return An CoGroupOperator that represents the co-grouped result DataSet. - * + * * @see org.apache.flink.api.common.functions.RichCoGroupFunction * @see DataSet */ @@ -615,26 +618,28 @@ public CoGroupOperator with(CoGroupFunction function) } TypeInformation returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType(), Utils.getCallLocationName(), true); - + return new CoGroupOperator<>(input1, input2, keys1, keys2, input1.clean(function), returnType, groupSortKeyOrderFirst, groupSortKeyOrderSecond, customPartitioner, Utils.getCallLocationName()); } - + // -------------------------------------------------------------------------------- // Group Operations // -------------------------------------------------------------------------------- - + /** * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the first input on the - * specified field in the specified {@link Order}.
- * Note: Only groups of Tuple elements and Pojos can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(int, Order)} calls. - * + * specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple elements and Pojos can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(int, Order)} calls. + * * @param field The Tuple field on which the group is sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see org.apache.flink.api.java.tuple.Tuple * @see Order */ @@ -642,24 +647,26 @@ public CoGroupOperatorWithoutFunction sortFirstGroup(int field, Order order) { ExpressionKeys ek = new ExpressionKeys<>(field, input1.getType()); int[] groupOrderKeys = ek.computeLogicalKeyPositions(); - + for (int key : groupOrderKeys) { this.groupSortKeyOrderFirst.add(new ImmutablePair<>(key, order)); } - + return this; } - + /** * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the second input on the - * specified field in the specified {@link Order}.
- * Note: Only groups of Tuple elements and Pojos can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(int, Order)} calls. - * + * specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple elements and Pojos can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(int, Order)} calls. + * * @param field The Tuple field on which the group is sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see org.apache.flink.api.java.tuple.Tuple * @see Order */ @@ -667,57 +674,59 @@ public CoGroupOperatorWithoutFunction sortSecondGroup(int field, Order order) { ExpressionKeys ek = new ExpressionKeys<>(field, input2.getType()); int[] groupOrderKeys = ek.computeLogicalKeyPositions(); - + for (int key : groupOrderKeys) { this.groupSortKeyOrderSecond.add(new ImmutablePair<>(key, order)); } - + return this; } - + /** * Sorts Pojo or {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the first input on the - * specified field in the specified {@link Order}.
- * Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(String, Order)} calls. - * + * specified field in the specified {@link Order}. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(String, Order)} calls. + * * @param fieldExpression The expression to the field on which the group is to be sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see Order */ public CoGroupOperatorWithoutFunction sortFirstGroup(String fieldExpression, Order order) { ExpressionKeys ek = new ExpressionKeys<>(fieldExpression, input1.getType()); int[] groupOrderKeys = ek.computeLogicalKeyPositions(); - + for (int key : groupOrderKeys) { this.groupSortKeyOrderFirst.add(new ImmutablePair<>(key, order)); } - + return this; } - + /** * Sorts Pojo or {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the second input on the - * specified field in the specified {@link Order}.
- * Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(String, Order)} calls. - * + * specified field in the specified {@link Order}. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(String, Order)} calls. + * * @param fieldExpression The expression to the field on which the group is to be sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see Order */ public CoGroupOperatorWithoutFunction sortSecondGroup(String fieldExpression, Order order) { ExpressionKeys ek = new ExpressionKeys<>(fieldExpression, input2.getType()); int[] groupOrderKeys = ek.computeLogicalKeyPositions(); - + for (int key : groupOrderKeys) { this.groupSortKeyOrderSecond.add(new ImmutablePair<>(key, order)); } - + return this; } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java index 9a6d80090a4d6..e607ac188cb7e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java @@ -15,6 +15,7 @@ * 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.annotation.Internal; @@ -23,19 +24,19 @@ import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; /** - * A {@link DataSet} that is the result of a CoGroup transformation. - * + * A {@link DataSet} that is the result of a CoGroup transformation. + * * @param The type of the first input DataSet of the CoGroup transformation. * @param The type of the second input DataSet of the CoGroup transformation. * @param The type of the result of the CoGroup transformation. - * + * * @see DataSet */ @Internal @@ -95,10 +96,12 @@ protected Keys getKeys2() { int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - CoGroupRawOperatorBase> po - = new CoGroupRawOperatorBase>( - function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), - getResultType()), logicalKeyPositions1, logicalKeyPositions2, name); + CoGroupRawOperatorBase> po = new CoGroupRawOperatorBase<>( + function, + new BinaryOperatorInformation<>(getInput1Type(), getInput2Type(), getResultType()), + logicalKeyPositions1, + logicalKeyPositions2, + name); // set inputs po.setFirstInput(input1); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 98956bbcbd2fe..711c215181853 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -18,8 +18,6 @@ package org.apache.flink.api.java.operators; -import java.util.Arrays; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -34,12 +32,37 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.SemanticPropUtil; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple10; +import org.apache.flink.api.java.tuple.Tuple11; +import org.apache.flink.api.java.tuple.Tuple12; +import org.apache.flink.api.java.tuple.Tuple13; +import org.apache.flink.api.java.tuple.Tuple14; +import org.apache.flink.api.java.tuple.Tuple15; +import org.apache.flink.api.java.tuple.Tuple16; +import org.apache.flink.api.java.tuple.Tuple17; +import org.apache.flink.api.java.tuple.Tuple18; +import org.apache.flink.api.java.tuple.Tuple19; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple20; +import org.apache.flink.api.java.tuple.Tuple21; +import org.apache.flink.api.java.tuple.Tuple22; +import org.apache.flink.api.java.tuple.Tuple23; +import org.apache.flink.api.java.tuple.Tuple24; +import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.tuple.Tuple6; +import org.apache.flink.api.java.tuple.Tuple7; +import org.apache.flink.api.java.tuple.Tuple8; +import org.apache.flink.api.java.tuple.Tuple9; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; -//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator -import org.apache.flink.api.java.tuple.*; +import java.util.Arrays; /** * A {@link DataSet} that is the result of a Cross transformation. @@ -54,17 +77,16 @@ public class CrossOperator extends TwoInputUdfOperator> { private final CrossFunction function; - + private final String defaultName; - + private final CrossHint hint; public CrossOperator(DataSet input1, DataSet input2, CrossFunction function, TypeInformation returnType, CrossHint hint, - String defaultName) - { + String defaultName) { super(input1, input2, returnType); this.function = function; @@ -83,21 +105,20 @@ protected CrossFunction getFunction() { public CrossHint getCrossHint() { return hint; } - + private String getDefaultName() { return defaultName; } @Override - protected org.apache.flink.api.common.operators.base.CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { - - String name = getName() != null ? getName() : "Cross at "+defaultName; + protected CrossOperatorBase> translateToDataFlow(Operator input1, Operator input2) { + + String name = getName() != null ? getName() : "Cross at " + defaultName; // create operator CrossOperatorBase> po = - new CrossOperatorBase>(function, + new CrossOperatorBase>(function, new BinaryOperatorInformation(getInput1Type(), getInput2Type(), getResultType()), name); - po.setFirstInput(input1); po.setSecondInput(input2); @@ -112,12 +133,13 @@ protected org.apache.flink.api.common.operators.base.CrossOperatorBase - * It also represents the {@link DataSet} that is the result of a Cross transformation. - * + * A Cross transformation that wraps pairs of crossed elements into {@link Tuple2}. + * + *

It also represents the {@link DataSet} that is the result of a Cross transformation. + * * @param The type of the first input DataSet of the Cross transformation. * @param The type of the second input DataSet of the Cross transformation. - * + * * @see Tuple2 * @see DataSet */ @@ -133,12 +155,13 @@ public DefaultCross(DataSet input1, DataSet input2, CrossHint hint, Stri } /** - * Finalizes a Cross transformation by applying a {@link CrossFunction} to each pair of crossed elements.
- * Each CrossFunction call returns exactly one element. - * + * Finalizes a Cross transformation by applying a {@link CrossFunction} to each pair of crossed elements. + * + *

Each CrossFunction call returns exactly one element. + * * @param function The CrossFunction that is called for each pair of crossed elements. * @return An CrossOperator that represents the crossed result DataSet - * + * * @see CrossFunction * @see DataSet */ @@ -151,13 +174,14 @@ public CrossOperator with(CrossFunction function) { return new CrossOperator(getInput1(), getInput2(), clean(function), returnType, getCrossHint(), Utils.getCallLocationName()); } - + /** - * Initiates a ProjectCross transformation and projects the first cross input
- * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first cross input is not a Tuple DataSet, no parameters should be passed.
+ * Initiates a ProjectCross transformation and projects the first cross input. + * + *

If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first cross input is not a Tuple DataSet, no parameters should be passed. * - * Fields of the first and second input can be added by chaining the method calls of + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectSecond(int...)}. * @@ -176,13 +200,14 @@ public ProjectCross projectFirst(int... firstFi return new CrossProjection(getInput1(), getInput2(), firstFieldIndexes, null, getCrossHint()) .projectTupleX(); } - + /** - * Initiates a ProjectCross transformation and projects the second cross input
- * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second cross input is not a Tuple DataSet, no parameters should be passed.
+ * Initiates a ProjectCross transformation and projects the second cross input. + * + *

If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second cross input is not a Tuple DataSet, no parameters should be passed. * - * Fields of the first and second input can be added by chaining the method calls of + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectSecond(int...)}. * @@ -201,13 +226,14 @@ public ProjectCross projectSecond(int... second return new CrossProjection(getInput1(), getInput2(), null, secondFieldIndexes, getCrossHint()) .projectTupleX(); } - + } /** * A Cross transformation that projects crossing elements or fields of crossing {@link Tuple Tuples} - * into result {@link Tuple Tuples}.
- * It also represents the {@link DataSet} that is the result of a Cross transformation. + * into result {@link Tuple Tuples}. + * + *

It also represents the {@link DataSet} that is the result of a Cross transformation. * * @param The type of the first input DataSet of the Cross transformation. * @param The type of the second input DataSet of the Cross transformation. @@ -218,26 +244,24 @@ public ProjectCross projectSecond(int... second */ @Public public static final class ProjectCross extends CrossOperator { - + private CrossProjection crossProjection; protected ProjectCross(DataSet input1, DataSet input2, int[] fields, boolean[] isFromFirst, - TupleTypeInfo returnType, CrossHint hint) - { + TupleTypeInfo returnType, CrossHint hint) { super(input1, input2, new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, "unknown"); - + crossProjection = null; } - + protected ProjectCross(DataSet input1, DataSet input2, int[] fields, boolean[] isFromFirst, - TupleTypeInfo returnType, CrossProjection crossProjection, CrossHint hint) - { + TupleTypeInfo returnType, CrossProjection crossProjection, CrossHint hint) { super(input1, input2, new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, "unknown"); - + this.crossProjection = crossProjection; } @@ -245,17 +269,18 @@ protected ProjectCross(DataSet input1, DataSet input2, int[] fields, boo protected ProjectCrossFunction getFunction() { return (ProjectCrossFunction) super.getFunction(); } - + /** - * Continues a ProjectCross transformation and adds fields of the first cross input to the projection.
- * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first cross input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectCross transformation and adds fields of the first cross input to the projection. * - * Additional fields of the first and second input can be added by chaining the method calls of + *

If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first cross input is not a Tuple DataSet, no parameters should be passed. + * + *

Additional fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectSecond(int...)}. * - * Note: With the current implementation, the Project transformation looses type information. + *

Note: With the current implementation, the Project transformation looses type information. * * @param firstFieldIndexes If the first input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. @@ -269,16 +294,17 @@ protected ProjectCrossFunction getFunction() { @SuppressWarnings("hiding") public ProjectCross projectFirst(int... firstFieldIndexes) { crossProjection = crossProjection.projectFirst(firstFieldIndexes); - + return crossProjection.projectTupleX(); } /** - * Continues a ProjectCross transformation and adds fields of the second cross input to the projection.
- * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second cross input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectCross transformation and adds fields of the second cross input to the projection. * - * Additional fields of the first and second input can be added by chaining the method calls of + *

If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second cross input is not a Tuple DataSet, no parameters should be passed. + * + *

Additional fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectSecond(int...)}. * @@ -296,7 +322,7 @@ public ProjectCross projectFirst(int... firstFi @SuppressWarnings("hiding") public ProjectCross projectSecond(int... secondFieldIndexes) { crossProjection = crossProjection.projectSecond(secondFieldIndexes); - + return crossProjection.projectTupleX(); } @@ -307,15 +333,15 @@ public ProjectCross projectSecond(int... second @Deprecated @PublicEvolving public CrossOperator types(Class... types) { - TupleTypeInfo typeInfo = (TupleTypeInfo)this.getResultType(); + TupleTypeInfo typeInfo = (TupleTypeInfo) this.getResultType(); - if(types.length != typeInfo.getArity()) { + if (types.length != typeInfo.getArity()) { throw new InvalidProgramException("Provided types do not match projection."); } - for (int i=0; i typeClass = types[i]; if (!typeClass.equals(typeInfo.getTypeAt(i).getTypeClass())) { - throw new InvalidProgramException("Provided type "+typeClass.getSimpleName()+" at position "+i+" does not match projection"); + throw new InvalidProgramException("Provided type " + typeClass.getSimpleName() + " at position " + i + " does not match projection"); } } return (CrossOperator) this; @@ -330,7 +356,7 @@ public CrossOperator withForwardedFieldsFirst(String... forwardedFi public CrossOperator withForwardedFieldsSecond(String... forwardedFieldsSecond) { throw new InvalidProgramException("The semantic properties (forwarded fields) are automatically calculated."); } - + @Override protected DualInputSemanticProperties extractSemanticAnnotationsFromUdf(Class udfClass) { // we do not extract anything, but construct the properties from the projection @@ -339,6 +365,12 @@ protected DualInputSemanticProperties extractSemanticAnnotationsFromUdf(Class } } + /** + * @see ProjectCross + * @param + * @param + * @param + */ @Internal public static final class ProjectCrossFunction implements CrossFunction { @@ -346,7 +378,7 @@ public static final class ProjectCrossFunction implemen private final int[] fields; private final boolean[] isFromFirst; - + private final R outTuple; /** @@ -360,7 +392,7 @@ public static final class ProjectCrossFunction implemen */ private ProjectCrossFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) { - if(fields.length != isFromFirst.length) { + if (fields.length != isFromFirst.length) { throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); } this.fields = fields; @@ -369,16 +401,16 @@ private ProjectCrossFunction(int[] fields, boolean[] isFromFirst, R outTupleInst } public R cross(T1 in1, T2 in2) { - for(int i=0; i= 0) { - outTuple.setField(((Tuple)in1).getField(fields[i]), i); + for (int i = 0; i < fields.length; i++) { + if (isFromFirst[i]) { + if (fields[i] >= 0) { + outTuple.setField(((Tuple) in1).getField(fields[i]), i); } else { outTuple.setField(in1, i); } } else { - if(fields[i] >= 0) { - outTuple.setField(((Tuple)in2).getField(fields[i]), i); + if (fields[i] >= 0) { + outTuple.setField(((Tuple) in2).getField(fields[i]), i); } else { outTuple.setField(in2, i); } @@ -394,12 +426,17 @@ protected int[] getFields() { protected boolean[] getIsFromFirst() { return isFromFirst; } - + } + /** + * @see ProjectCross + * @param + * @param + */ @Internal public static final class CrossProjection { - + private final DataSet ds1; private final DataSet ds2; @@ -408,7 +445,7 @@ public static final class CrossProjection { private final int numFieldsDs1; private final int numFieldsDs2; - + private final CrossHint hint; public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes, int[] secondFieldIndexes, CrossHint hint) { @@ -420,15 +457,15 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes boolean isFirstTuple; boolean isSecondTuple; - if(ds1.getType() instanceof TupleTypeInfo) { - numFieldsDs1 = ((TupleTypeInfo)ds1.getType()).getArity(); + if (ds1.getType() instanceof TupleTypeInfo) { + numFieldsDs1 = ((TupleTypeInfo) ds1.getType()).getArity(); isFirstTuple = true; } else { numFieldsDs1 = 1; isFirstTuple = false; } - if(ds2.getType() instanceof TupleTypeInfo) { - numFieldsDs2 = ((TupleTypeInfo)ds2.getType()).getArity(); + if (ds2.getType() instanceof TupleTypeInfo) { + numFieldsDs2 = ((TupleTypeInfo) ds2.getType()).getArity(); isSecondTuple = true; } else { numFieldsDs2 = 1; @@ -438,13 +475,13 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes boolean isTuple; boolean firstInput; - if(firstFieldIndexes != null && secondFieldIndexes == null) { + if (firstFieldIndexes != null && secondFieldIndexes == null) { // index array for first input is provided firstInput = true; isTuple = isFirstTuple; this.fieldIndexes = firstFieldIndexes; - if(this.fieldIndexes.length == 0) { + if (this.fieldIndexes.length == 0) { // no indexes provided, treat tuple as regular object isTuple = false; } @@ -454,7 +491,7 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes isTuple = isSecondTuple; this.fieldIndexes = secondFieldIndexes; - if(this.fieldIndexes.length == 0) { + if (this.fieldIndexes.length == 0) { // no indexes provided, treat tuple as regular object isTuple = false; } @@ -464,22 +501,22 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes throw new IllegalArgumentException("You must provide at most one field index array."); } - if(!isTuple && this.fieldIndexes.length != 0) { + if (!isTuple && this.fieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectFirst() (or projectSecond()) without arguments to include it."); - } else if(this.fieldIndexes.length > 22) { + } else if (this.fieldIndexes.length > 22) { throw new IllegalArgumentException("You may select only up to twenty-two (22) fields."); } - if(isTuple) { + if (isTuple) { this.isFieldInFirst = new boolean[this.fieldIndexes.length]; // check field indexes and adapt to position in tuple int maxFieldIndex = firstInput ? numFieldsDs1 : numFieldsDs2; - for(int i=0; i ds1, DataSet ds2, int[] firstFieldIndexes } /** - * Continues a ProjectCross transformation and adds fields of the first cross input.
- * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first cross input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectCross transformation and adds fields of the first cross input. * - * Fields of the first and second input can be added by chaining the method calls of + *

If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first cross input is not a Tuple DataSet, no parameters should be passed. + * + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectSecond(int...)}. * @@ -515,30 +553,30 @@ protected CrossProjection projectFirst(int... firstFieldIndexes) { boolean isFirstTuple; - if(ds1.getType() instanceof TupleTypeInfo && firstFieldIndexes.length > 0) { + if (ds1.getType() instanceof TupleTypeInfo && firstFieldIndexes.length > 0) { isFirstTuple = true; } else { isFirstTuple = false; } - if(!isFirstTuple && firstFieldIndexes.length != 0) { + if (!isFirstTuple && firstFieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectFirst() without arguments to include it."); - } else if(firstFieldIndexes.length > (22 - this.fieldIndexes.length)) { + } else if (firstFieldIndexes.length > (22 - this.fieldIndexes.length)) { // to many field indexes provided throw new IllegalArgumentException("You may select only up to twenty-two (22) fields in total."); } int offset = this.fieldIndexes.length; - if(isFirstTuple) { + if (isFirstTuple) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + firstFieldIndexes.length); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + firstFieldIndexes.length); // copy field indexes int maxFieldIndex = numFieldsDs1; - for(int i = 0; i < firstFieldIndexes.length; i++) { + for (int i = 0; i < firstFieldIndexes.length; i++) { // check if indexes in range Preconditions.checkElementIndex(firstFieldIndexes[i], maxFieldIndex); @@ -559,11 +597,12 @@ protected CrossProjection projectFirst(int... firstFieldIndexes) { } /** - * Continues a ProjectCross transformation and adds fields of the second cross input.
- * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second cross input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectCross transformation and adds fields of the second cross input. + * + *

If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second cross input is not a Tuple DataSet, no parameters should be passed. * - * Fields of the first and second input can be added by chaining the method calls of + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectSecond(int...)}. * @@ -581,30 +620,30 @@ protected CrossProjection projectSecond(int... secondFieldIndexes) { boolean isSecondTuple; - if(ds2.getType() instanceof TupleTypeInfo && secondFieldIndexes.length > 0) { + if (ds2.getType() instanceof TupleTypeInfo && secondFieldIndexes.length > 0) { isSecondTuple = true; } else { isSecondTuple = false; } - if(!isSecondTuple && secondFieldIndexes.length != 0) { + if (!isSecondTuple && secondFieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectSecond() without arguments to include it."); - } else if(secondFieldIndexes.length > (22 - this.fieldIndexes.length)) { + } else if (secondFieldIndexes.length > (22 - this.fieldIndexes.length)) { // to many field indexes provided throw new IllegalArgumentException("You may select only up to twenty-two (22) fields in total."); } int offset = this.fieldIndexes.length; - if(isSecondTuple) { + if (isSecondTuple) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + secondFieldIndexes.length); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + secondFieldIndexes.length); // copy field indexes int maxFieldIndex = numFieldsDs2; - for(int i = 0; i < secondFieldIndexes.length; i++) { + for (int i = 0; i < secondFieldIndexes.length; i++) { // check if indexes in range Preconditions.checkElementIndex(secondFieldIndexes[i], maxFieldIndex); @@ -623,7 +662,7 @@ protected CrossProjection projectSecond(int... secondFieldIndexes) { return this; } - + // -------------------------------------------------------------------------------------------- // The following lines are generated. // -------------------------------------------------------------------------------------------- @@ -632,8 +671,8 @@ protected CrossProjection projectSecond(int... secondFieldIndexes) { /** * Chooses a projectTupleX according to the length of - * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#fieldIndexes} - * + * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#fieldIndexes}. + * * @return The projected DataSet. */ @SuppressWarnings("unchecked") @@ -673,10 +712,10 @@ public ProjectCross projectTupleX() { } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -688,10 +727,10 @@ public ProjectCross> projectTuple1() { } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -703,10 +742,10 @@ public ProjectCross> projectTuple2() { } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -718,10 +757,10 @@ public ProjectCross> projectTuple3() { } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -733,10 +772,10 @@ public ProjectCross> projectTupl } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -748,10 +787,10 @@ public ProjectCross> pro } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -763,10 +802,10 @@ public ProjectCross ProjectCross ProjectCross ProjectCross ProjectCross ProjectCross ProjectCross ProjectCross ProjectCross } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -898,10 +937,10 @@ public Project } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -913,10 +952,10 @@ public Pr } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -928,10 +967,10 @@ public [] extractFieldTypes(int[] fields) { TypeInformation[] fieldTypes = new TypeInformation[fields.length]; - for(int i=0; i typeInfo; - if(isFieldInFirst[i]) { - if(fields[i] >= 0) { - typeInfo = ((TupleTypeInfo)ds1.getType()).getTypeAt(fields[i]); + if (isFieldInFirst[i]) { + if (fields[i] >= 0) { + typeInfo = ((TupleTypeInfo) ds1.getType()).getTypeAt(fields[i]); } else { typeInfo = ds1.getType(); } } else { - if(fields[i] >= 0) { - typeInfo = ((TupleTypeInfo)ds2.getType()).getTypeAt(fields[i]); + if (fields[i] >= 0) { + typeInfo = ((TupleTypeInfo) ds2.getType()).getTypeAt(fields[i]); } else { typeInfo = ds2.getType(); } @@ -1083,10 +1122,10 @@ private TypeInformation[] extractFieldTypes(int[] fields) { // -------------------------------------------------------------------------------------------- @Internal - public static final class DefaultCrossFunction implements CrossFunction> { + private static final class DefaultCrossFunction implements CrossFunction> { private static final long serialVersionUID = 1L; - + private final Tuple2 outTuple = new Tuple2(); @Override diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CustomUnaryOperation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CustomUnaryOperation.java index 8bbdfd18dffb5..164a944f95e2a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CustomUnaryOperation.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CustomUnaryOperation.java @@ -24,12 +24,12 @@ /** * * @param The type of the data set consumed by this operator. - * @param The type of the data set produced by this operator. + * @param The type of the data set produced by this operator. */ @Public public interface CustomUnaryOperation { - + void setInput(DataSet inputData); - + DataSet createResult(); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java index ee5ab2e7217b8..bac85ee92f8cd 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java @@ -33,23 +33,27 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.typeinfo.NothingTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; import org.apache.flink.api.java.DataSet; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Preconditions; import java.util.Arrays; +/** + * An operation that allows storing data results. + * @param + */ @Public public class DataSink { - + private final OutputFormat format; - + private final TypeInformation type; - + private final DataSet data; - + private String name; - + private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; private ResourceSpec minResources = ResourceSpec.DEFAULT; @@ -72,14 +76,12 @@ public DataSink(DataSet data, OutputFormat format, TypeInformation type if (data == null) { throw new IllegalArgumentException("The data set must not be null."); } - - + this.format = format; this.data = data; this.type = type; } - @Internal public OutputFormat getFormat() { return format; @@ -96,7 +98,7 @@ public DataSet getDataSet() { } /** - * Pass a configuration to the OutputFormat + * Pass a configuration to the OutputFormat. * @param parameters Configuration parameters */ public DataSink withParameters(Configuration parameters) { @@ -106,9 +108,11 @@ public DataSink withParameters(Configuration parameters) { /** * Sorts each local partition of a {@link org.apache.flink.api.java.tuple.Tuple} data set - * on the specified field in the specified {@link Order} before it is emitted by the output format.
- * Note: Only tuple data sets can be sorted using integer field indices.
- * The tuple data set can be sorted on multiple fields in different orders + * on the specified field in the specified {@link Order} before it is emitted by the output format. + * + *

Note: Only tuple data sets can be sorted using integer field indices. + * + *

The tuple data set can be sorted on multiple fields in different orders * by chaining {@link #sortLocalOutput(int, Order)} calls. * * @param field The Tuple field on which the data set is locally sorted. @@ -132,7 +136,7 @@ public DataSink sortLocalOutput(int field, Order order) { throw new InvalidProgramException("Selected sort key is not a sortable type"); } - if(this.sortKeyPositions == null) { + if (this.sortKeyPositions == null) { // set sorting info this.sortKeyPositions = flatKeys; this.sortOrders = new Order[flatKeys.length]; @@ -144,9 +148,9 @@ public DataSink sortLocalOutput(int field, Order order) { this.sortKeyPositions = Arrays.copyOf(this.sortKeyPositions, newLength); this.sortOrders = Arrays.copyOf(this.sortOrders, newLength); - for(int i=0; i sortLocalOutput(int field, Order order) { /** * Sorts each local partition of a data set on the field(s) specified by the field expression - * in the specified {@link Order} before it is emitted by the output format.
- * Note: Non-composite types can only be sorted on the full element which is specified by - * a wildcard expression ("*" or "_").
- * Data sets of composite types (Tuple or Pojo) can be sorted on multiple fields in different orders + * in the specified {@link Order} before it is emitted by the output format. + * + *

Note: Non-composite types can only be sorted on the full element which is specified by + * a wildcard expression ("*" or "_"). + * + *

Data sets of composite types (Tuple or Pojo) can be sorted on multiple fields in different orders * by chaining {@link #sortLocalOutput(String, Order)} calls. * * @param fieldExpression The field expression for the field(s) on which the data set is locally sorted. @@ -189,7 +195,7 @@ public DataSink sortLocalOutput(String fieldExpression, Order order) { orders = new Order[numFields]; Arrays.fill(orders, order); - if(this.sortKeyPositions == null) { + if (this.sortKeyPositions == null) { // set sorting info this.sortKeyPositions = fields; this.sortOrders = orders; @@ -199,9 +205,9 @@ public DataSink sortLocalOutput(String fieldExpression, Order order) { int newLength = oldLength + numFields; this.sortKeyPositions = Arrays.copyOf(this.sortKeyPositions, newLength); this.sortOrders = Arrays.copyOf(this.sortOrders, newLength); - for(int i=0; i sortLocalOutput(String fieldExpression, Order order) { public Configuration getParameters() { return this.parameters; } - + // -------------------------------------------------------------------------------------------- - + public DataSink name(String name) { this.name = name; return this; } - + // -------------------------------------------------------------------------------------------- - + protected GenericDataSinkBase translateToDataFlow(Operator input) { // select the name (or create a default one) String name = this.name != null ? this.name : this.format.toString(); @@ -231,11 +237,11 @@ protected GenericDataSinkBase translateToDataFlow(Operator input) { // set input sink.setInput(input); // set parameters - if(this.parameters != null) { + if (this.parameters != null) { sink.getParameters().addAll(this.parameters); } // set parallelism - if(this.parallelism > 0) { + if (this.parallelism > 0) { // use specified parallelism sink.setParallelism(this.parallelism); } else { @@ -243,34 +249,34 @@ protected GenericDataSinkBase translateToDataFlow(Operator input) { sink.setParallelism(input.getParallelism()); } - if(this.sortKeyPositions != null) { + if (this.sortKeyPositions != null) { // configure output sorting Ordering ordering = new Ordering(); - for(int i=0; i" : this.name) + "' (" + this.format.toString() + ")"; } - + /** * Returns the parallelism of this data sink. - * + * * @return The parallelism of this data sink. */ public int getParallelism() { return this.parallelism; } - + /** * Sets the parallelism for this data sink. * The degree must be 1 or more. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java index af6f65bb3c94a..8ae1c7d25b4c5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java @@ -34,7 +34,7 @@ * An operation that creates a new data set (data source). The operation acts as the * data set on which to apply further transformations. It encapsulates additional * configuration parameters, to customize the execution. - * + * * @param The type of the elements produced by this data source. */ @Public @@ -49,25 +49,25 @@ public class DataSource extends Operator> { private SplitDataProperties splitDataProperties; // -------------------------------------------------------------------------------------------- - + /** * Creates a new data source. - * + * * @param context The environment in which the data source gets executed. * @param inputFormat The input format that the data source executes. * @param type The type of the elements produced by this input format. */ public DataSource(ExecutionEnvironment context, InputFormat inputFormat, TypeInformation type, String dataSourceLocationName) { super(context, type); - + this.dataSourceLocationName = dataSourceLocationName; - + if (inputFormat == null) { throw new IllegalArgumentException("The input format may not be null."); } - + this.inputFormat = inputFormat; - + if (inputFormat instanceof NonParallelInput) { this.parallelism = 1; } @@ -75,23 +75,23 @@ public DataSource(ExecutionEnvironment context, InputFormat inputFormat, /** * Gets the input format that is executed by this data source. - * + * * @return The input format that is executed by this data source. */ @Internal public InputFormat getInputFormat() { return this.inputFormat; } - + /** - * Pass a configuration to the InputFormat + * Pass a configuration to the InputFormat. * @param parameters Configuration parameters */ public DataSource withParameters(Configuration parameters) { this.parameters = parameters; return this; } - + /** * @return Configuration for the InputFormat. */ @@ -99,15 +99,15 @@ public Configuration getParameters() { return this.parameters; } - /** * Returns the {@link org.apache.flink.api.java.io.SplitDataProperties} for the * {@link org.apache.flink.core.io.InputSplit}s of this DataSource * for configurations. * - * SplitDataProperties can help to generate more efficient execution plans. - *
- * + *

SplitDataProperties can help to generate more efficient execution plans. + * + * + *

* IMPORTANT: Incorrect configuration of SplitDataProperties can cause wrong results! * * @@ -115,28 +115,28 @@ public Configuration getParameters() { */ @PublicEvolving public SplitDataProperties getSplitDataProperties() { - if(this.splitDataProperties == null) { + if (this.splitDataProperties == null) { this.splitDataProperties = new SplitDataProperties(this); } return this.splitDataProperties; } // -------------------------------------------------------------------------------------------- - + protected GenericDataSourceBase translateToDataFlow() { - String name = this.name != null ? this.name : "at "+dataSourceLocationName+" ("+inputFormat.getClass().getName()+")"; + String name = this.name != null ? this.name : "at " + dataSourceLocationName + " (" + inputFormat.getClass().getName() + ")"; if (name.length() > 150) { name = name.substring(0, 150); } - - @SuppressWarnings({ "unchecked", "rawtypes" }) + + @SuppressWarnings({"unchecked", "rawtypes"}) GenericDataSourceBase source = new GenericDataSourceBase(this.inputFormat, - new OperatorInformation(getType()), name); + new OperatorInformation(getType()), name); source.setParallelism(parallelism); - if(this.parameters != null) { + if (this.parameters != null) { source.getParameters().addAll(this.parameters); } - if(this.splitDataProperties != null) { + if (this.splitDataProperties != null) { source.setSplitDataProperties(this.splitDataProperties); } return source; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java index 61f83b16ec959..dc80e70e2e248 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java @@ -18,10 +18,8 @@ package org.apache.flink.api.java.operators; -import java.util.Arrays; - -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.aggregators.Aggregator; @@ -35,10 +33,12 @@ import org.apache.flink.types.Value; import org.apache.flink.util.Preconditions; +import java.util.Arrays; + /** * The DeltaIteration represents the start of a delta iteration. It is created from the DataSet that * represents the initial solution set via the {@link DataSet#iterateDelta(DataSet, int, int...)} method. - * + * * @param The data type of the solution set. * @param The data type of the workset (the feedback data set). * @@ -47,27 +47,27 @@ */ @Public public class DeltaIteration { - + private final AggregatorRegistry aggregators = new AggregatorRegistry(); - + private final DataSet initialSolutionSet; private final DataSet initialWorkset; - + private final SolutionSetPlaceHolder solutionSetPlaceholder; private final WorksetPlaceHolder worksetPlaceholder; private final Keys keys; - + private final int maxIterations; - + private String name; - + private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; private ResourceSpec minResources = ResourceSpec.DEFAULT; private ResourceSpec preferredResources = ResourceSpec.DEFAULT; - + private boolean solutionSetUnManaged; public DeltaIteration(ExecutionEnvironment context, TypeInformation type, DataSet solutionSet, DataSet workset, Keys keys, int maxIterations) { @@ -80,15 +80,15 @@ public DeltaIteration(ExecutionEnvironment context, TypeInformation type, Da } // -------------------------------------------------------------------------------------------- - + /** * Closes the delta iteration. This method defines the end of the delta iteration's function. - * + * * @param solutionSetDelta The delta for the solution set. The delta will be merged into the solution set at the end of * each iteration. * @param newWorkset The new workset (feedback data set) that will be fed back to the next iteration. * @return The DataSet that represents the result of the iteration, after the computation has terminated. - * + * * @see DataSet#iterateDelta(DataSet, int, int...) */ public DataSet closeWith(DataSet solutionSetDelta, DataSet newWorkset) { @@ -98,18 +98,18 @@ public DataSet closeWith(DataSet solutionSetDelta, DataSet newWorkse /** * Gets the initial solution set. This is the data set on which the delta iteration was started. - *

- * Consider the following example: + * + *

Consider the following example: *

 	 * {@code
 	 * DataSet solutionSetData = ...;
 	 * DataSet worksetData = ...;
-	 * 
+	 *
 	 * DeltaIteration iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
 	 * }
 	 * 
* The solutionSetData would be the data set returned by {@code iteration.getInitialSolutionSet();}. - * + * * @return The data set that forms the initial solution set. */ public DataSet getInitialSolutionSet() { @@ -119,18 +119,18 @@ public DataSet getInitialSolutionSet() { /** * Gets the initial workset. This is the data set passed to the method that starts the delta * iteration. - *

- * Consider the following example: + * + *

Consider the following example: *

 	 * {@code
 	 * DataSet solutionSetData = ...;
 	 * DataSet worksetData = ...;
-	 * 
+	 *
 	 * DeltaIteration iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
 	 * }
 	 * 
* The worksetData would be the data set returned by {@code iteration.getInitialWorkset();}. - * + * * @return The data set that forms the initial workset. */ public DataSet getInitialWorkset() { @@ -139,7 +139,7 @@ public DataSet getInitialWorkset() { /** * Gets the solution set of the delta iteration. The solution set represents the state that is kept across iterations. - * + * * @return The solution set of the delta iteration. */ public SolutionSetPlaceHolder getSolutionSet() { @@ -157,7 +157,7 @@ public WorksetPlaceHolder getWorkset() { /** * Sets the name for the iteration. The name is displayed in logs and messages. - * + * * @param name The name for the iteration. * @return The iteration object, for function call chaining. */ @@ -165,16 +165,16 @@ public DeltaIteration name(String name) { this.name = name; return this; } - + /** * Gets the name from this iteration. - * + * * @return The name of the iteration. */ public String getName() { return name; } - + /** * Sets the parallelism for the iteration. * @@ -187,10 +187,10 @@ public DeltaIteration parallelism(int parallelism) { this.parallelism = parallelism; return this; } - + /** * Gets the iteration's parallelism. - * + * * @return The iteration's parallelism, or {@link ExecutionConfig#PARALLELISM_DEFAULT} if not set. */ public int getParallelism() { @@ -266,13 +266,13 @@ public ResourceSpec getPreferredResources() { * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step, * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances. * The value of an aggregator can be accessed in the next iteration. - *

- * Aggregators can be accessed inside a function via the + * + *

Aggregators can be accessed inside a function via the * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method. - * + * * @param name The name under which the aggregator is registered. * @param aggregator The aggregator class. - * + * * @return The DeltaIteration itself, to allow chaining function calls. */ @PublicEvolving @@ -297,62 +297,61 @@ public DeltaIteration registerAggregator(String name, Aggregator aggr */ @PublicEvolving public DeltaIteration registerAggregationConvergenceCriterion( - String name, Aggregator aggregator, ConvergenceCriterion convergenceCheck) - { + String name, Aggregator aggregator, ConvergenceCriterion convergenceCheck) { this.aggregators.registerAggregationConvergenceCriterion(name, aggregator, convergenceCheck); return this; } - + /** * Gets the registry for aggregators for the iteration. - * + * * @return The registry with all aggregators. */ @PublicEvolving public AggregatorRegistry getAggregators() { return this.aggregators; } - + /** * Sets whether to keep the solution set in managed memory (safe against heap exhaustion) or unmanaged memory * (objects on heap). - * + * * @param solutionSetUnManaged True to keep the solution set in unmanaged memory, false to keep it in managed memory. - * + * * @see #isSolutionSetUnManaged() */ public void setSolutionSetUnManaged(boolean solutionSetUnManaged) { this.solutionSetUnManaged = solutionSetUnManaged; } - + /** * gets whether the solution set is in managed or unmanaged memory. - * + * * @return True, if the solution set is in unmanaged memory (object heap), false if in managed memory. - * + * * @see #setSolutionSetUnManaged(boolean) */ public boolean isSolutionSetUnManaged() { return solutionSetUnManaged; } - + // -------------------------------------------------------------------------------------------- - + /** * A {@link DataSet} that acts as a placeholder for the solution set during the iteration. - * + * * @param The type of the elements in the solution set. */ @Public public static class SolutionSetPlaceHolder extends DataSet{ - + private final DeltaIteration deltaIteration; - + private SolutionSetPlaceHolder(ExecutionEnvironment context, TypeInformation type, DeltaIteration deltaIteration) { super(context, type); this.deltaIteration = deltaIteration; } - + public void checkJoinKeyFields(int[] keyFields) { int[] ssKeys = deltaIteration.keys.computeLogicalKeyPositions(); if (!Arrays.equals(ssKeys, keyFields)) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java index e59348877034e..de93dbbe62aa7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java @@ -17,6 +17,7 @@ */ package org.apache.flink.api.java.operators; + import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.operators.Keys; @@ -24,19 +25,24 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +/** + * Resulting {@link DataSet} of a delta iteration operation. + * @param + * @param + */ @Public public class DeltaIterationResultSet extends DataSet { private DeltaIteration iterationHead; private DataSet nextSolutionSet; - + private DataSet nextWorkset; - + private Keys keys; - + private int maxIterations; - + private TypeInformation typeWS; DeltaIterationResultSet(ExecutionEnvironment context, @@ -46,8 +52,7 @@ public class DeltaIterationResultSet extends DataSet { DataSet nextSolutionSet, DataSet nextWorkset, Keys keys, - int maxIterations) - { + int maxIterations) { super(context, typeSS); this.iterationHead = iterationHead; this.nextWorkset = nextWorkset; @@ -60,7 +65,7 @@ public class DeltaIterationResultSet extends DataSet { public DeltaIteration getIterationHead() { return iterationHead; } - + public DataSet getNextSolutionSet() { return nextSolutionSet; } @@ -78,7 +83,7 @@ public int[] getKeyPositions() { public int getMaxIterations() { return maxIterations; } - + public TypeInformation getWorksetType() { return typeWS; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index 267513d4421f9..9dc2a9c84918c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -111,7 +111,7 @@ else if (keys instanceof SelectorFunctionKeys) { /** * Sets the strategy to use for the combine phase of the reduce. * - * If this method is not called, then the default hint will be used. + *

If this method is not called, then the default hint will be used. * ({@link org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint#OPTIMIZER_CHOOSES}) * * @param strategy The hint to use. @@ -132,8 +132,7 @@ public DistinctOperator setCombineHint(CombineHint strategy) { String name, Operator input, int parallelism, - CombineHint hint) - { + CombineHint hint) { @SuppressWarnings("unchecked") final SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; @@ -150,7 +149,7 @@ public DistinctOperator setCombineHint(CombineHint strategy) { } @Internal - public static final class DistinctFunction implements ReduceFunction { + private static final class DistinctFunction implements ReduceFunction { private static final long serialVersionUID = 1L; 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 ae8b5eaabe385..901274e518fcf 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,45 +22,45 @@ 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; +import org.apache.flink.api.java.operators.translation.PlanFilterOperator; /** * This operator represents the application of a "filter" function on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set filtered by the operator. */ @Public public class FilterOperator extends SingleInputUdfOperator> { - + protected final FilterFunction function; - + protected final String defaultName; public FilterOperator(DataSet input, FilterFunction function, String defaultName) { super(input, input.getType()); - + this.function = function; this.defaultName = defaultName; UdfOperatorUtils.analyzeSingleInputUdf(this, FilterFunction.class, defaultName, function, null); } - + @Override protected FilterFunction getFunction() { return function; } - + @Override - protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { - - String name = getName() != null ? getName() : "Filter at "+defaultName; - + protected org.apache.flink.api.common.operators.base.FilterOperatorBase> translateToDataFlow(Operator input) { + + String name = getName() != null ? getName() : "Filter at " + defaultName; + // create operator PlanFilterOperator po = new PlanFilterOperator(function, name, getInputType()); po.setInput(input); - + // set parallelism if (getParallelism() > 0) { // use specified parallelism @@ -69,7 +69,7 @@ protected org.apache.flink.api.common.operators.base.FilterOperatorBase The type of the data set consumed by the operator. * @param The type of the data set created by the operator. */ @Public public class FlatMapOperator extends SingleInputUdfOperator> { - + protected final FlatMapFunction function; - + protected final String defaultName; - + public FlatMapOperator(DataSet input, TypeInformation resultType, FlatMapFunction function, String defaultName) { super(input, resultType); - + this.function = function; this.defaultName = defaultName; UdfOperatorUtils.analyzeSingleInputUdf(this, FlatMapFunction.class, defaultName, function, null); } - + @Override protected FlatMapFunction getFunction() { return function; } @Override - protected FlatMapOperatorBase> translateToDataFlow(Operator input) { - String name = getName() != null ? getName() : "FlatMap at "+defaultName; + protected FlatMapOperatorBase> translateToDataFlow(Operator input) { + String name = getName() != null ? getName() : "FlatMap at " + defaultName; // create operator FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, - new UnaryOperatorInformation(getInputType(), getResultType()), name); + new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set parallelism - if(this.getParallelism() > 0) { + if (this.getParallelism() > 0) { // use specified parallelism po.setParallelism(this.getParallelism()); } else { // if no parallelism has been specified, use parallelism of input operator to enable chaining po.setParallelism(input.getParallelism()); } - + return po; } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java index 0c8e657e73cf4..e4ed07f87ca98 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -33,13 +34,12 @@ import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.translation.PlanUnwrappingGroupCombineOperator; import org.apache.flink.api.java.operators.translation.PlanUnwrappingSortedGroupCombineOperator; -import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; /** * This operator behaves like the GroupReduceOperator with Combine but only runs the Combine part which reduces all data - * locally in their partitions. The combine part can return an arbitrary data type. This is useful to pre-combine values + * locally in their partitions. The combine part can return an arbitrary data type. This is useful to pre-combine values * into an intermediate representation before applying a proper reduce operation. * * @param The type of the data set consumed by the operator. @@ -95,12 +95,12 @@ public SingleInputSemanticProperties getSemanticProperties() { SingleInputSemanticProperties props = super.getSemanticProperties(); // offset semantic information by extracted key fields - if(props != null && + if (props != null && this.grouper != null && this.grouper.keys instanceof SelectorFunctionKeys) { - int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); - if(this.grouper instanceof SortedGrouping) { + int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); + if (this.grouper instanceof SortedGrouping) { offset += ((SortedGrouping) this.grouper).getSortSelectionFunctionKey().getKeyType().getTotalFields(); } @@ -174,7 +174,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { Order[] sortOrders = sortedGrouper.getGroupSortOrders(); Ordering o = new Ordering(); - for(int i=0; i < sortKeyPositions.length; i++) { + for (int i = 0; i < sortKeyPositions.length; i++) { o.appendOrdering(sortKeyPositions[i], null, sortOrders[i]); } po.setGroupOrder(o); @@ -187,7 +187,6 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { } } - // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") @@ -196,8 +195,7 @@ private static PlanUnwrappingGroupCombineOperator trans GroupCombineFunction function, TypeInformation outputType, String name, - Operator input) - { + Operator input) { final SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); @@ -218,10 +216,9 @@ private static PlanUnwrappingSortedGroupCombineOperator function, TypeInformation outputType, String name, - Operator input) - { + Operator input) { final SelectorFunctionKeys groupingKey = (SelectorFunctionKeys) rawGroupingKey; - final SelectorFunctionKeys sortingKey = (SelectorFunctionKeys)rawSortingKeys; + final SelectorFunctionKeys sortingKey = (SelectorFunctionKeys) rawSortingKeys; TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(groupingKey, sortingKey); Operator> inputWithKey = KeyFunctions.appendKeyExtractor(input, groupingKey, sortingKey); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java index b339baf0c9c46..069ac44a97ad4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java @@ -24,7 +24,8 @@ import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.java.operators.translation.CombineToGroupCombineWrapper; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -32,15 +33,15 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; -import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.java.operators.translation.CombineToGroupCombineWrapper; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.operators.translation.PlanUnwrappingSortedReduceGroupOperator; import org.apache.flink.api.java.operators.translation.RichCombineToGroupCombineWrapper; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.DataSet; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ /** * This operator represents the application of a "reduceGroup" function on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. */ @@ -62,14 +63,14 @@ public class GroupReduceOperator extends SingleInputUdfOperator function; private final Grouping grouper; - + private final String defaultName; private boolean combinable; /** * Constructor for a non-grouped reduce (all reduce). - * + * * @param input The input data set to the groupReduce function. * @param function The user-defined GroupReduce function. */ @@ -82,10 +83,10 @@ public GroupReduceOperator(DataSet input, TypeInformation resultType, G this.combinable = checkCombinability(); } - + /** * Constructor for a grouped reduce. - * + * * @param input The grouped input to be processed group-wise by the groupReduce function. * @param function The user-defined GroupReduce function. */ @@ -149,14 +150,12 @@ else if (reduceTypes == null || reduceTypes.length != 2) { } return false; } - - + @Override protected GroupReduceFunction getFunction() { return function; } - // -------------------------------------------------------------------------------------------- // Properties // -------------------------------------------------------------------------------------------- @@ -164,10 +163,10 @@ protected GroupReduceFunction getFunction() { public boolean isCombinable() { return combinable; } - + public GroupReduceOperator setCombinable(boolean combinable) { - if(combinable) { + if (combinable) { // sanity check that the function is a subclass of the combine interface if (!checkCombinability()) { throw new IllegalArgumentException("Either the function does not implement a combine interface, " + @@ -188,12 +187,12 @@ public SingleInputSemanticProperties getSemanticProperties() { SingleInputSemanticProperties props = super.getSemanticProperties(); // offset semantic information by extracted key fields - if(props != null && + if (props != null && this.grouper != null && this.grouper.keys instanceof SelectorFunctionKeys) { - int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); - if(this.grouper instanceof SortedGrouping) { + int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); + if (this.grouper instanceof SortedGrouping) { offset += ((SortedGrouping) this.grouper).getSortSelectionFunctionKey().getKeyType().getTotalFields(); } props = SemanticPropUtil.addSourceFieldOffset(props, this.getInputType().getTotalFields(), offset); @@ -205,7 +204,7 @@ public SingleInputSemanticProperties getSemanticProperties() { // -------------------------------------------------------------------------------------------- // Translation // -------------------------------------------------------------------------------------------- - + @Override @SuppressWarnings("unchecked") protected GroupReduceOperatorBase translateToDataFlow(Operator input) { @@ -225,16 +224,16 @@ public SingleInputSemanticProperties getSemanticProperties() { UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getInputType(), getResultType()); GroupReduceOperatorBase> po = new GroupReduceOperatorBase<>(function, operatorInfo, new int[0], name); - + po.setCombinable(combinable); po.setInput(input); // the parallelism for a non grouped reduce can only be 1 po.setParallelism(1); return po; } - + if (grouper.getKeys() instanceof SelectorFunctionKeys) { - + @SuppressWarnings("unchecked") SelectorFunctionKeys selectorKeys = (SelectorFunctionKeys) grouper.getKeys(); @@ -271,29 +270,28 @@ else if (grouper.getKeys() instanceof ExpressionKeys) { po.setInput(input); po.setParallelism(getParallelism()); po.setCustomPartitioner(grouper.getCustomPartitioner()); - + // set group order if (grouper instanceof SortedGrouping) { SortedGrouping sortedGrouper = (SortedGrouping) grouper; int[] sortKeyPositions = sortedGrouper.getGroupSortKeyPositions(); Order[] sortOrders = sortedGrouper.getGroupSortOrders(); - + Ordering o = new Ordering(); - for(int i=0; i < sortKeyPositions.length; i++) { + for (int i = 0; i < sortKeyPositions.length; i++) { o.appendOrdering(sortKeyPositions[i], null, sortOrders[i]); } po.setGroupOrder(o); } - + return po; } else { throw new UnsupportedOperationException("Unrecognized key type."); } } - - + // -------------------------------------------------------------------------------------------- @SuppressWarnings("unchecked") @@ -303,8 +301,7 @@ private static PlanUnwrappingReduceGroupOperator transl TypeInformation outputType, String name, Operator input, - boolean combinable) - { + boolean combinable) { SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); @@ -326,11 +323,10 @@ private static PlanUnwrappingSortedReduceGroupOperator outputType, String name, Operator input, - boolean combinable) - { + boolean combinable) { final SelectorFunctionKeys groupingKey = (SelectorFunctionKeys) rawGroupingKey; final SelectorFunctionKeys sortingKey = (SelectorFunctionKeys) rawSortingKey; - TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(groupingKey,sortingKey); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(groupingKey, sortingKey); Operator> inputWithKey = KeyFunctions.appendKeyExtractor(input, groupingKey, sortingKey); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java index dbaaa9d52c426..74bd9e5acb7a8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java @@ -26,8 +26,9 @@ import org.apache.flink.api.java.DataSet; /** - * Grouping is an intermediate step for a transformation on a grouped DataSet.
- * The following transformation can be applied on Grouping: + * Grouping is an intermediate step for a transformation on a grouped DataSet. + * + *

The following transformation can be applied on Grouping: *

    *
  • {@link UnsortedGrouping#reduce(org.apache.flink.api.common.functions.ReduceFunction)},
  • *
  • {@link UnsortedGrouping#reduceGroup(org.apache.flink.api.common.functions.GroupReduceFunction)}, and
  • @@ -35,24 +36,23 @@ *
* * @param The type of the elements of the grouped DataSet. - * + * * @see DataSet */ @Public public abstract class Grouping { - + protected final DataSet inputDataSet; - + protected final Keys keys; - + protected Partitioner customPartitioner; - public Grouping(DataSet set, Keys keys) { if (set == null || keys == null) { throw new NullPointerException(); } - + if (keys.isEmpty()) { throw new InvalidProgramException("The grouping keys must not be empty."); } @@ -60,17 +60,17 @@ public Grouping(DataSet set, Keys keys) { this.inputDataSet = set; this.keys = keys; } - + /** * Returns the input DataSet of a grouping operation, that is the one before the grouping. This means that * if it is applied directly to the result of a grouping operation, it will cancel its effect. As an example, in the * following snippet: - *

+	 * 
{@code
 	 * DataSet notGrouped = input.groupBy().getDataSet();
 	 * DataSet allReduced = notGrouped.reduce()
-	 * 
- * the groupBy() is as if it never happened, as the notGrouped DataSet corresponds - * to the input of the groupBy() (because of the getDataset()). + * }
+ * the {@code groupBy()} is as if it never happened, as the {@code notGrouped} DataSet corresponds + * to the input of the {@code groupBy()} (because of the {@code getDataset()}). * */ @Internal public DataSet getInputDataSet() { @@ -81,11 +81,11 @@ public DataSet getInputDataSet() { public Keys getKeys() { return this.keys; } - + /** * Gets the custom partitioner to be used for this grouping, or {@code null}, if * none was defined. - * + * * @return The custom partitioner to be used for this grouping. */ @Internal diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java index c7ff6ab0cabee..d522f02924615 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java @@ -31,9 +31,9 @@ import org.apache.flink.types.Value; /** - * The IterativeDataSet represents the start of an iteration. It is created from the DataSet that + * The IterativeDataSet represents the start of an iteration. It is created from the DataSet that * represents the initial solution set via the {@link DataSet#iterate(int)} method. - * + * * @param The data type of set that is the input and feedback of the iteration. * * @see DataSet#iterate(int) @@ -42,33 +42,33 @@ public class IterativeDataSet extends SingleInputOperator> { private final AggregatorRegistry aggregators = new AggregatorRegistry(); - + private int maxIterations; public IterativeDataSet(ExecutionEnvironment context, TypeInformation type, DataSet input, int maxIterations) { super(input, type); this.maxIterations = maxIterations; } - + // -------------------------------------------------------------------------------------------- - + /** * Closes the iteration. This method defines the end of the iterative program part. - * + * * @param iterationResult The data set that will be fed back to the next iteration. * @return The DataSet that represents the result of the iteration, after the computation has terminated. - * + * * @see DataSet#iterate(int) */ public DataSet closeWith(DataSet iterationResult) { return new BulkIterationResultSet(getExecutionEnvironment(), getType(), this, iterationResult); } - + /** * Closes the iteration and specifies a termination criterion. This method defines the end of * the iterative program part. - *

- * The termination criterion is a means of dynamically signaling the iteration to halt. It is expressed via a data + * + *

The termination criterion is a means of dynamically signaling the iteration to halt. It is expressed via a data * set that will trigger to halt the loop as soon as the data set is empty. A typical way of using the termination * criterion is to have a filter that filters out all elements that are considered non-converged. As soon as no more * such elements exist, the iteration finishes. @@ -76,7 +76,7 @@ public DataSet closeWith(DataSet iterationResult) { * @param iterationResult The data set that will be fed back to the next iteration. * @param terminationCriterion The data set that being used to trigger halt on operation once it is empty. * @return The DataSet that represents the result of the iteration, after the computation has terminated. - * + * * @see DataSet#iterate(int) */ public DataSet closeWith(DataSet iterationResult, DataSet terminationCriterion) { @@ -85,25 +85,25 @@ public DataSet closeWith(DataSet iterationResult, DataSet terminationCr /** * Gets the maximum number of iterations. - * + * * @return The maximum number of iterations. */ public int getMaxIterations() { return maxIterations; } - + /** * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step, * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances. * The value of an aggregator can be accessed in the next iteration. - *

- * Aggregators can be accessed inside a function via the + * + *

Aggregators can be accessed inside a function via the * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method. - * + * * @param name The name under which the aggregator is registered. * @param aggregator The aggregator class. - * + * * @return The IterativeDataSet itself, to allow chaining function calls. */ @PublicEvolving @@ -111,7 +111,7 @@ public IterativeDataSet registerAggregator(String name, Aggregator aggrega this.aggregators.registerAggregator(name, aggregator); return this; } - + /** * Registers an {@link Aggregator} for the iteration together with a {@link ConvergenceCriterion}. For a general description * of aggregators, see {@link #registerAggregator(String, Aggregator)} and {@link Aggregator}. @@ -119,33 +119,32 @@ public IterativeDataSet registerAggregator(String name, Aggregator aggrega * the iteration should terminate. A typical use case is to have an aggregator that sums up the total error of change * in an iteration step and have to have a convergence criterion that signals termination as soon as the aggregate value * is below a certain threshold. - * + * * @param name The name under which the aggregator is registered. * @param aggregator The aggregator class. * @param convergenceCheck The convergence criterion. - * + * * @return The IterativeDataSet itself, to allow chaining function calls. */ @PublicEvolving public IterativeDataSet registerAggregationConvergenceCriterion( - String name, Aggregator aggregator, ConvergenceCriterion convergenceCheck) - { + String name, Aggregator aggregator, ConvergenceCriterion convergenceCheck) { this.aggregators.registerAggregationConvergenceCriterion(name, aggregator, convergenceCheck); return this; } - + /** - * Gets the registry for aggregators. On the registry, one can add {@link Aggregator}s and an aggregator-based + * Gets the registry for aggregators. On the registry, one can add {@link Aggregator}s and an aggregator-based * {@link ConvergenceCriterion}. This method offers an alternative way to registering the aggregators via - * {@link #registerAggregator(String, Aggregator)} and {@link #registerAggregationConvergenceCriterion(String, Aggregator, ConvergenceCriterion))}. - * + * {@link #registerAggregator(String, Aggregator)} and {@link #registerAggregationConvergenceCriterion(String, Aggregator, ConvergenceCriterion)}. + * * @return The registry for aggregators. */ @PublicEvolving public AggregatorRegistry getAggregators() { return aggregators; } - + // -------------------------------------------------------------------------------------------- @Override diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index 86ee2a2d5d095..4b5dc6bb00142 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -18,8 +18,6 @@ package org.apache.flink.api.java.operators; -import java.util.Arrays; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -31,9 +29,12 @@ import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; +import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -44,32 +45,53 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.common.operators.Keys.ExpressionKeys; -import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; -import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.java.operators.join.JoinFunctionAssigner; import org.apache.flink.api.java.operators.join.JoinOperatorSetsBase; import org.apache.flink.api.java.operators.join.JoinType; -import org.apache.flink.api.java.operators.join.JoinFunctionAssigner; -import org.apache.flink.api.java.operators.translation.TupleRightUnwrappingJoiner; import org.apache.flink.api.java.operators.translation.TupleLeftUnwrappingJoiner; +import org.apache.flink.api.java.operators.translation.TupleRightUnwrappingJoiner; import org.apache.flink.api.java.operators.translation.TupleUnwrappingJoiner; import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple10; +import org.apache.flink.api.java.tuple.Tuple11; +import org.apache.flink.api.java.tuple.Tuple12; +import org.apache.flink.api.java.tuple.Tuple13; +import org.apache.flink.api.java.tuple.Tuple14; +import org.apache.flink.api.java.tuple.Tuple15; +import org.apache.flink.api.java.tuple.Tuple16; +import org.apache.flink.api.java.tuple.Tuple17; +import org.apache.flink.api.java.tuple.Tuple18; +import org.apache.flink.api.java.tuple.Tuple19; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple20; +import org.apache.flink.api.java.tuple.Tuple21; +import org.apache.flink.api.java.tuple.Tuple22; +import org.apache.flink.api.java.tuple.Tuple23; +import org.apache.flink.api.java.tuple.Tuple24; +import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.tuple.Tuple6; +import org.apache.flink.api.java.tuple.Tuple7; +import org.apache.flink.api.java.tuple.Tuple8; +import org.apache.flink.api.java.tuple.Tuple9; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; -//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator -import org.apache.flink.api.java.tuple.*; -//CHECKSTYLE.ON: AvoidStarImport +import java.util.Arrays; /** - * A {@link DataSet} that is the result of a Join transformation. - * + * A {@link DataSet} that is the result of a Join transformation. + * * @param The type of the first input DataSet of the Join transformation. * @param The type of the second input DataSet of the Join transformation. * @param The type of the result of the Join transformation. - * + * * @see DataSet */ @Public @@ -77,22 +99,20 @@ public abstract class JoinOperator extends TwoInputUdfOperator keys1; protected final Keys keys2; - + private final JoinHint joinHint; protected final JoinType joinType; private Partitioner customPartitioner; - - + protected JoinOperator(DataSet input1, DataSet input2, Keys keys1, Keys keys2, - TypeInformation returnType, JoinHint hint, JoinType type) - { + TypeInformation returnType, JoinHint hint, JoinType type) { super(input1, input2, returnType); - + Preconditions.checkNotNull(keys1); Preconditions.checkNotNull(keys2); - + try { if (!keys1.areCompatible(keys2)) { throw new InvalidProgramException("The types of the key fields do not match."); @@ -125,18 +145,18 @@ protected JoinOperator(DataSet input1, DataSet input2, this.joinHint = hint == null ? InnerJoinOperatorBase.JoinHint.OPTIMIZER_CHOOSES : hint; this.joinType = type; } - + protected Keys getKeys1() { return this.keys1; } - + protected Keys getKeys2() { return this.keys2; } - + /** * Gets the JoinHint that describes how the join is executed. - * + * * @return The JoinHint. */ @Internal @@ -153,14 +173,14 @@ public JoinHint getJoinHint() { public JoinType getJoinType() { return this.joinType; } - + /** * Sets a custom partitioner for this join. The partitioner will be called on the join keys to determine * the partition a key should be assigned to. The partitioner is evaluated on both join inputs in the * same way. - *

- * NOTE: A custom partitioner can only be used with single-field join keys, not with composite join keys. - * + * + *

NOTE: A custom partitioner can only be used with single-field join keys, not with composite join keys. + * * @param partitioner The custom partitioner to be used. * @return This join operator, to allow for function chaining. */ @@ -172,42 +192,43 @@ public JoinOperator withPartitioner(Partitioner partitioner) { this.customPartitioner = getInput1().clean(partitioner); return this; } - + /** * Gets the custom partitioner used by this join, or {@code null}, if none is set. - * + * * @return The custom partitioner used by this join; */ @Internal public Partitioner getPartitioner() { return customPartitioner; } - + // -------------------------------------------------------------------------------------------- // special join types // -------------------------------------------------------------------------------------------- - + /** - * A Join transformation that applies a {@link JoinFunction} on each pair of joining elements.
- * It also represents the {@link DataSet} that is the result of a Join transformation. - * + * A Join transformation that applies a {@link JoinFunction} on each pair of joining elements. + * + *

It also represents the {@link DataSet} that is the result of a Join transformation. + * * @param The type of the first input DataSet of the Join transformation. * @param The type of the second input DataSet of the Join transformation. * @param The type of the result of the Join transformation. - * + * * @see org.apache.flink.api.common.functions.RichFlatJoinFunction * @see DataSet */ @Public public static class EquiJoin extends JoinOperator { - + private final FlatJoinFunction function; - + @SuppressWarnings("unused") private boolean preserve1; @SuppressWarnings("unused") private boolean preserve2; - + private final String joinLocationName; public EquiJoin(DataSet input1, DataSet input2, @@ -226,11 +247,11 @@ public EquiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, FlatJoinFunction function, TypeInformation returnType, JoinHint hint, String joinLocationName, JoinType type) { super(input1, input2, keys1, keys2, returnType, hint, type); - + if (function == null) { throw new NullPointerException(); } - + this.function = function; this.joinLocationName = joinLocationName; @@ -241,7 +262,7 @@ public EquiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, FlatJoinFunction generatedFunction, JoinFunction function, TypeInformation returnType, JoinHint hint, String joinLocationName, JoinType type) { super(input1, input2, keys1, keys2, returnType, hint, type); - + this.joinLocationName = joinLocationName; if (function == null) { @@ -252,7 +273,7 @@ public EquiJoin(DataSet input1, DataSet input2, UdfOperatorUtils.analyzeDualInputUdf(this, JoinFunction.class, joinLocationName, function, keys1, keys2); } - + @Override protected FlatJoinFunction getFunction() { return function; @@ -265,16 +286,16 @@ public DualInputSemanticProperties getSemanticProperties() { DualInputSemanticProperties props = super.getSemanticProperties(); // offset semantic information by extracted key fields - if(props != null && + if (props != null && (this.keys1 instanceof SelectorFunctionKeys || this.keys2 instanceof SelectorFunctionKeys)) { int numFields1 = this.getInput1Type().getTotalFields(); int numFields2 = this.getInput2Type().getTotalFields(); int offset1 = (this.keys1 instanceof SelectorFunctionKeys) ? - ((SelectorFunctionKeys) this.keys1).getKeyType().getTotalFields() : 0; + ((SelectorFunctionKeys) this.keys1).getKeyType().getTotalFields() : 0; int offset2 = (this.keys2 instanceof SelectorFunctionKeys) ? - ((SelectorFunctionKeys) this.keys2).getKeyType().getTotalFields() : 0; + ((SelectorFunctionKeys) this.keys2).getKeyType().getTotalFields() : 0; props = SemanticPropUtil.addSourceFieldOffsets(props, numFields1, numFields2, offset1, offset2); } @@ -371,7 +392,6 @@ protected boolean udfWithForwardedFieldsSecondAnnotation(Class udfClass) { return builder.build(); } - private static final class JoinOperatorBaseBuilder { private final String name; @@ -402,7 +422,7 @@ public JoinOperatorBaseBuilder withWrappedInput1( SelectorFunctionKeys rawKeys1) { @SuppressWarnings("unchecked") - SelectorFunctionKeys keys1 = (SelectorFunctionKeys)rawKeys1; + SelectorFunctionKeys keys1 = (SelectorFunctionKeys) rawKeys1; TypeInformation> typeInfoWithKey1 = KeyFunctions.createTypeWithKey(keys1); Operator> keyMapper1 = KeyFunctions.appendKeyExtractor(input1, keys1); @@ -414,7 +434,7 @@ public JoinOperatorBaseBuilder withWrappedInput2( SelectorFunctionKeys rawKeys2) { @SuppressWarnings("unchecked") - SelectorFunctionKeys keys2 = (SelectorFunctionKeys)rawKeys2; + SelectorFunctionKeys keys2 = (SelectorFunctionKeys) rawKeys2; TypeInformation> typeInfoWithKey2 = KeyFunctions.createTypeWithKey(keys2); Operator> keyMapper2 = KeyFunctions.appendKeyExtractor(input2, keys2); @@ -508,11 +528,12 @@ private OuterJoinOperatorBase.OuterJoinType getOuterJoinType() { } } } - + /** - * A Join transformation that wraps pairs of joining elements into {@link Tuple2}.
- * It also represents the {@link DataSet} that is the result of a Join transformation. - * + * A Join transformation that wraps pairs of joining elements into {@link Tuple2}. + * + *

It also represents the {@link DataSet} that is the result of a Join transformation. + * * @param The type of the first input DataSet of the Join transformation. * @param The type of the second input DataSet of the Join transformation. * @@ -523,20 +544,20 @@ private OuterJoinOperatorBase.OuterJoinType getOuterJoinType() { public static final class DefaultJoin extends EquiJoin> implements JoinFunctionAssigner { public DefaultJoin(DataSet input1, DataSet input2, - Keys keys1, Keys keys2, JoinHint hint, String joinLocationName, JoinType type) - { + Keys keys1, Keys keys2, JoinHint hint, String joinLocationName, JoinType type) { super(input1, input2, keys1, keys2, new DefaultFlatJoinFunction(), new TupleTypeInfo>(input1.getType(), input2.getType()), hint, joinLocationName, type); } /** - * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements.
- * Each JoinFunction call returns exactly one element. - * + * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements. + * + *

Each JoinFunction call returns exactly one element. + * * @param function The JoinFunction that is called for each pair of joined elements. * @return An EquiJoin that represents the joined result DataSet - * + * * @see org.apache.flink.api.common.functions.RichFlatJoinFunction * @see org.apache.flink.api.java.operators.JoinOperator.EquiJoin * @see DataSet @@ -558,8 +579,15 @@ public EquiJoin with(JoinFunction function) { return new EquiJoin<>(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint(), Utils.getCallLocationName(), joinType); } + /** + * Wrapper around {@link JoinFunction}. + * + * @param type of elements of first collection + * @param type of elements of second collection + * @param type of elements of resulting elements + */ @Internal - public static class WrappingFlatJoinFunction extends WrappingFunction> implements FlatJoinFunction { + public static class WrappingFlatJoinFunction extends WrappingFunction> implements FlatJoinFunction { private static final long serialVersionUID = 1L; @@ -574,11 +602,12 @@ public void join(IN1 left, IN2 right, Collector out) throws Exception { } /** - * Applies a ProjectJoin transformation and projects the first join input
- * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first join input is not a Tuple DataSet, no parameters should be passed.
- * - * Fields of the first and second input can be added by chaining the method calls of + * Applies a ProjectJoin transformation and projects the first join input + * + *

If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first join input is not a Tuple DataSet, no parameters should be passed. + * + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}. * @@ -588,7 +617,7 @@ public void join(IN1 left, IN2 right, Collector out) throws Exception { * For a non-Tuple DataSet, do not provide parameters. * The order of fields in the output tuple is defined by to the order of field indexes. * @return A ProjectJoin which represents the projected join result. - * + * * @see Tuple * @see DataSet * @see org.apache.flink.api.java.operators.JoinOperator.ProjectJoin @@ -598,67 +627,69 @@ public ProjectJoin projectFirst(int... firstFie return joinProjection.projectTupleX(); } - + /** - * Applies a ProjectJoin transformation and projects the second join input
- * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second join input is not a Tuple DataSet, no parameters should be passed.
- * - * Fields of the first and second input can be added by chaining the method calls of + * Applies a ProjectJoin transformation and projects the second join input + * + *

If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second join input is not a Tuple DataSet, no parameters should be passed. + * + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}. * - * Note: With the current implementation, the Project transformation loses type information. + *

Note: With the current implementation, the Project transformation loses type information. * - * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. + * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. * The order of fields in the output tuple is defined by to the order of field indexes. * @return A ProjectJoin which represents the projected join result. - * + * * @see Tuple * @see DataSet * @see org.apache.flink.api.java.operators.JoinOperator.ProjectJoin */ public ProjectJoin projectSecond(int... secondFieldIndexes) { JoinProjection joinProjection = new JoinProjection<>(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint(), null, secondFieldIndexes); - + return joinProjection.projectTupleX(); } // public JoinOperator leftSemiJoin() { // return new LeftSemiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint()); // } - + // public JoinOperator rightSemiJoin() { // return new RightSemiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint()); // } - + // public JoinOperator leftAntiJoin() { // return new LeftAntiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint()); // } - + // public JoinOperator rightAntiJoin() { // return new RightAntiJoin(getInput1(), getInput2(), getKeys1(), getKeys2(), getJoinHint()); // } } - + /** - * A Join transformation that projects joining elements or fields of joining {@link Tuple Tuples} - * into result {@link Tuple Tuples}.
- * It also represents the {@link DataSet} that is the result of a Join transformation. - * + * A Join transformation that projects joining elements or fields of joining {@link Tuple Tuples} + * into result {@link Tuple Tuples}. + * + *

It also represents the {@link DataSet} that is the result of a Join transformation. + * * @param The type of the first input DataSet of the Join transformation. * @param The type of the second input DataSet of the Join transformation. * @param The type of the result of the Join transformation. - * + * * @see Tuple * @see DataSet */ @Public public static class ProjectJoin extends EquiJoin { - + private JoinProjection joinProj; - + protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo returnType) { super(input1, input2, keys1, keys2, new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), @@ -666,7 +697,7 @@ protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Ke joinProj = null; } - + protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo returnType, JoinProjection joinProj) { super(input1, input2, keys1, keys2, new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), @@ -681,15 +712,16 @@ protected ProjectFlatJoinFunction getFunction() { } /** - * Continues a ProjectJoin transformation and adds fields of the first join input to the projection.
- * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first join input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectJoin transformation and adds fields of the first join input to the projection. + * + *

If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first join input is not a Tuple DataSet, no parameters should be passed. * - * Additional fields of the first and second input can be added by chaining the method calls of + *

Additional fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}. * - * Note: With the current implementation, the Project transformation loses type information. + *

Note: With the current implementation, the Project transformation loses type information. * * @param firstFieldIndexes If the first input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. @@ -701,22 +733,23 @@ protected ProjectFlatJoinFunction getFunction() { * @see org.apache.flink.api.java.operators.JoinOperator.ProjectJoin */ @SuppressWarnings("hiding") - public ProjectJoin projectFirst(int... firstFieldIndexes) { + public ProjectJoin projectFirst(int... firstFieldIndexes) { joinProj = joinProj.projectFirst(firstFieldIndexes); - + return joinProj.projectTupleX(); } /** - * Continues a ProjectJoin transformation and adds fields of the second join input to the projection.
- * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second join input is not a Tuple DataSet, no parameters should be passed.
+ * Continues a ProjectJoin transformation and adds fields of the second join input to the projection. * - * Additional fields of the first and second input can be added by chaining the method calls of + *

If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second join input is not a Tuple DataSet, no parameters should be passed. + * + *

Additional fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectSecond(int...)}. * - * Note: With the current implementation, the Project transformation loses type information. + *

Note: With the current implementation, the Project transformation loses type information. * * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. @@ -730,7 +763,7 @@ public ProjectJoin projectFirst(int... firstFie @SuppressWarnings("hiding") public ProjectJoin projectSecond(int... secondFieldIndexes) { joinProj = joinProj.projectSecond(secondFieldIndexes); - + return joinProj.projectTupleX(); } @@ -743,15 +776,15 @@ public ProjectJoin projectSecond(int... secondF @Deprecated @PublicEvolving public JoinOperator types(Class... types) { - TupleTypeInfo typeInfo = (TupleTypeInfo)this.getResultType(); + TupleTypeInfo typeInfo = (TupleTypeInfo) this.getResultType(); - if(types.length != typeInfo.getArity()) { + if (types.length != typeInfo.getArity()) { throw new InvalidProgramException("Provided types do not match projection."); } - for (int i=0; i typeClass = types[i]; if (!typeClass.equals(typeInfo.getTypeAt(i).getTypeClass())) { - throw new InvalidProgramException("Provided type "+typeClass.getSimpleName()+" at position "+i+" does not match projection"); + throw new InvalidProgramException("Provided type " + typeClass.getSimpleName() + " at position " + i + " does not match projection"); } } return (JoinOperator) this; @@ -766,7 +799,7 @@ public JoinOperator withForwardedFieldsFirst(String... forwardedFie public JoinOperator withForwardedFieldsSecond(String... forwardedFieldsSecond) { throw new InvalidProgramException("The semantic properties (forwarded fields) are automatically calculated."); } - + @Override protected DualInputSemanticProperties extractSemanticAnnotationsFromUdf(Class udfClass) { // we do not extract the annotation, we construct the properties from the projection# @@ -775,68 +808,69 @@ protected DualInputSemanticProperties extractSemanticAnnotationsFromUdf(Class } } - + // @SuppressWarnings("unused") // private static final class LeftAntiJoin extends JoinOperator { -// +// // protected LeftAntiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { // super(input1, input2, keys1, keys2, input1.getType(), hint); // } -// +// // @Override // protected Operator translateToDataFlow(Operator input1, Operator input2) { // throw new UnsupportedOperationException("LeftAntiJoin operator currently not supported."); // } // } - + // @SuppressWarnings("unused") // private static final class RightAntiJoin extends JoinOperator { -// +// // protected RightAntiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { // super(input1, input2, keys1, keys2, input2.getType(), hint); // } -// +// // @Override // protected Operator translateToDataFlow(Operator input1, Operator input2) { // throw new UnsupportedOperationException("RightAntiJoin operator currently not supported."); // } // } - + // @SuppressWarnings("unused") // private static final class LeftSemiJoin extends EquiJoin { -// +// // protected LeftSemiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { // super(input1, input2, keys1, keys2, new LeftSemiJoinFunction(), input1.getType(), hint); // } -// +// // @Override // protected Operator translateToDataFlow(Operator input1, Operator input2) { // // TODO: Runtime support required. Each left tuple may be returned only once. -// // Special exec strategy (runtime + optimizer) based on hash join required. +// // Special exec strategy (runtime + optimizer) based on hash join required. // // Either no duplicates of right side in HT or left tuples removed from HT after first match. // throw new UnsupportedOperationException("LeftSemiJoin operator currently not supported."); // } // } - + // @SuppressWarnings("unused") // private static final class RightSemiJoin extends EquiJoin { -// +// // protected RightSemiJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint) { // super(input1, input2, keys1, keys2, new RightSemiJoinFunction(), input2.getType(), hint); // } -// +// // @Override // protected Operator translateToDataFlow(Operator input1, Operator input2) { // // TODO: Runtime support required. Each right tuple may be returned only once. -// // Special exec strategy (runtime + optimizer) based on hash join required. +// // Special exec strategy (runtime + optimizer) based on hash join required. // // Either no duplicates of left side in HT or right tuples removed from HT after first match. // throw new UnsupportedOperationException("RightSemiJoin operator currently not supported."); // } // } /** - * Intermediate step of a Join transformation.
- * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling + * Intermediate step of a Join transformation. + * + *

To continue the Join transformation, select the join key of the first input {@link DataSet} by calling * {@link JoinOperatorSets#where(int...)} or * {@link JoinOperatorSets#where(org.apache.flink.api.java.functions.KeySelector)}. * @@ -894,10 +928,10 @@ public JoinOperatorSetsPredicate where(KeySelector keySelector) { return new JoinOperatorSetsPredicate(new SelectorFunctionKeys<>(input1.clean(keySelector), input1.getType(), keyType)); } - /** - * Intermediate step of a Join transformation.
- * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling + * Intermediate step of a Join transformation. + * + *

To continue the Join transformation, select the join key of the second input {@link DataSet} by calling * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}. */ @@ -910,10 +944,11 @@ private JoinOperatorSetsPredicate(Keys keys1) { /** * Continues a Join transformation and defines the {@link Tuple} fields of the second join - * {@link DataSet} that should be used as join keys.
- * Note: Fields can only be selected as join keys on Tuple DataSets.
- *

- * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with + * {@link DataSet} that should be used as join keys. + * + *

Note: Fields can only be selected as join keys on Tuple DataSets. + * + *

The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with * the element of the first input being the first field of the tuple and the element of the * second input being the second field of the tuple. * @@ -927,9 +962,9 @@ public DefaultJoin equalTo(int... fields) { /** * Continues a Join transformation and defines the fields of the second join - * {@link DataSet} that should be used as join keys.
- *

- * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with + * {@link DataSet} that should be used as join keys. + * + *

The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with * the element of the first input being the first field of the tuple and the element of the * second input being the second field of the tuple. * @@ -942,11 +977,12 @@ public DefaultJoin equalTo(String... fields) { } /** - * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.
- * The KeySelector function is called for each element of the second DataSet and extracts a single - * key value on which the DataSet is joined.
- *

- * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with + * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}. + * + *

The KeySelector function is called for each element of the second DataSet and extracts a single + * key value on which the DataSet is joined. + * + *

The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with * the element of the first input being the first field of the tuple and the element of the * second input being the second field of the tuple. * @@ -961,7 +997,6 @@ public DefaultJoin equalTo(KeySelector keySelector) { } } - // -------------------------------------------------------------------------------------------- // default join functions // -------------------------------------------------------------------------------------------- @@ -969,13 +1004,13 @@ public DefaultJoin equalTo(KeySelector keySelector) { @ForwardedFieldsFirst("*->0") @ForwardedFieldsSecond("*->1") @Internal - public static final class DefaultFlatJoinFunction extends RichFlatJoinFunction> { + private static final class DefaultFlatJoinFunction extends RichFlatJoinFunction> { private static final long serialVersionUID = 1L; private final Tuple2 outTuple = new Tuple2<>(); @Override - public void join(T1 first, T2 second, Collector> out) throws Exception { + public void join(T1 first, T2 second, Collector> out) throws Exception { outTuple.f0 = first; outTuple.f1 = second; out.collect(outTuple); @@ -983,26 +1018,26 @@ public void join(T1 first, T2 second, Collector> out) throws Excep } @Internal - public static final class ProjectFlatJoinFunction extends RichFlatJoinFunction { - + private static final class ProjectFlatJoinFunction extends RichFlatJoinFunction { + private static final long serialVersionUID = 1L; - + private final int[] fields; private final boolean[] isFromFirst; private final R outTuple; - + /** * Instantiates and configures a ProjectJoinFunction. * Creates output tuples by copying fields of joined input tuples (or a full input object) into an output tuple. - * - * @param fields List of indexes fields that should be copied to the output tuple. - * If the full input object should be copied (for example in case of a non-tuple input) the index should be -1. + * + * @param fields List of indexes fields that should be copied to the output tuple. + * If the full input object should be copied (for example in case of a non-tuple input) the index should be -1. * @param isFromFirst List of flags indicating whether the field should be copied from the first (true) or the second (false) input. * @param outTupleInstance An instance of an output tuple. */ private ProjectFlatJoinFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) { - if(fields.length != isFromFirst.length) { - throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); + if (fields.length != isFromFirst.length) { + throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); } this.fields = fields; @@ -1036,12 +1071,12 @@ public void join(T1 in1, T2 in2, Collector out) { } out.collect(outTuple); } - + } @Internal - public static final class JoinProjection { - + private static final class JoinProjection { + private final DataSet ds1; private final DataSet ds2; private final Keys keys1; @@ -1050,10 +1085,10 @@ public static final class JoinProjection { private int[] fieldIndexes; private boolean[] isFieldInFirst; - + private final int numFieldsDs1; private final int numFieldsDs2; - + public JoinProjection(DataSet ds1, DataSet ds2, Keys keys1, Keys keys2, JoinHint hint, int[] firstFieldIndexes, int[] secondFieldIndexes) { this.ds1 = ds1; this.ds2 = ds2; @@ -1063,32 +1098,32 @@ public JoinProjection(DataSet ds1, DataSet ds2, Keys keys1, Keys boolean isFirstTuple; boolean isSecondTuple; - - if(ds1.getType() instanceof TupleTypeInfo) { + + if (ds1.getType() instanceof TupleTypeInfo) { numFieldsDs1 = ds1.getType().getArity(); isFirstTuple = true; } else { numFieldsDs1 = 1; isFirstTuple = false; } - if(ds2.getType() instanceof TupleTypeInfo) { + if (ds2.getType() instanceof TupleTypeInfo) { numFieldsDs2 = ds2.getType().getArity(); isSecondTuple = true; } else { numFieldsDs2 = 1; isSecondTuple = false; } - + boolean isTuple; boolean firstInput; - - if(firstFieldIndexes != null && secondFieldIndexes == null) { + + if (firstFieldIndexes != null && secondFieldIndexes == null) { // index array for first input is provided firstInput = true; isTuple = isFirstTuple; this.fieldIndexes = firstFieldIndexes; - - if(this.fieldIndexes.length == 0) { + + if (this.fieldIndexes.length == 0) { // no indexes provided, treat tuple as regular object isTuple = false; } @@ -1097,8 +1132,8 @@ public JoinProjection(DataSet ds1, DataSet ds2, Keys keys1, Keys firstInput = false; isTuple = isSecondTuple; this.fieldIndexes = secondFieldIndexes; - - if(this.fieldIndexes.length == 0) { + + if (this.fieldIndexes.length == 0) { // no indexes provided, treat tuple as regular object isTuple = false; } @@ -1107,23 +1142,23 @@ public JoinProjection(DataSet ds1, DataSet ds2, Keys keys1, Keys } else { throw new IllegalArgumentException("You must provide at most one field index array."); } - - if(!isTuple && this.fieldIndexes.length != 0) { + + if (!isTuple && this.fieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectFirst() (or projectSecond()) without arguments to include it."); - } else if(this.fieldIndexes.length > 22) { + } else if (this.fieldIndexes.length > 22) { throw new IllegalArgumentException("You may select only up to twenty-two (22) fields."); } - - if(isTuple) { + + if (isTuple) { this.isFieldInFirst = new boolean[this.fieldIndexes.length]; - + // check field indexes and adapt to position in tuple int maxFieldIndex = firstInput ? numFieldsDs1 : numFieldsDs2; - for(int i=0; i ds1, DataSet ds2, Keys keys1, Keys } } - + /** - * Continues a ProjectJoin transformation and adds fields of the first join input.
- * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the first join input is not a Tuple DataSet, no parameters should be passed.
- * - * Fields of the first and second input can be added by chaining the method calls of - * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectFirst(int...)} and + * Continues a ProjectJoin transformation and adds fields of the first join input. + * + *

If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the first join input is not a Tuple DataSet, no parameters should be passed. + * + *

Fields of the first and second input can be added by chaining the method calls of + * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectSecond(int...)}. - * + * * @param firstFieldIndexes If the first input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. * The order of fields in the output tuple is defined by to the order of field indexes. * @return An extended JoinProjection. - * + * * @see Tuple * @see DataSet */ protected JoinProjection projectFirst(int... firstFieldIndexes) { - + boolean isFirstTuple; isFirstTuple = ds1.getType() instanceof TupleTypeInfo && firstFieldIndexes.length > 0; - - if(!isFirstTuple && firstFieldIndexes.length != 0) { + + if (!isFirstTuple && firstFieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectFirst() without arguments to include it."); - } else if(firstFieldIndexes.length > (22 - this.fieldIndexes.length)) { + } else if (firstFieldIndexes.length > (22 - this.fieldIndexes.length)) { // to many field indexes provided throw new IllegalArgumentException("You may select only up to twenty-two (22) fields in total."); } - + int offset = this.fieldIndexes.length; - - if(isFirstTuple) { + + if (isFirstTuple) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + firstFieldIndexes.length); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + firstFieldIndexes.length); - + // copy field indexes int maxFieldIndex = numFieldsDs1; - for(int i = 0; i < firstFieldIndexes.length; i++) { + for (int i = 0; i < firstFieldIndexes.length; i++) { // check if indexes in range Preconditions.checkElementIndex(firstFieldIndexes[i], maxFieldIndex); @@ -1187,59 +1223,60 @@ protected JoinProjection projectFirst(int... firstFieldIndexes) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + 1); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + 1); - + // add input object to output tuple this.isFieldInFirst[offset] = true; this.fieldIndexes[offset] = -1; } - + return this; } - + /** - * Continues a ProjectJoin transformation and adds fields of the second join input.
- * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. - * If the second join input is not a Tuple DataSet, no parameters should be passed.
- * - * Fields of the first and second input can be added by chaining the method calls of + * Continues a ProjectJoin transformation and adds fields of the second join input. + * + *

If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index. + * If the second join input is not a Tuple DataSet, no parameters should be passed. + * + *

Fields of the first and second input can be added by chaining the method calls of * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectFirst(int...)} and * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectSecond(int...)}. - * - * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. + * + * @param secondFieldIndexes If the second input is a Tuple DataSet, the indexes of the selected fields. * For a non-Tuple DataSet, do not provide parameters. * The order of fields in the output tuple is defined by to the order of field indexes. * @return An extended JoinProjection. - * + * * @see Tuple * @see DataSet */ protected JoinProjection projectSecond(int... secondFieldIndexes) { - + boolean isSecondTuple; isSecondTuple = ds2.getType() instanceof TupleTypeInfo && secondFieldIndexes.length > 0; - - if(!isSecondTuple && secondFieldIndexes.length != 0) { + + if (!isSecondTuple && secondFieldIndexes.length != 0) { // field index provided for non-Tuple input throw new IllegalArgumentException("Input is not a Tuple. Call projectSecond() without arguments to include it."); - } else if(secondFieldIndexes.length > (22 - this.fieldIndexes.length)) { + } else if (secondFieldIndexes.length > (22 - this.fieldIndexes.length)) { // to many field indexes provided throw new IllegalArgumentException("You may select only up to twenty-two (22) fields in total."); } - + int offset = this.fieldIndexes.length; - - if(isSecondTuple) { + + if (isSecondTuple) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + secondFieldIndexes.length); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + secondFieldIndexes.length); - + // copy field indexes int maxFieldIndex = numFieldsDs2; - for(int i = 0; i < secondFieldIndexes.length; i++) { + for (int i = 0; i < secondFieldIndexes.length; i++) { // check if indexes in range Preconditions.checkElementIndex(secondFieldIndexes[i], maxFieldIndex); - + this.isFieldInFirst[offset + i] = false; this.fieldIndexes[offset + i] = secondFieldIndexes[i]; } @@ -1247,27 +1284,27 @@ protected JoinProjection projectSecond(int... secondFieldIndexes) { // extend index and flag arrays this.fieldIndexes = Arrays.copyOf(this.fieldIndexes, this.fieldIndexes.length + 1); this.isFieldInFirst = Arrays.copyOf(this.isFieldInFirst, this.isFieldInFirst.length + 1); - + // add input object to output tuple this.isFieldInFirst[offset] = false; this.fieldIndexes[offset] = -1; } - + return this; } - - // -------------------------------------------------------------------------------------------- + + // -------------------------------------------------------------------------------------------- // The following lines are generated. - // -------------------------------------------------------------------------------------------- - // BEGIN_OF_TUPLE_DEPENDENT_CODE + // -------------------------------------------------------------------------------------------- + // BEGIN_OF_TUPLE_DEPENDENT_CODE // GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. /** * Chooses a projectTupleX according to the length of - * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#fieldIndexes} - * + * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#fieldIndexes}. + * * @return The projected DataSet. - * + * * @see org.apache.flink.api.java.operators.JoinOperator.ProjectJoin */ @SuppressWarnings("unchecked") @@ -1307,11 +1344,11 @@ public ProjectJoin projectTupleX() { } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1323,11 +1360,11 @@ public ProjectJoin> projectTuple1() { } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1339,11 +1376,11 @@ public ProjectJoin> projectTuple2() { } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1355,11 +1392,11 @@ public ProjectJoin> projectTuple3() { } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1371,11 +1408,11 @@ public ProjectJoin> projectTuple } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1387,11 +1424,11 @@ public ProjectJoin> proj } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1403,11 +1440,11 @@ public ProjectJoin ProjectJoin ProjectJoin ProjectJoin ProjectJoin ProjectJoin ProjectJoin ProjectJoin ProjectJoin< } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1547,11 +1584,11 @@ public Project } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1563,11 +1600,11 @@ public Pr } /** - * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -1579,11 +1616,11 @@ public [] extractFieldTypes(int[] fields) { - + TypeInformation[] fieldTypes = new TypeInformation[fields.length]; - for(int i=0; i typeInfo; - if(isFieldInFirst[i]) { - if(fields[i] >= 0) { - typeInfo = ((TupleTypeInfo)ds1.getType()).getTypeAt(fields[i]); + if (isFieldInFirst[i]) { + if (fields[i] >= 0) { + typeInfo = ((TupleTypeInfo) ds1.getType()).getTypeAt(fields[i]); } else { typeInfo = ds1.getType(); } } else { - if(fields[i] >= 0) { - typeInfo = ((TupleTypeInfo)ds2.getType()).getTypeAt(fields[i]); + if (fields[i] >= 0) { + typeInfo = ((TupleTypeInfo) ds2.getType()).getTypeAt(fields[i]); } else { typeInfo = ds2.getType(); } @@ -1732,9 +1769,9 @@ private TypeInformation[] extractFieldTypes(int[] fields) { fieldTypes[i] = typeInfo; } - + return fieldTypes; } - + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java index 057048c23f019..f6336cde2b755 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java @@ -20,10 +20,10 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; import org.apache.flink.api.java.operators.translation.KeyRemovingMapper; import org.apache.flink.api.java.operators.translation.TwoKeyExtractingMapper; @@ -41,8 +41,7 @@ public class KeyFunctions { @SuppressWarnings("unchecked") public static org.apache.flink.api.common.operators.Operator> appendKeyExtractor( org.apache.flink.api.common.operators.Operator input, - SelectorFunctionKeys key) - { + SelectorFunctionKeys key) { TypeInformation inputType = key.getInputType(); TypeInformation> typeInfoWithKey = createTypeWithKey(key); @@ -65,8 +64,7 @@ public static org.apache.flink.api.common.operators.Operator public static org.apache.flink.api.common.operators.Operator> appendKeyExtractor( org.apache.flink.api.common.operators.Operator input, SelectorFunctionKeys key1, - SelectorFunctionKeys key2) - { + SelectorFunctionKeys key2) { TypeInformation inputType = key1.getInputType(); TypeInformation> typeInfoWithKey = createTypeWithKey(key1, key2); @@ -88,8 +86,7 @@ public static org.apache.flink.api.common.operators.Operator org.apache.flink.api.common.operators.SingleInputOperator appendKeyRemover( org.apache.flink.api.common.operators.Operator> inputWithKey, - SelectorFunctionKeys key) - { + SelectorFunctionKeys key) { TypeInformation inputType = key.getInputType(); TypeInformation> typeInfoWithKey = createTypeWithKey(key); @@ -107,15 +104,13 @@ public static org.apache.flink.api.common.operators.Operator TypeInformation> createTypeWithKey( - SelectorFunctionKeys key) - { + SelectorFunctionKeys key) { return new TupleTypeInfo<>(key.getKeyType(), key.getInputType()); } public static TypeInformation> createTypeWithKey( SelectorFunctionKeys key1, - SelectorFunctionKeys key2) - { + SelectorFunctionKeys key2) { return new TupleTypeInfo<>(key1.getKeyType(), key2.getKeyType(), key1.getInputType()); } } 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 2f398fb6180b7..a9d5672d6d7ba 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 @@ -29,52 +29,52 @@ /** * This operator represents the application of a "map" function on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. - * + * * @see org.apache.flink.api.common.functions.MapFunction */ @Public public class MapOperator extends SingleInputUdfOperator> { - + protected final MapFunction function; - + protected final String defaultName; public MapOperator(DataSet input, TypeInformation resultType, MapFunction function, String defaultName) { super(input, resultType); - + this.defaultName = defaultName; this.function = function; UdfOperatorUtils.analyzeSingleInputUdf(this, MapFunction.class, defaultName, function, null); } - + @Override protected MapFunction getFunction() { return function; } - + @Override protected MapOperatorBase> translateToDataFlow(Operator input) { - - String name = getName() != null ? getName() : "Map at "+defaultName; + + String name = getName() != null ? getName() : "Map at " + defaultName; // create operator MapOperatorBase> po = new MapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set parallelism - if(this.getParallelism() > 0) { + if (this.getParallelism() > 0) { // use specified parallelism po.setParallelism(this.getParallelism()); } else { // if no parallelism has been specified, use parallelism of input operator to enable chaining po.setParallelism(input.getParallelism()); } - + return po; } - + } 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 6bc48b8d7784f..e03f39d2b3724 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 @@ -29,48 +29,48 @@ /** * This operator represents the application of a "mapPartition" function on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set consumed by the operator. * @param The type of the data set created by the operator. - * + * * @see MapPartitionFunction */ @Public public class MapPartitionOperator extends SingleInputUdfOperator> { - + protected final MapPartitionFunction function; - + protected final String defaultName; - + public MapPartitionOperator(DataSet input, TypeInformation resultType, MapPartitionFunction function, String defaultName) { super(input, resultType); - + this.function = function; this.defaultName = defaultName; } - + @Override protected MapPartitionFunction getFunction() { return function; } - + @Override protected MapPartitionOperatorBase> translateToDataFlow(Operator input) { - - String name = getName() != null ? getName() : "MapPartition at "+defaultName; + + String name = getName() != null ? getName() : "MapPartition at " + defaultName; // create operator MapPartitionOperatorBase> po = new MapPartitionOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); // set parallelism - if(this.getParallelism() > 0) { + if (this.getParallelism() > 0) { // use specified parallelism po.setParallelism(this.getParallelism()); } else { // if no parallelism has been specified, use parallelism of input operator to enable chaining po.setParallelism(input.getParallelism()); } - + return po; } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java index e496c62e4b7c1..463dc3532db97 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java @@ -28,7 +28,7 @@ /** * Base class of all operators in the Java API. - * + * * @param The type of the data set produced by this operator. * @param The type of the operator, so that we can return it. */ @@ -36,21 +36,20 @@ public abstract class Operator> extends DataSet { protected String name; - + protected int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; protected ResourceSpec minResources = ResourceSpec.DEFAULT; protected ResourceSpec preferredResources = ResourceSpec.DEFAULT; - protected Operator(ExecutionEnvironment context, TypeInformation resultType) { super(context, resultType); } - + /** * Returns the type of the result of this operator. - * + * * @return The result type of the operator. */ public TypeInformation getResultType() { @@ -60,16 +59,16 @@ public TypeInformation getResultType() { /** * Returns the name of the operator. If no name has been set, it returns the name of the * operation, or the name of the class implementing the function of this operator. - * + * * @return The name of the operator. */ public String getName() { return name; } - + /** * Returns the parallelism of this operator. - * + * * @return The parallelism of this operator. */ public int getParallelism() { @@ -100,7 +99,7 @@ public ResourceSpec getPreferredResources() { * Sets the name of this operator. This overrides the default name, which is either * a generated description of the operation (such as for example "Aggregate(1:SUM, 2:MIN)") * or the name the user-defined function or input/output format executed by the operator. - * + * * @param newName The name for this operator. * @return The operator with a new name. */ @@ -110,11 +109,11 @@ public O name(String newName) { O returnType = (O) this; return returnType; } - + /** * Sets the parallelism for this operator. * The parallelism must be 1 or more. - * + * * @param parallelism The parallelism for this operator. A value equal to {@link ExecutionConfig#PARALLELISM_DEFAULT} * will use the system default. * @return The operator with set parallelism. 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 22b9186eae191..facadc015391c 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 @@ -36,31 +36,32 @@ import java.util.List; import java.util.Map; +/** + * Used for translating data sets into corresponding operators. + */ @Internal public class OperatorTranslation { - - /** The already translated operations */ + + /** The already translated operations. */ private Map, Operator> translated = new HashMap<>(); - - + public Plan translateToPlan(List> sinks, String jobName) { List> planSinks = new ArrayList<>(); - + for (DataSink sink : sinks) { planSinks.add(translate(sink)); } - + Plan p = new Plan(planSinks); p.setJobName(jobName); return p; } - - + private GenericDataSinkBase translate(DataSink sink) { - + // translate the input recursively Operator input = translate(sink.getDataSet()); - + // translate the sink itself and connect it to the input GenericDataSinkBase translatedSink = sink.translateToDataFlow(input); @@ -68,8 +69,7 @@ private GenericDataSinkBase translate(DataSink sink) { return translatedSink; } - - + private Operator translate(DataSet dataSet) { while (dataSet instanceof NoOpOperator) { dataSet = ((NoOpOperator) dataSet).getInput(); @@ -89,9 +89,9 @@ private Operator translate(DataSet dataSet) { return typedPrevious; } } - + Operator dataFlowOp; - + if (dataSet instanceof DataSource) { DataSource dataSource = (DataSource) dataSet; dataFlowOp = dataSource.translateToDataFlow(); @@ -126,28 +126,27 @@ else if (dataSet instanceof DeltaIteration.SolutionSetPlaceHolder || dataSet ins else { throw new RuntimeException("Error while creating the data flow plan for the program: Unknown operator or data set type: " + dataSet); } - + this.translated.put(dataSet, dataFlowOp); - + // take care of broadcast variables translateBcVariables(dataSet, dataFlowOp); - + return dataFlowOp; } - - + private org.apache.flink.api.common.operators.Operator translateSingleInputOperator(SingleInputOperator op) { - + @SuppressWarnings("unchecked") SingleInputOperator typedOp = (SingleInputOperator) op; - + @SuppressWarnings("unchecked") DataSet typedInput = (DataSet) op.getInput(); - + Operator input = translate(typedInput); - + org.apache.flink.api.common.operators.Operator dataFlowOp = typedOp.translateToDataFlow(input); - + if (op instanceof UdfOperator) { @SuppressWarnings("unchecked") SingleInputUdfOperator udfOp = (SingleInputUdfOperator) op; @@ -165,29 +164,29 @@ private org.apache.flink.api.common.operators.Operator translateSingle unaryOp.setSemanticProperties(udfOp.getSemanticProperties()); } } - + return dataFlowOp; } - + private org.apache.flink.api.common.operators.Operator translateTwoInputOperator(TwoInputOperator op) { - + @SuppressWarnings("unchecked") TwoInputOperator typedOp = (TwoInputOperator) op; - + @SuppressWarnings("unchecked") DataSet typedInput1 = (DataSet) op.getInput1(); @SuppressWarnings("unchecked") DataSet typedInput2 = (DataSet) op.getInput2(); - + Operator input1 = translate(typedInput1); Operator input2 = translate(typedInput2); - + org.apache.flink.api.common.operators.Operator dataFlowOp = typedOp.translateToDataFlow(input1, input2); - - if (op instanceof UdfOperator ) { + + if (op instanceof UdfOperator) { @SuppressWarnings("unchecked") TwoInputUdfOperator udfOp = (TwoInputUdfOperator) op; - + // set configuration parameters Configuration opParams = udfOp.getParameters(); if (opParams != null) { @@ -201,16 +200,14 @@ private org.apache.flink.api.common.operators.Operator translateT binaryOp.setSemanticProperties(udfOp.getSemanticProperties()); } } - + return dataFlowOp; } - - + private BulkIterationBase translateBulkIteration(BulkIterationResultSet untypedIterationEnd) { @SuppressWarnings("unchecked") BulkIterationResultSet iterationEnd = (BulkIterationResultSet) untypedIterationEnd; IterativeDataSet iterationHead = iterationEnd.getIterationHead(); - BulkIterationBase iterationOperator = new BulkIterationBase<>(new UnaryOperatorInformation<>(iterationEnd.getType(), iterationEnd.getType()), "Bulk Iteration"); @@ -224,28 +221,28 @@ private BulkIterationBase translateBulkIteration(BulkIterationResultSet DeltaIterationBase translateDeltaIteration(DeltaIterationResultSet untypedIterationEnd) { @SuppressWarnings("unchecked") DeltaIterationResultSet iterationEnd = (DeltaIterationResultSet) untypedIterationEnd; DeltaIteration iterationHead = iterationEnd.getIterationHead(); - + String name = iterationHead.getName() == null ? "Unnamed Delta Iteration" : iterationHead.getName(); - + DeltaIterationBase iterationOperator = new DeltaIterationBase<>(new BinaryOperatorInformation<>(iterationEnd.getType(), iterationEnd.getWorksetType(), iterationEnd.getType()), iterationEnd.getKeyPositions(), name); - + iterationOperator.setMaximumNumberOfIterations(iterationEnd.getMaxIterations()); - + if (iterationHead.getParallelism() > 0) { iterationOperator.setParallelism(iterationHead.getParallelism()); } @@ -258,31 +255,31 @@ private DeltaIterationBase translateDeltaIteration(DeltaIterationRe Operator translatedSolutionSet = translate(iterationEnd.getNextSolutionSet()); Operator translatedWorkset = translate(iterationEnd.getNextWorkset()); - + iterationOperator.setNextWorkset(translatedWorkset); iterationOperator.setSolutionSetDelta(translatedSolutionSet); iterationOperator.setInitialSolutionSet(translate(iterationHead.getInitialSolutionSet())); iterationOperator.setInitialWorkset(translate(iterationHead.getInitialWorkset())); - + // register all aggregators iterationOperator.getAggregators().addAll(iterationHead.getAggregators()); - + iterationOperator.setSolutionSetUnManaged(iterationHead.isSolutionSetUnManaged()); - + return iterationOperator; } - + private void translateBcVariables(DataSet setOrOp, Operator dataFlowOp) { // check if this is actually an operator that could have broadcast variables if (setOrOp instanceof UdfOperator) { if (!(dataFlowOp instanceof AbstractUdfOperator)) { throw new RuntimeException("Error while creating the data flow plan for the program: A UDF operation was not translated to a UDF operator."); } - + UdfOperator udfOp = (UdfOperator) setOrOp; AbstractUdfOperator udfDataFlowOp = (AbstractUdfOperator) dataFlowOp; - + for (Map.Entry> bcVariable : udfOp.getBroadcastSets().entrySet()) { Operator bcInput = translate(bcVariable.getValue()); udfDataFlowOp.setBroadcastVariable(bcVariable.getKey(), bcInput); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java index b3234b8e1e40b..8a9a72d8e42c3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java @@ -18,13 +18,13 @@ package org.apache.flink.api.java.operators; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.distributions.DataDistribution; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -33,7 +33,6 @@ 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.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Preconditions; @@ -46,7 +45,7 @@ */ @Public public class PartitionOperator extends SingleInputOperator> { - + private final Keys pKeys; private final PartitionMethod pMethod; private final String partitionLocationName; @@ -54,7 +53,6 @@ public class PartitionOperator extends SingleInputOperator input, PartitionMethod pMethod, Keys pKeys, String partitionLocationName) { this(input, pMethod, pKeys, null, null, null, partitionLocationName); } @@ -66,37 +64,35 @@ public PartitionOperator(DataSet input, PartitionMethod pMethod, Keys pKey public PartitionOperator(DataSet input, PartitionMethod pMethod, String partitionLocationName) { this(input, pMethod, null, null, null, null, partitionLocationName); } - + public PartitionOperator(DataSet input, Keys pKeys, Partitioner customPartitioner, String partitionLocationName) { this(input, PartitionMethod.CUSTOM, pKeys, customPartitioner, null, null, partitionLocationName); } - + public

PartitionOperator(DataSet input, Keys pKeys, Partitioner

customPartitioner, - TypeInformation

partitionerTypeInfo, String partitionLocationName) - { + TypeInformation

partitionerTypeInfo, String partitionLocationName) { this(input, PartitionMethod.CUSTOM, pKeys, customPartitioner, partitionerTypeInfo, null, partitionLocationName); } - + private

PartitionOperator(DataSet input, PartitionMethod pMethod, Keys pKeys, Partitioner

customPartitioner, - TypeInformation

partitionerTypeInfo, DataDistribution distribution, String partitionLocationName) - { + TypeInformation

partitionerTypeInfo, DataDistribution distribution, String partitionLocationName) { super(input, input.getType()); - + Preconditions.checkNotNull(pMethod); Preconditions.checkArgument(pKeys != null || pMethod == PartitionMethod.REBALANCE, "Partitioning requires keys"); Preconditions.checkArgument(pMethod != PartitionMethod.CUSTOM || customPartitioner != null, "Custom partioning requires a partitioner."); Preconditions.checkArgument(distribution == null || pMethod == PartitionMethod.RANGE, "Customized data distribution is only neccessary for range partition."); - + if (distribution != null) { Preconditions.checkArgument(pKeys.getNumberOfKeyFields() <= distribution.getNumberOfFields(), "The distribution must provide at least as many fields as flat key fields are specified."); Preconditions.checkArgument(Arrays.equals(pKeys.getKeyFieldTypes(), Arrays.copyOfRange(distribution.getKeyTypes(), 0, pKeys.getNumberOfKeyFields())), "The types of the flat key fields must be equal to the types of the fields of the distribution."); } - + if (customPartitioner != null) { pKeys.validateCustomPartitioner(customPartitioner, partitionerTypeInfo); } - + this.pMethod = pMethod; this.pKeys = pKeys; this.partitionLocationName = partitionLocationName; @@ -121,43 +117,43 @@ public PartitionOperator withOrders(Order... orders) { return this; } - + // -------------------------------------------------------------------------------------------- // Properties // -------------------------------------------------------------------------------------------- - + /** * Gets the custom partitioner from this partitioning. - * + * * @return The custom partitioner. */ @Internal public Partitioner getCustomPartitioner() { return customPartitioner; } - + // -------------------------------------------------------------------------------------------- // Translation // -------------------------------------------------------------------------------------------- - + protected org.apache.flink.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { - + String name = "Partition at " + partitionLocationName; - + // distinguish between partition types if (pMethod == PartitionMethod.REBALANCE) { - + UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getType(), getType()); PartitionOperatorBase rebalancedInput = new PartitionOperatorBase<>(operatorInfo, pMethod, name); rebalancedInput.setInput(input); rebalancedInput.setParallelism(getParallelism()); - + return rebalancedInput; - } + } else if (pMethod == PartitionMethod.HASH || pMethod == PartitionMethod.CUSTOM || pMethod == PartitionMethod.RANGE) { - + if (pKeys instanceof Keys.ExpressionKeys) { - + int[] logicalKeyPositions = pKeys.computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getType(), getType()); PartitionOperatorBase partitionedInput = new PartitionOperatorBase<>(operatorInfo, pMethod, logicalKeyPositions, name); @@ -166,11 +162,11 @@ else if (pMethod == PartitionMethod.HASH || pMethod == PartitionMethod.CUSTOM || partitionedInput.setDistribution(distribution); partitionedInput.setCustomPartitioner(customPartitioner); partitionedInput.setOrdering(computeOrdering(pKeys, orders)); - + return partitionedInput; } else if (pKeys instanceof Keys.SelectorFunctionKeys) { - + @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys = (Keys.SelectorFunctionKeys) pKeys; return translateSelectorFunctionPartitioner(selectorKeys, pMethod, name, input, getParallelism(), @@ -179,8 +175,8 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) { else { throw new UnsupportedOperationException("Unrecognized key type."); } - - } + + } else { throw new UnsupportedOperationException("Unsupported partitioning method: " + pMethod.name()); } @@ -217,8 +213,7 @@ private static Ordering computeOrdering(Keys pKeys, Order[] orders) { Operator input, int partitionDop, Partitioner customPartitioner, - Order[] orders) - { + Order[] orders) { final SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); @@ -234,5 +229,4 @@ private static Ordering computeOrdering(Keys pKeys, Order[] orders) { return KeyFunctions.appendKeyRemover(keyedPartitionedInput, keys); } - } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index 00685824de8c6..d8cc9166a17d8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -18,8 +18,6 @@ package org.apache.flink.api.java.operators; -import java.util.Arrays; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -31,34 +29,58 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.translation.PlanProjectOperator; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple10; +import org.apache.flink.api.java.tuple.Tuple11; +import org.apache.flink.api.java.tuple.Tuple12; +import org.apache.flink.api.java.tuple.Tuple13; +import org.apache.flink.api.java.tuple.Tuple14; +import org.apache.flink.api.java.tuple.Tuple15; +import org.apache.flink.api.java.tuple.Tuple16; +import org.apache.flink.api.java.tuple.Tuple17; +import org.apache.flink.api.java.tuple.Tuple18; +import org.apache.flink.api.java.tuple.Tuple19; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple20; +import org.apache.flink.api.java.tuple.Tuple21; +import org.apache.flink.api.java.tuple.Tuple22; +import org.apache.flink.api.java.tuple.Tuple23; +import org.apache.flink.api.java.tuple.Tuple24; +import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.tuple.Tuple6; +import org.apache.flink.api.java.tuple.Tuple7; +import org.apache.flink.api.java.tuple.Tuple8; +import org.apache.flink.api.java.tuple.Tuple9; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.util.Preconditions; -//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator -import org.apache.flink.api.java.tuple.*; -//CHECKSTYLE.ON: AvoidStarImport +import java.util.Arrays; /** * This operator represents the application of a projection operation on a data set, and the * result data set produced by the function. - * + * * @param The type of the data set projected by the operator. * @param The type of data set that is the result of the projection. */ @Public -public class ProjectOperator +public class ProjectOperator extends SingleInputOperator> { - + protected final int[] fields; public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo returnType) { super(input, returnType); - + this.fields = fields; } @Override - protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { + protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection " + Arrays.toString(fields); // create operator PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType(), context.getConfig()); @@ -70,6 +92,7 @@ protected org.apache.flink.api.common.operators.base.MapOperatorBase ProjectOperator types(Class... types) { - TupleTypeInfo typeInfo = (TupleTypeInfo)this.getResultType(); + TupleTypeInfo typeInfo = (TupleTypeInfo) this.getResultType(); - if(types.length != typeInfo.getArity()) { + if (types.length != typeInfo.getArity()) { throw new InvalidProgramException("Provided types do not match projection."); } - for (int i=0; i typeClass = types[i]; if (!typeClass.equals(typeInfo.getTypeAt(i).getTypeClass())) { - throw new InvalidProgramException("Provided type "+typeClass.getSimpleName()+" at position "+i+" does not match projection"); + throw new InvalidProgramException("Provided type " + typeClass.getSimpleName() + " at position " + i + " does not match projection"); } } return (ProjectOperator) this; } + /** + * A projection of {@link DataSet}. + * + * @param + */ @Internal public static class Projection { - + private final DataSet ds; private int[] fieldIndexes; - + public Projection(DataSet ds, int[] fieldIndexes) { - - if(!(ds.getType() instanceof TupleTypeInfo)) { + + if (!(ds.getType() instanceof TupleTypeInfo)) { throw new UnsupportedOperationException("project() can only be applied to DataSets of Tuples."); } - - if(fieldIndexes.length == 0) { + + if (fieldIndexes.length == 0) { throw new IllegalArgumentException("project() needs to select at least one (1) field."); - } else if(fieldIndexes.length > Tuple.MAX_ARITY - 1) { + } else if (fieldIndexes.length > Tuple.MAX_ARITY - 1) { throw new IllegalArgumentException( - "project() may select only up to (" + (Tuple.MAX_ARITY - 1) + ") fields."); + "project() may select only up to (" + (Tuple.MAX_ARITY - 1) + ") fields."); } - + int maxFieldIndex = ds.getType().getArity(); for (int fieldIndexe : fieldIndexes) { Preconditions.checkElementIndex(fieldIndexe, maxFieldIndex); } - + this.ds = ds; this.fieldIndexes = fieldIndexes; } - - - // -------------------------------------------------------------------------------------------- + + // -------------------------------------------------------------------------------------------- // The following lines are generated. - // -------------------------------------------------------------------------------------------- - // BEGIN_OF_TUPLE_DEPENDENT_CODE - // GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. + // -------------------------------------------------------------------------------------------- + // BEGIN_OF_TUPLE_DEPENDENT_CODE + // GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. /** * Chooses a projectTupleX according to the length of - * {@link org.apache.flink.api.java.operators.ProjectOperator.Projection#fieldIndexes} - * + * {@link org.apache.flink.api.java.operators.ProjectOperator.Projection#fieldIndexes}. + * * @return The projected DataSet. - * * @see org.apache.flink.api.java.operators.ProjectOperator.Projection */ @SuppressWarnings("unchecked") @@ -171,10 +197,9 @@ public ProjectOperator projectTupleX() { } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -186,10 +211,9 @@ public ProjectOperator> projectTuple1() { } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -201,10 +225,9 @@ public ProjectOperator> projectTuple2() { } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -216,10 +239,9 @@ public ProjectOperator> projectTuple3() { } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -231,10 +253,9 @@ public ProjectOperator> projectTuple4 } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -246,10 +267,9 @@ public ProjectOperator> proje } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -261,10 +281,9 @@ public ProjectOperator ProjectOperator ProjectOperator ProjectOperator ProjectOperator ProjectOperator ProjectOperator ProjectOperator ProjectOpera } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -396,10 +407,9 @@ public Project } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -411,10 +421,9 @@ public Pr } /** - * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. - * + * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. + * * @return The projected DataSet. - * * @see Tuple * @see DataSet */ @@ -426,10 +435,9 @@ public [] extractFieldTypes(int[] fields, TypeInformation inType) { - + TupleTypeInfo inTupleType = (TupleTypeInfo) inType; TypeInformation[] fieldTypes = new TypeInformation[fields.length]; - - for(int i=0; i The type of the data set reduced by the operator. - * + * * @see org.apache.flink.api.common.functions.ReduceFunction */ @Public public class ReduceOperator extends SingleInputUdfOperator> { - + private final ReduceFunction function; - + private final Grouping grouper; - + private final String defaultName; // should be null in case of an all reduce private CombineHint hint; - + /** - * * This is the case for a reduce-all case (in contrast to the reduce-per-group case). - * + * * @param input * @param function */ public ReduceOperator(DataSet input, ReduceFunction function, String defaultName) { super(input, input.getType()); - + this.function = function; this.grouper = null; this.defaultName = defaultName; this.hint = null; } - - + public ReduceOperator(Grouping input, ReduceFunction function, String defaultName) { super(input.getInputDataSet(), input.getInputDataSet().getType()); - + this.function = function; this.grouper = input; this.defaultName = defaultName; @@ -82,7 +80,7 @@ public ReduceOperator(Grouping input, ReduceFunction function, String de UdfOperatorUtils.analyzeSingleInputUdf(this, ReduceFunction.class, defaultName, function, grouper.keys); } - + @Override protected ReduceFunction getFunction() { return function; @@ -95,12 +93,12 @@ public SingleInputSemanticProperties getSemanticProperties() { SingleInputSemanticProperties props = super.getSemanticProperties(); // offset semantic information by extracted key fields - if(props != null && + if (props != null && this.grouper != null && this.grouper.keys instanceof SelectorFunctionKeys) { - int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); - if(this.grouper instanceof SortedGrouping) { + int offset = ((SelectorFunctionKeys) this.grouper.keys).getKeyType().getTotalFields(); + if (this.grouper instanceof SortedGrouping) { offset += ((SortedGrouping) this.grouper).getSortSelectionFunctionKey().getKeyType().getTotalFields(); } props = SemanticPropUtil.addSourceFieldOffset(props, this.getInputType().getTotalFields(), offset); @@ -111,25 +109,25 @@ public SingleInputSemanticProperties getSemanticProperties() { @Override protected org.apache.flink.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { - - String name = getName() != null ? getName() : "Reduce at "+defaultName; - + + String name = getName() != null ? getName() : "Reduce at " + defaultName; + // distinguish between grouped reduce and non-grouped reduce if (grouper == null) { // non grouped reduce UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getInputType(), getInputType()); ReduceOperatorBase> po = new ReduceOperatorBase<>(function, operatorInfo, new int[0], name); - + po.setInput(input); // the parallelism for a non grouped reduce can only be 1 po.setParallelism(1); - + return po; } - + if (grouper.getKeys() instanceof SelectorFunctionKeys) { - + // reduce with key selector function @SuppressWarnings("unchecked") SelectorFunctionKeys selectorKeys = (SelectorFunctionKeys) grouper.getKeys(); @@ -141,19 +139,19 @@ public SingleInputSemanticProperties getSemanticProperties() { return po; } else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { - + // reduce with field positions int[] logicalKeyPositions = grouper.getKeys().computeLogicalKeyPositions(); UnaryOperatorInformation operatorInfo = new UnaryOperatorInformation<>(getInputType(), getInputType()); ReduceOperatorBase> po = new ReduceOperatorBase<>(function, operatorInfo, logicalKeyPositions, name); - + po.setCustomPartitioner(grouper.getCustomPartitioner()); - + po.setInput(input); po.setParallelism(getParallelism()); po.setCombineHint(hint); - + return po; } else { @@ -164,7 +162,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { /** * Sets the strategy to use for the combine phase of the reduce. * - * If this method is not called, then the default hint will be used. + *

If this method is not called, then the default hint will be used. * ({@link org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint#OPTIMIZER_CHOOSES}) * * @param strategy The hint to use. @@ -177,7 +175,7 @@ public ReduceOperator setCombineHint(CombineHint strategy) { } // -------------------------------------------------------------------------------------------- - + private static org.apache.flink.api.common.operators.SingleInputOperator translateSelectorFunctionReducer( SelectorFunctionKeys rawKeys, ReduceFunction function, @@ -185,14 +183,13 @@ public ReduceOperator setCombineHint(CombineHint strategy) { String name, Operator input, int parallelism, - CombineHint hint) - { + CombineHint hint) { @SuppressWarnings("unchecked") final SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; - + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); - + PlanUnwrappingReduceOperator reducer = new PlanUnwrappingReduceOperator<>(function, keys, name, inputType, typeInfoWithKey); reducer.setInput(keyedInput); reducer.setParallelism(parallelism); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputOperator.java index 359f4b80e70e6..b718a56571ed2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputOperator.java @@ -24,47 +24,46 @@ /** * Base class for operations that operates on a single input data set. - * + * * @param The data type of the input data set. * @param The data type of the returned data set. */ @Public public abstract class SingleInputOperator> extends Operator { - + private final DataSet input; - - + protected SingleInputOperator(DataSet input, TypeInformation resultType) { super(input.getExecutionEnvironment(), resultType); this.input = input; } - + /** * Gets the data set that this operation uses as its input. - * + * * @return The data set that this operation uses as its input. */ public DataSet getInput() { return this.input; } - + /** * Gets the type information of the data type of the input data set. * This method returns equivalent information as {@code getInput().getType()}. - * + * * @return The input data type. */ public TypeInformation getInputType() { return this.input.getType(); } - + /** * Translates this operation to a data flow operator of the common data flow API. - * + * * @param input The data flow operator that produces this operation's input data. * @return The translated data flow operator. */ protected abstract org.apache.flink.api.common.operators.Operator translateToDataFlow( org.apache.flink.api.common.operators.Operator input); - + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java index eb485fe764714..4ce44aa2fdd4b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java @@ -18,12 +18,6 @@ package org.apache.flink.api.java.operators; -import java.lang.annotation.Annotation; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -39,6 +33,12 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; +import java.lang.annotation.Annotation; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + import static java.util.Objects.requireNonNull; /** @@ -46,16 +46,15 @@ * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that * have one input (such as {@link org.apache.flink.api.common.functions.RichMapFunction} or * {@link org.apache.flink.api.common.functions.RichReduceFunction}). - *

- * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization + * + *

This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. * @param The data type of the input data set. * @param The data type of the returned data set. */ @Public public abstract class SingleInputUdfOperator> - extends SingleInputOperator implements UdfOperator -{ + extends SingleInputOperator implements UdfOperator { private Configuration parameters; private Map> broadcastVariables; @@ -77,8 +76,7 @@ public abstract class SingleInputUdfOperator input, TypeInformation resultType) { super(input, resultType); } - - + protected abstract Function getFunction(); // -------------------------------------------------------------------------------------------- @@ -102,7 +100,7 @@ public O withBroadcastSet(DataSet data, String name) { if (name == null) { throw new IllegalArgumentException("Broadcast variable name must not be null."); } - + if (this.broadcastVariables == null) { this.broadcastVariables = new HashMap>(); } @@ -115,46 +113,34 @@ public O withBroadcastSet(DataSet data, String name) { } /** - *

* Adds semantic information about forwarded fields of the user-defined function. * The forwarded fields information declares fields which are never modified by the function and * which are forwarded at the same position to the output or unchanged copied to another position in the output. - *

* - *

- * Fields that are forwarded at the same position are specified by their position. + *

Fields that are forwarded at the same position are specified by their position. * The specified position must be valid for the input and output data type and have the same type. * For example withForwardedFields("f2") declares that the third field of a Java input tuple is * copied to the third field of an output tuple. - *

* - *

- * Fields which are unchanged copied to another position in the output are declared by specifying the + *

Fields which are unchanged copied to another position in the output are declared by specifying the * source field reference in the input and the target field reference in the output. * {@code withForwardedFields("f0->f2")} denotes that the first field of the Java input tuple is * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that * the number of declared fields and their types in input and output type match. - *

* - *

- * Multiple forwarded fields can be annotated in one ({@code withForwardedFields("f2; f3->f0; f4")}) + *

Multiple forwarded fields can be annotated in one ({@code withForwardedFields("f2; f3->f0; f4")}) * or separate Strings ({@code withForwardedFields("f2", "f3->f0", "f4")}). * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for * details on field references such as nested fields and wildcard. - *

* - *

- * It is not possible to override existing semantic information about forwarded fields which was + *

It is not possible to override existing semantic information about forwarded fields which was * for example added by a {@link org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields} class annotation. - *

* - *

- * NOTE: Adding semantic information for functions is optional! + *

NOTE: Adding semantic information for functions is optional! * If used correctly, semantic information can help the Flink optimizer to generate more efficient execution plans. * However, incorrect semantic information can cause the optimizer to generate incorrect execution plans which compute wrong results! * So be careful when adding semantic information. * - *

* * @param forwardedFields A list of field forward expressions. * @return This operator with annotated forwarded field information. @@ -164,17 +150,17 @@ public O withBroadcastSet(DataSet data, String name) { */ public O withForwardedFields(String... forwardedFields) { - if(this.udfSemantics == null) { + if (this.udfSemantics == null) { // extract semantic properties from function annotations setSemanticProperties(extractSemanticAnnotations(getFunction().getClass())); } - if(this.udfSemantics == null + if (this.udfSemantics == null || this.analyzedUdfSemantics) { // discard analyzed semantic properties setSemanticProperties(new SingleInputSemanticProperties()); SemanticPropUtil.getSemanticPropsSingleFromString(this.udfSemantics, forwardedFields, null, null, this.getInputType(), this.getResultType()); } else { - if(udfWithForwardedFieldsAnnotation(getFunction().getClass())) { + if (udfWithForwardedFieldsAnnotation(getFunction().getClass())) { // refuse semantic information as it would override the function annotation throw new SemanticProperties.InvalidSemanticAnnotationException("Forwarded field information " + "has already been added by a function annotation for this operator. " + @@ -202,7 +188,7 @@ public O withForwardedFields(String... forwardedFields) { *

Classes can be used as type hints for non-generic types (classes without generic parameters), * but not for generic types like for example Tuples. For those generic types, please * use the {@link #returns(TypeHint)} method. - * + * *

Use this method the following way: *

{@code
 	 *     DataSet result =
@@ -215,7 +201,7 @@ public O withForwardedFields(String... forwardedFields) {
 	 */
 	public O returns(Class typeClass) {
 		requireNonNull(typeClass, "type class must not be null");
-		
+
 		try {
 			return returns(TypeInformation.of(typeClass));
 		}
@@ -225,7 +211,7 @@ public O returns(Class typeClass) {
 					"please use the 'returns(TypeHint)' method instead.", e);
 		}
 	}
-	
+
 	/**
 	 * Adds a type information hint about the return type of this operator. This method
 	 * can be used in cases where Flink cannot determine automatically what the produced
@@ -244,7 +230,7 @@ public O returns(Class typeClass) {
 	 */
 	public O returns(TypeHint typeHint) {
 		requireNonNull(typeHint, "TypeHint must not be null");
-	
+
 		try {
 			return returns(TypeInformation.of(typeHint));
 		}
@@ -259,7 +245,7 @@ public O returns(TypeHint typeHint) {
 	 * can be used in cases where Flink cannot determine automatically what the produced
 	 * type of a function is. That can be the case if the function uses generic type variables
 	 * in the return type that cannot be inferred from the input type.
-	 * 
+	 *
 	 * 

In most cases, the methods {@link #returns(Class)} and {@link #returns(TypeHint)} * are preferable. * @@ -268,22 +254,22 @@ public O returns(TypeHint typeHint) { */ public O returns(TypeInformation typeInfo) { requireNonNull(typeInfo, "TypeInformation must not be null"); - + fillInType(typeInfo); @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } - + /** - * Adds a type information hint about the return type of this operator. - * - *

- * Type hints are important in cases where the Java compiler + * Adds a type information hint about the return type of this operator. + * + * + *

Type hints are important in cases where the Java compiler * throws away generic type information necessary for efficient execution. - * - *

- * This method takes a type information string that will be parsed. A type information string can contain the following + * + * + *

This method takes a type information string that will be parsed. A type information string can contain the following * types: * *

    @@ -303,13 +289,13 @@ public O returns(TypeInformation typeInfo) { *
  • Enum types such as Enum<org.my.CustomEnum>
  • *
* - * Example: + *

Example: * "Tuple2<String,Tuple2<Integer,org.my.MyJob$Pojo<word=String>>>" * * @param typeInfoString * type information string to be parsed * @return This operator with a given return type hint. - * + * * @deprecated Please use {@link #returns(Class)} or {@link #returns(TypeHint)} instead. */ @Deprecated @@ -320,7 +306,7 @@ public O returns(String typeInfoString) { } return returns(TypeInfoParser.parse(typeInfoString)); } - + // -------------------------------------------------------------------------------------------- // Accessors // -------------------------------------------------------------------------------------------- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java index 7f30a30d8d533..862ad8912b033 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java @@ -58,7 +58,6 @@ private SortPartitionOperator(DataSet dataSet, String sortLocationName) { this.sortLocationName = sortLocationName; } - public SortPartitionOperator(DataSet dataSet, int sortField, Order sortOrder, String sortLocationName) { this(dataSet, sortLocationName); this.useKeySelector = false; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 11645dfbeed6a..2e3709da51587 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -19,40 +19,40 @@ package org.apache.flink.api.java.operators; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.FirstReducer; - -import java.util.Arrays; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; +import java.util.Arrays; + /** - * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.
- * The following transformation can be applied on sorted groups: + * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet. + * + *

The following transformation can be applied on sorted groups: *

    *
  • {@link SortedGrouping#reduceGroup(org.apache.flink.api.common.functions.GroupReduceFunction)},
  • *
- * + * * @param The type of the elements of the sorted and grouped DataSet. */ @Public public class SortedGrouping extends Grouping { - + private int[] groupSortKeyPositions; private Order[] groupSortOrders; private Keys.SelectorFunctionKeys groupSortSelectorFunctionKey = null; - + /* * int sorting keys for tuples */ @@ -70,7 +70,7 @@ public SortedGrouping(DataSet set, Keys keys, int field, Order order) { this.groupSortOrders = new Order[groupSortKeyPositions.length]; Arrays.fill(this.groupSortOrders, order); } - + /* * String sorting for Pojos and tuples */ @@ -99,8 +99,8 @@ public SortedGrouping(DataSet set, Keys keys, Keys.SelectorFunctionKey throw new InvalidProgramException("Sorting on KeySelector keys only works with KeySelector grouping."); } TypeInformation sortKeyType = keySelector.getKeyType(); - if(!sortKeyType.isSortKeyType()) { - throw new InvalidProgramException("Key type " + sortKeyType +" is not sortable."); + if (!sortKeyType.isSortKeyType()) { + throw new InvalidProgramException("Key type " + sortKeyType + " is not sortable."); } this.groupSortKeyPositions = keySelector.computeLogicalKeyPositions(); @@ -112,13 +112,13 @@ public SortedGrouping(DataSet set, Keys keys, Keys.SelectorFunctionKey this.groupSortOrders = new Order[groupSortKeyPositions.length]; Arrays.fill(this.groupSortOrders, order); } - + // -------------------------------------------------------------------------------------------- - + protected int[] getGroupSortKeyPositions() { return this.groupSortKeyPositions; } - + protected Order[] getGroupSortOrders() { return this.groupSortOrders; } @@ -126,24 +126,24 @@ protected Order[] getGroupSortOrders() { protected Ordering getGroupOrdering() { Ordering o = new Ordering(); - for(int i=0; i < this.groupSortKeyPositions.length; i++) { + for (int i = 0; i < this.groupSortKeyPositions.length; i++) { o.appendOrdering(this.groupSortKeyPositions[i], null, this.groupSortOrders[i]); } return o; } - + /** * Uses a custom partitioner for the grouping. - * + * * @param partitioner The custom partitioner. * @return The grouping object itself, to allow for method chaining. */ public SortedGrouping withPartitioner(Partitioner partitioner) { Preconditions.checkNotNull(partitioner); - + getKeys().validateCustomPartitioner(partitioner, null); - + this.customPartitioner = partitioner; return this; } @@ -153,14 +153,15 @@ public SortedGrouping withPartitioner(Partitioner partitioner) { } /** - * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.
- * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet. + * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}. + * + *

The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet. * A GroupReduceFunction can iterate over all elements of a group and emit any * number of output elements including none. - * + * * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. * @return A GroupReduceOperator that represents the reduced DataSet. - * + * * @see org.apache.flink.api.common.functions.RichGroupReduceFunction * @see GroupReduceOperator * @see DataSet @@ -196,33 +197,34 @@ public GroupCombineOperator combineGroup(GroupCombineFunction co return new GroupCombineOperator<>(this, resultType, inputDataSet.clean(combiner), Utils.getCallLocationName()); } - /** - * Returns a new set containing the first n elements in this grouped and sorted {@link DataSet}.
+ * Returns a new set containing the first n elements in this grouped and sorted {@link DataSet}. * @param n The desired number of elements for each group. * @return A GroupReduceOperator that represents the DataSet containing the elements. */ public GroupReduceOperator first(int n) { - if(n < 1) { + if (n < 1) { throw new InvalidProgramException("Parameter n of first(n) must be at least 1."); } - + return reduceGroup(new FirstReducer(n)); } - + // -------------------------------------------------------------------------------------------- // Group Operations // -------------------------------------------------------------------------------------------- - + /** - * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.
- * Note: Only groups of Tuple or Pojo elements can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. - * + * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple or Pojo elements can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. + * * @param field The Tuple field on which the group is sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see org.apache.flink.api.java.tuple.Tuple * @see Order */ @@ -241,9 +243,11 @@ public SortedGrouping sortGroup(int field, Order order) { } /** - * Sorts {@link org.apache.flink.api.java.tuple.Tuple} or POJO elements within a group on the specified field in the specified {@link Order}.
- * Note: Only groups of Tuple or Pojo elements can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls. + * Sorts {@link org.apache.flink.api.java.tuple.Tuple} or POJO elements within a group on the specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple or Pojo elements can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls. * * @param field The Tuple or Pojo field on which the group is sorted. * @param order The Order in which the specified field is sorted. @@ -265,17 +269,17 @@ public SortedGrouping sortGroup(String field, Order order) { addSortGroupInternal(ek, order); return this; } - + private void addSortGroupInternal(ExpressionKeys ek, Order order) { Preconditions.checkArgument(order != null, "Order can not be null"); int[] additionalKeyPositions = ek.computeLogicalKeyPositions(); - + int newLength = this.groupSortKeyPositions.length + additionalKeyPositions.length; this.groupSortKeyPositions = Arrays.copyOf(this.groupSortKeyPositions, newLength); this.groupSortOrders = Arrays.copyOf(this.groupSortOrders, newLength); int pos = newLength - additionalKeyPositions.length; int off = newLength - additionalKeyPositions.length; - for(;pos < newLength; pos++) { + for (; pos < newLength; pos++) { this.groupSortKeyPositions[pos] = additionalKeyPositions[pos - off]; this.groupSortOrders[pos] = order; // use the same order } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputOperator.java index 28dec3261efeb..b91584949f2ee 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputOperator.java @@ -25,18 +25,17 @@ /** * Base class for operations that operates on two input data sets. - * + * * @param The data type of the first input data set. * @param The data type of the second input data set. * @param The data type of the returned data set. */ @Public public abstract class TwoInputOperator> extends Operator { - + private final DataSet input1; private final DataSet input2; - - + protected TwoInputOperator(DataSet input1, DataSet input2, TypeInformation resultType) { super(Preconditions.checkNotNull(input1, "input1 is null").getExecutionEnvironment(), resultType); Preconditions.checkNotNull(input2, "input2 is null"); @@ -44,48 +43,48 @@ protected TwoInputOperator(DataSet input1, DataSet input2, TypeInforma this.input1 = input1; this.input2 = input2; } - + /** * Gets the data set that this operation uses as its first input. - * + * * @return The data set that this operation uses as its first input. */ public DataSet getInput1() { return this.input1; } - + /** * Gets the data set that this operation uses as its second input. - * + * * @return The data set that this operation uses as its second input. */ public DataSet getInput2() { return this.input2; } - + /** * Gets the type information of the data type of the first input data set. * This method returns equivalent information as {@code getInput1().getType()}. - * + * * @return The first input data type. */ public TypeInformation getInput1Type() { return this.input1.getType(); } - + /** * Gets the type information of the data type of the second input data set. * This method returns equivalent information as {@code getInput2().getType()}. - * + * * @return The second input data type. */ public TypeInformation getInput2Type() { return this.input2.getType(); } - + /** * Translates this java API operator into a common API operator with two inputs. - * + * * @param input1 The first input of the operation, as a common API operator. * @param input2 The second input of the operation, as a common API operator. * @return The created common API operator. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java index 695ed3a4958cd..b78d17ea4ca20 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java @@ -18,12 +18,6 @@ package org.apache.flink.api.java.operators; -import java.lang.annotation.Annotation; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -39,6 +33,12 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; +import java.lang.annotation.Annotation; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + import static java.util.Objects.requireNonNull; /** @@ -46,8 +46,8 @@ * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that * have two inputs (such as {@link org.apache.flink.api.common.functions.RichJoinFunction} or * {@link org.apache.flink.api.common.functions.RichCoGroupFunction}). - *

- * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization + * + *

This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. * * @param The data type of the first input data set. @@ -56,8 +56,7 @@ */ @Public public abstract class TwoInputUdfOperator> - extends TwoInputOperator implements UdfOperator -{ + extends TwoInputOperator implements UdfOperator { private Configuration parameters; private Map> broadcastVariables; @@ -104,7 +103,7 @@ public O withBroadcastSet(DataSet data, String name) { if (name == null) { throw new IllegalArgumentException("Broadcast variable name must not be null."); } - + if (this.broadcastVariables == null) { this.broadcastVariables = new HashMap>(); } @@ -117,46 +116,34 @@ public O withBroadcastSet(DataSet data, String name) { } /** - *

* Adds semantic information about forwarded fields of the first input of the user-defined function. * The forwarded fields information declares fields which are never modified by the function and * which are forwarded at the same position to the output or unchanged copied to another position in the output. - *

* - *

- * Fields that are forwarded at the same position are specified by their position. + *

Fields that are forwarded at the same position are specified by their position. * The specified position must be valid for the input and output data type and have the same type. * For example withForwardedFieldsFirst("f2") declares that the third field of a Java input tuple * from the first input is copied to the third field of an output tuple. - *

* - *

- * Fields which are unchanged copied from the first input to another position in the output are declared + *

Fields which are unchanged copied from the first input to another position in the output are declared * by specifying the source field reference in the first input and the target field reference in the output. * {@code withForwardedFieldsFirst("f0->f2")} denotes that the first field of the first input Java tuple is * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that * the number of declared fields and their types in first input and output type match. - *

* - *

- * Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsFirst("f2; f3->f0; f4")}) + *

Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsFirst("f2; f3->f0; f4")}) * or separate Strings ({@code withForwardedFieldsFirst("f2", "f3->f0", "f4")}). * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for * details on field references such as nested fields and wildcard. - *

* - *

- * It is not possible to override existing semantic information about forwarded fields of the first input which was + *

It is not possible to override existing semantic information about forwarded fields of the first input which was * for example added by a {@link org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst} class annotation. - *

* - *

- * NOTE: Adding semantic information for functions is optional! + *

NOTE: Adding semantic information for functions is optional! * If used correctly, semantic information can help the Flink optimizer to generate more efficient execution plans. * However, incorrect semantic information can cause the optimizer to generate incorrect execution plans which compute wrong results! * So be careful when adding semantic information. * - *

* * @param forwardedFieldsFirst A list of forwarded field expressions for the first input of the function. * @return This operator with annotated forwarded field information. @@ -171,12 +158,12 @@ public O withForwardedFieldsFirst(String... forwardedFieldsFirst) { setSemanticProperties(extractSemanticAnnotationsFromUdf(getFunction().getClass())); } - if(this.udfSemantics == null || this.analyzedUdfSemantics) { + if (this.udfSemantics == null || this.analyzedUdfSemantics) { setSemanticProperties(new DualInputSemanticProperties()); SemanticPropUtil.getSemanticPropsDualFromString(this.udfSemantics, forwardedFieldsFirst, null, null, null, null, null, getInput1Type(), getInput2Type(), getResultType()); } else { - if(this.udfWithForwardedFieldsFirstAnnotation(getFunction().getClass())) { + if (this.udfWithForwardedFieldsFirstAnnotation(getFunction().getClass())) { // refuse semantic information as it would override the function annotation throw new SemanticProperties.InvalidSemanticAnnotationException("Forwarded field information " + "has already been added by a function annotation for the first input of this operator. " + @@ -192,46 +179,34 @@ public O withForwardedFieldsFirst(String... forwardedFieldsFirst) { } /** - *

* Adds semantic information about forwarded fields of the second input of the user-defined function. * The forwarded fields information declares fields which are never modified by the function and * which are forwarded at the same position to the output or unchanged copied to another position in the output. - *

* - *

- * Fields that are forwarded at the same position are specified by their position. + *

Fields that are forwarded at the same position are specified by their position. * The specified position must be valid for the input and output data type and have the same type. * For example withForwardedFieldsSecond("f2") declares that the third field of a Java input tuple * from the second input is copied to the third field of an output tuple. - *

* - *

- * Fields which are unchanged copied from the second input to another position in the output are declared + *

Fields which are unchanged copied from the second input to another position in the output are declared * by specifying the source field reference in the second input and the target field reference in the output. * {@code withForwardedFieldsSecond("f0->f2")} denotes that the first field of the second input Java tuple is * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that * the number of declared fields and their types in second input and output type match. - *

* - *

- * Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsSecond("f2; f3->f0; f4")}) + *

Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsSecond("f2; f3->f0; f4")}) * or separate Strings ({@code withForwardedFieldsSecond("f2", "f3->f0", "f4")}). * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for * details on field references such as nested fields and wildcard. - *

* - *

- * It is not possible to override existing semantic information about forwarded fields of the second input which was + *

It is not possible to override existing semantic information about forwarded fields of the second input which was * for example added by a {@link org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond} class annotation. - *

* - *

- * NOTE: Adding semantic information for functions is optional! + *

NOTE: Adding semantic information for functions is optional! * If used correctly, semantic information can help the Flink optimizer to generate more efficient execution plans. * However, incorrect semantic information can cause the optimizer to generate incorrect execution plans which compute wrong results! * So be careful when adding semantic information. * - *

* * @param forwardedFieldsSecond A list of forwarded field expressions for the second input of the function. * @return This operator with annotated forwarded field information. @@ -246,12 +221,12 @@ public O withForwardedFieldsSecond(String... forwardedFieldsSecond) { setSemanticProperties(extractSemanticAnnotationsFromUdf(getFunction().getClass())); } - if(this.udfSemantics == null || this.analyzedUdfSemantics) { + if (this.udfSemantics == null || this.analyzedUdfSemantics) { setSemanticProperties(new DualInputSemanticProperties()); SemanticPropUtil.getSemanticPropsDualFromString(this.udfSemantics, null, forwardedFieldsSecond, null, null, null, null, getInput1Type(), getInput2Type(), getResultType()); } else { - if(udfWithForwardedFieldsSecondAnnotation(getFunction().getClass())) { + if (udfWithForwardedFieldsSecondAnnotation(getFunction().getClass())) { // refuse semantic information as it would override the function annotation throw new SemanticProperties.InvalidSemanticAnnotationException("Forwarded field information " + "has already been added by a function annotation for the second input of this operator. " + @@ -282,7 +257,7 @@ public O withForwardedFieldsSecond(String... forwardedFieldsSecond) { * *

Use this method the following way: *

{@code
-	 *     DataSet result = 
+	 *     DataSet result =
 	 *         data1.join(data2).where("id").equalTo("fieldX")
 	 *              .with(new JoinFunctionWithNonInferrableReturnType())
 	 *              .returns(String[].class);
@@ -312,7 +287,7 @@ public O returns(Class typeClass) {
 	 *
 	 * 

Use this method the following way: *

{@code
-	 *     DataSet> result = 
+	 *     DataSet> result =
 	 *         data1.join(data2).where("id").equalTo("fieldX")
 	 *              .with(new JoinFunctionWithNonInferrableReturnType())
 	 *              .returns(new TypeHint>(){});
@@ -355,14 +330,14 @@ public O returns(TypeInformation typeInfo) {
 	}
 
 	/**
-	 * Adds a type information hint about the return type of this operator. 
+	 * Adds a type information hint about the return type of this operator.
 	 *
-	 * 

- * Type hints are important in cases where the Java compiler + * + *

Type hints are important in cases where the Java compiler * throws away generic type information necessary for efficient execution. * - *

- * This method takes a type information string that will be parsed. A type information string can contain the following + * + *

This method takes a type information string that will be parsed. A type information string can contain the following * types: * *

    @@ -382,7 +357,7 @@ public O returns(TypeInformation typeInfo) { *
  • Enum types such as Enum<org.my.CustomEnum>
  • *
* - * Example: + *

Example: * "Tuple2<String,Tuple2<Integer,org.my.MyJob$Pojo<word=String>>>" * * @param typeInfoString diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java index e4f78886131b2..249a5cb8fc3cc 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java @@ -18,14 +18,13 @@ package org.apache.flink.api.java.operators; -import java.util.Map; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.operators.SemanticProperties; +import org.apache.flink.api.java.DataSet; import org.apache.flink.configuration.Configuration; -import org.apache.flink.api.java.DataSet; +import java.util.Map; /** * This interface marks operators as operators that execute user-defined functions (UDFs), such as @@ -35,60 +34,60 @@ */ @Public public interface UdfOperator> { - + // -------------------------------------------------------------------------------------------- // Accessors // -------------------------------------------------------------------------------------------- - + /** * Gets the configuration parameters that will be passed to the UDF's open method * {@link org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)}. * The configuration is set via the {@link #withParameters(Configuration)} * method. - * + * * @return The configuration parameters for the UDF. */ Configuration getParameters(); - + /** * Gets the broadcast sets (name and data set) that have been added to context of the UDF. * Broadcast sets are added to a UDF via the method {@link #withBroadcastSet(DataSet, String)}. - * + * * @return The broadcast data sets that have been added to this UDF. */ @Internal Map> getBroadcastSets(); - + /** * Gets the semantic properties that have been set for the user-defined functions (UDF). - * + * * @return The semantic properties of the UDF. */ @Internal SemanticProperties getSemanticProperties(); - + // -------------------------------------------------------------------------------------------- // Fluent API methods // -------------------------------------------------------------------------------------------- - + /** * Sets the configuration parameters for the UDF. These are optional parameters that are passed * to the UDF in the {@link org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)} method. - * + * * @param parameters The configuration parameters for the UDF. * @return The operator itself, to allow chaining function calls. */ O withParameters(Configuration parameters); - + /** * Adds a certain data set as a broadcast set to this operator. Broadcasted data sets are available at all * parallel instances of this operator. A broadcast data set is registered under a certain name, and can be * retrieved under that name from the operators runtime context via * {@link org.apache.flink.api.common.functions.RuntimeContext#getBroadcastVariable(String)}. - * - * The runtime context itself is available in all UDFs via + * + *

The runtime context itself is available in all UDFs via * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()}. - * + * * @param data The data set to be broadcasted. * @param name The name under which the broadcast data set retrieved. * @return The operator itself, to allow chaining function calls. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java index 660f84509bd7f..f8e04a9e1cbd2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.functions.FunctionAnnotation; import org.apache.flink.api.java.sca.CodeAnalyzerException; import org.apache.flink.api.java.sca.UdfAnalyzer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +46,7 @@ public static void analyzeSingleInputUdf(SingleInputUdfOperator operato if (mode != CodeAnalysisMode.DISABLE && !udf.getClass().isAnnotationPresent(FunctionAnnotation.SkipCodeAnalysis.class)) { final String operatorName = operator.getName() != null ? operator.getName() - : udfBaseClass.getSimpleName() + " at "+defaultName; + : udfBaseClass.getSimpleName() + " at " + defaultName; try { final UdfAnalyzer analyzer = new UdfAnalyzer(udfBaseClass, udf.getClass(), operatorName, operator.getInputType(), null, operator.getResultType(), key, null, mode == CodeAnalysisMode.OPTIMIZE); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnionOperator.java index 70ad374cd4d56..0da5e01a0b380 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnionOperator.java @@ -25,35 +25,35 @@ import org.apache.flink.api.java.DataSet; /** - * Java API operator for union of two data sets - * - * @param The type of the two input data sets and the result data set + * Java API operator for union of two data sets. + * + * @param The type of the two input data sets and the result data set */ @Public public class UnionOperator extends TwoInputOperator> { private final String unionLocationName; - + /** * Create an operator that produces the union of the two given data sets. - * + * * @param input1 The first data set to be unioned. * @param input2 The second data set to be unioned. */ public UnionOperator(DataSet input1, DataSet input2, String unionLocationName) { super(input1, input2, input1.getType()); - + if (!input1.getType().equals(input2.getType())) { - throw new InvalidProgramException("Cannot union inputs of different types. Input1=" + throw new InvalidProgramException("Cannot union inputs of different types. Input1=" + input1.getType() + ", input2=" + input2.getType()); } - + this.unionLocationName = unionLocationName; } - + /** * Returns the BinaryNodeTranslation of the Union. - * + * * @param input1 The first input of the union, as a common API operator. * @param input2 The second input of the union, as a common API operator. * @return The common API union operator. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 2ad133c7b7646..ccd613907fc63 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -38,23 +38,27 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; +/** + * A {@link Grouping} that is unsorted. + * @param type of elements + */ @Public public class UnsortedGrouping extends Grouping { public UnsortedGrouping(DataSet set, Keys keys) { super(set, keys); } - + /** * Uses a custom partitioner for the grouping. - * + * * @param partitioner The custom partitioner. * @return The grouping object itself, to allow for method chaining. */ public UnsortedGrouping withPartitioner(Partitioner partitioner) { Preconditions.checkNotNull(partitioner); getKeys().validateCustomPartitioner(partitioner, null); - + this.customPartitioner = partitioner; return this; } @@ -62,18 +66,19 @@ public UnsortedGrouping withPartitioner(Partitioner partitioner) { // -------------------------------------------------------------------------------------------- // Operations / Transformations // -------------------------------------------------------------------------------------------- - + /** - * Applies an Aggregate transformation on a grouped {@link org.apache.flink.api.java.tuple.Tuple} {@link DataSet}.
- * Note: Only Tuple DataSets can be aggregated. - * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field - * of a Tuple group. Additional aggregation functions can be added to the resulting + * Applies an Aggregate transformation on a grouped {@link org.apache.flink.api.java.tuple.Tuple} {@link DataSet}. + * + *

Note: Only Tuple DataSets can be aggregated. + * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field + * of a Tuple group. Additional aggregation functions can be added to the resulting * {@link AggregateOperator} by calling {@link AggregateOperator#and(Aggregations, int)}. - * + * * @param agg The built-in aggregation function that is computed. * @param field The index of the Tuple field on which the aggregation function is applied. - * @return An AggregateOperator that represents the aggregated DataSet. - * + * @return An AggregateOperator that represents the aggregated DataSet. + * * @see org.apache.flink.api.java.tuple.Tuple * @see Aggregations * @see AggregateOperator @@ -82,14 +87,14 @@ public UnsortedGrouping withPartitioner(Partitioner partitioner) { public AggregateOperator aggregate(Aggregations agg, int field) { return aggregate(agg, field, Utils.getCallLocationName()); } - + // private helper that allows to set a different call location name private AggregateOperator aggregate(Aggregations agg, int field, String callLocationName) { return new AggregateOperator(this, agg, field, callLocationName); } /** - * Syntactic sugar for aggregate (SUM, field) + * Syntactic sugar for aggregate (SUM, field). * @param field The index of the Tuple field on which the aggregation function is applied. * @return An AggregateOperator that represents the summed DataSet. * @@ -100,7 +105,7 @@ public AggregateOperator sum (int field) { } /** - * Syntactic sugar for aggregate (MAX, field) + * Syntactic sugar for aggregate (MAX, field). * @param field The index of the Tuple field on which the aggregation function is applied. * @return An AggregateOperator that represents the max'ed DataSet. * @@ -111,7 +116,7 @@ public AggregateOperator max (int field) { } /** - * Syntactic sugar for aggregate (MIN, field) + * Syntactic sugar for aggregate (MIN, field). * @param field The index of the Tuple field on which the aggregation function is applied. * @return An AggregateOperator that represents the min'ed DataSet. * @@ -120,16 +125,17 @@ public AggregateOperator max (int field) { public AggregateOperator min (int field) { return this.aggregate (Aggregations.MIN, field, Utils.getCallLocationName()); } - + /** - * Applies a Reduce transformation on a grouped {@link DataSet}.
- * For each group, the transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction} - * until only a single element for each group remains. + * Applies a Reduce transformation on a grouped {@link DataSet}. + * + *

For each group, the transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction} + * until only a single element for each group remains. * A ReduceFunction combines two elements into one new element of the same type. - * + * * @param reducer The ReduceFunction that is applied on each group of the DataSet. * @return A ReduceOperator that represents the reduced DataSet. - * + * * @see org.apache.flink.api.common.functions.RichReduceFunction * @see ReduceOperator * @see DataSet @@ -140,16 +146,17 @@ public ReduceOperator reduce(ReduceFunction reducer) { } return new ReduceOperator(this, inputDataSet.clean(reducer), Utils.getCallLocationName()); } - + /** - * Applies a GroupReduce transformation on a grouped {@link DataSet}.
- * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet. + * Applies a GroupReduce transformation on a grouped {@link DataSet}. + * + *

The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet. * A GroupReduceFunction can iterate over all elements of a group and emit any * number of output elements including none. - * + * * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. * @return A GroupReduceOperator that represents the reduced DataSet. - * + * * @see org.apache.flink.api.common.functions.RichGroupReduceFunction * @see GroupReduceOperator * @see DataSet @@ -187,72 +194,77 @@ public GroupCombineOperator combineGroup(GroupCombineFunction co } /** - * Returns a new set containing the first n elements in this grouped {@link DataSet}.
+ * Returns a new set containing the first n elements in this grouped {@link DataSet}. + * * @param n The desired number of elements for each group. * @return A GroupReduceOperator that represents the DataSet containing the elements. */ public GroupReduceOperator first(int n) { - if(n < 1) { + if (n < 1) { throw new InvalidProgramException("Parameter n of first(n) must be at least 1."); } - + return reduceGroup(new FirstReducer(n)); } /** - * Applies a special case of a reduce transformation (minBy) on a grouped {@link DataSet}.
- * The transformation consecutively calls a {@link ReduceFunction} + * Applies a special case of a reduce transformation (minBy) on a grouped {@link DataSet}. + * + *

The transformation consecutively calls a {@link ReduceFunction} * until only a single element remains which is the result of the transformation. * A ReduceFunction combines two elements into one new element of the same type. - * + * * @param fields Keys taken into account for finding the minimum. * @return A {@link ReduceOperator} representing the minimum. */ @SuppressWarnings({ "unchecked", "rawtypes" }) public ReduceOperator minBy(int... fields) { - + // Check for using a tuple - if(!this.inputDataSet.getType().isTupleType()) { + if (!this.inputDataSet.getType().isTupleType()) { throw new InvalidProgramException("Method minBy(int) only works on tuples."); } - + return new ReduceOperator(this, new SelectByMinFunction( (TupleTypeInfo) this.inputDataSet.getType(), fields), Utils.getCallLocationName()); } - + /** - * Applies a special case of a reduce transformation (maxBy) on a grouped {@link DataSet}.
- * The transformation consecutively calls a {@link ReduceFunction} + * Applies a special case of a reduce transformation (maxBy) on a grouped {@link DataSet}. + * + *

The transformation consecutively calls a {@link ReduceFunction} * until only a single element remains which is the result of the transformation. * A ReduceFunction combines two elements into one new element of the same type. - * + * * @param fields Keys taken into account for finding the minimum. * @return A {@link ReduceOperator} representing the minimum. */ @SuppressWarnings({ "unchecked", "rawtypes" }) public ReduceOperator maxBy(int... fields) { - + // Check for using a tuple - if(!this.inputDataSet.getType().isTupleType()) { + if (!this.inputDataSet.getType().isTupleType()) { throw new InvalidProgramException("Method maxBy(int) only works on tuples."); } - + return new ReduceOperator(this, new SelectByMaxFunction( (TupleTypeInfo) this.inputDataSet.getType(), fields), Utils.getCallLocationName()); } // -------------------------------------------------------------------------------------------- // Group Operations // -------------------------------------------------------------------------------------------- - + /** - * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.
- * Note: Only groups of Tuple elements and Pojos can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. - * + * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple elements and Pojos can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. + * * @param field The Tuple field on which the group is sorted. * @param order The Order in which the specified Tuple field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see org.apache.flink.api.java.tuple.Tuple * @see Order */ @@ -265,16 +277,18 @@ public SortedGrouping sortGroup(int field, Order order) { sg.customPartitioner = getCustomPartitioner(); return sg; } - + /** - * Sorts Pojos within a group on the specified field in the specified {@link Order}.
- * Note: Only groups of Tuple elements and Pojos can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls. - * + * Sorts Pojos within a group on the specified field in the specified {@link Order}. + * + *

Note: Only groups of Tuple elements and Pojos can be sorted. + * + *

Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls. + * * @param field The Tuple or Pojo field on which the group is sorted. * @param order The Order in which the specified field is sorted. * @return A SortedGrouping with specified order of group element. - * + * * @see Order */ public SortedGrouping sortGroup(String field, Order order) { @@ -289,8 +303,9 @@ public SortedGrouping sortGroup(String field, Order order) { /** * Sorts elements within a group on a key extracted by the specified {@link org.apache.flink.api.java.functions.KeySelector} - * in the specified {@link Order}.
- * Chaining {@link #sortGroup(KeySelector, Order)} calls is not supported. + * in the specified {@link Order}. + * + *

Chaining {@link #sortGroup(KeySelector, Order)} calls is not supported. * * @param keySelector The KeySelector with which the group is sorted. * @param order The Order in which the extracted key is sorted. @@ -308,5 +323,5 @@ public SortedGrouping sortGroup(KeySelector keySelector, Order orde sg.customPartitioner = getCustomPartitioner(); return sg; } - + } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java index a4747831b4610..770e0e89feea3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; @@ -29,13 +30,13 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin; import org.apache.flink.api.java.operators.JoinOperator.EquiJoin; -import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TypeExtractor; /** - * Intermediate step of an Outer Join transformation.
- * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling + * Intermediate step of an Outer Join transformation. + * + *

To continue the Join transformation, select the join key of the first input {@link DataSet} by calling * {@link JoinOperatorSetsBase#where(int...)} or * {@link JoinOperatorSetsBase#where(KeySelector)}. * @@ -71,9 +72,11 @@ public JoinOperatorSetsBase(DataSet input1, DataSet input2, JoinHint hin } /** - * Continues a Join transformation.
- * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.
- * Note: Fields can only be selected as join keys on Tuple DataSets.
+ * Continues a Join transformation. + * + *

Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys. + * + *

Note: Fields can only be selected as join keys on Tuple DataSets. * * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys. * @return An incomplete Join transformation. @@ -89,8 +92,9 @@ public JoinOperatorSetsPredicateBase where(int... fields) { } /** - * Continues a Join transformation.
- * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields + * Continues a Join transformation. + * + *

Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields * are the names of member fields of the underlying type of the data set. * * @param fields The fields of the first join DataSets that should be used as keys. @@ -107,9 +111,10 @@ public JoinOperatorSetsPredicateBase where(String... fields) { } /** - * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.
- * The KeySelector function is called for each element of the first DataSet and extracts a single - * key value on which the DataSet is joined.
+ * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}. + * + *

The KeySelector function is called for each element of the first DataSet and extracts a single + * key value on which the DataSet is joined. * * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined. * @return An incomplete Join transformation. @@ -125,10 +130,10 @@ public JoinOperatorSetsPredicateBase where(KeySelector keySelector) { return new JoinOperatorSetsPredicateBase(new Keys.SelectorFunctionKeys<>(keySelector, input1.getType(), keyType)); } - /** - * Intermediate step of a Join transformation.
- * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling + * Intermediate step of a Join transformation. + * + *

To continue the Join transformation, select the join key of the second input {@link DataSet} by calling * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}. * @@ -151,10 +156,11 @@ protected JoinOperatorSetsPredicateBase(Keys keys1) { /** * Continues a Join transformation and defines the {@link Tuple} fields of the second join - * {@link DataSet} that should be used as join keys.
- * Note: Fields can only be selected as join keys on Tuple DataSets.
+ * {@link DataSet} that should be used as join keys. * - * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a + *

Note: Fields can only be selected as join keys on Tuple DataSets. + * + *

The resulting {@link JoinFunctionAssigner} needs to be finished by providing a * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)} * * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys. @@ -166,9 +172,9 @@ public JoinFunctionAssigner equalTo(int... fields) { /** * Continues a Join transformation and defines the fields of the second join - * {@link DataSet} that should be used as join keys.
+ * {@link DataSet} that should be used as join keys. * - * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a + *

The resulting {@link JoinFunctionAssigner} needs to be finished by providing a * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)} * * @param fields The fields of the second join DataSet that should be used as keys. @@ -179,11 +185,12 @@ public JoinFunctionAssigner equalTo(String... fields) { } /** - * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.
- * The KeySelector function is called for each element of the second DataSet and extracts a single - * key value on which the DataSet is joined.
+ * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}. + * + *

The KeySelector function is called for each element of the second DataSet and extracts a single + * key value on which the DataSet is joined. * - * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a + *

The resulting {@link JoinFunctionAssigner} needs to be finished by providing a * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)} * * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined. @@ -211,7 +218,7 @@ protected DefaultJoin createDefaultJoin(Keys keys2) { try { keys1.areCompatible(keys2); } catch (Keys.IncompatibleKeysException e) { - throw new InvalidProgramException("The pair of join keys are not compatible with each other.",e); + throw new InvalidProgramException("The pair of join keys are not compatible with each other.", e); } return new DefaultJoin<>(input1, input2, keys1, keys2, joinHint, Utils.getCallLocationName(4), joinType); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java index 58866de812080..2a668a89b5b3d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinType.java @@ -20,6 +20,9 @@ import org.apache.flink.annotation.Public; +/** + * Join types. + */ @Public public enum JoinType { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java index 09a705c38d16f..5597b8f14ffe0 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java @@ -18,11 +18,6 @@ package org.apache.flink.api.java.operators; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -35,9 +30,15 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.util.Collector; import org.apache.flink.util.Visitor; + import org.junit.Assert; import org.junit.Test; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + /** * Test proper automated assignment of the transformation's name, if not set by the user. */ @@ -50,7 +51,6 @@ public void testDefaultName() { DataSet strs = env.fromCollection(Arrays.asList("a", "b")); - // WARNING: The test will fail if this line is being moved down in the file (the line-number is hard-coded) strs.filter(new FilterFunction() { private static final long serialVersionUID = 1L; @@ -92,8 +92,7 @@ public void testJoinWith() { DataSet> strs1 = env.fromCollection(strLi); strs.join(strs1).where(0).equalTo(0).with(new FlatJoinFunction, Tuple1, String>() { @Override - public void join(Tuple1 first, Tuple1 second, - Collector out) throws Exception { + public void join(Tuple1 first, Tuple1 second, Collector out) throws Exception { // } }) @@ -102,11 +101,12 @@ public void join(Tuple1 first, Tuple1 second, plan.accept(new Visitor>() { @Override public boolean preVisit(Operator visitable) { - if(visitable instanceof InnerJoinOperatorBase) { + if (visitable instanceof InnerJoinOperatorBase) { Assert.assertEquals("Join at testJoinWith(NamesTest.java:93)", visitable.getName()); } return true; } + @Override public void postVisit(Operator visitable) {} }); @@ -116,7 +116,7 @@ private static void testForName(final String expected, Plan plan) { plan.accept(new Visitor>() { @Override public boolean preVisit(Operator visitable) { - if(visitable instanceof PlanFilterOperator) { + if (visitable instanceof PlanFilterOperator) { // cast is actually not required. Its just a check for the right element PlanFilterOperator filterOp = (PlanFilterOperator) visitable; Assert.assertEquals(expected, filterOp.getName()); diff --git a/tools/maven/suppressions-java.xml b/tools/maven/suppressions-java.xml index 3b7d60ba697bb..9ddacbac611ea 100644 --- a/tools/maven/suppressions-java.xml +++ b/tools/maven/suppressions-java.xml @@ -59,10 +59,6 @@ under the License. files="(.*)test[/\\](.*)api[/\\]java[/\\]operators[/\\]translation[/\\](.*)" checks="AvoidStarImport"/> - -