From d4f2a65fc4faa013181ceabcc8e3c0975c99156e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 29 Jul 2014 15:58:44 +0200 Subject: [PATCH 1/3] [FLINK-1023] Switch group-at-a-time function to java.lang.Iterable (from java.util.Iterator) --- .../example/ReflectiveAvroTypeExample.java | 9 +- .../api/avro/AvroWithEmptyArrayITCase.java | 11 +- .../mapred/record/example/WordCount.java | 11 +- .../example/WordCountWithOutputFormat.java | 11 +- .../spargel/java/VertexCentricIteration.java | 42 ++++--- .../spargel/java/record/SpargelIteration.java | 24 ++-- .../compiler/CoGroupSolutionSetFirstTest.java | 6 +- .../compiler/GroupReduceCompilationTest.java | 15 +-- .../compiler/IterationsCompilerTest.java | 5 +- .../WorksetIterationsJavaApiCompilerTest.java | 5 +- .../testfunctions/IdentityGroupReducer.java | 9 +- .../testfunctions/Top1GroupReducer.java | 7 +- .../flink/compiler/util/DummyCoGroupStub.java | 12 +- .../flink/compiler/util/IdentityReduce.java | 8 +- .../common/functions/GenericCoGrouper.java | 4 +- .../api/common/functions/GenericCombine.java | 9 +- .../common/functions/GenericGroupReduce.java | 11 +- .../java/graph/EnumTrianglesBasic.java | 13 +- .../example/java/graph/EnumTrianglesOpt.java | 19 ++- .../example/java/graph/PageRankBasic.java | 6 +- .../java/graph/TransitiveClosureNaive.java | 6 +- .../java/relational/WebLogAnalysis.java | 11 +- .../api/java/functions/CoGroupFunction.java | 4 +- .../java/functions/GroupReduceFunction.java | 11 +- .../java/functions/GroupReduceIterator.java | 4 +- .../api/java/operators/AggregateOperator.java | 3 +- .../api/java/operators/CoGroupOperator.java | 10 +- .../api/java/operators/DistinctOperator.java | 7 +- .../PlanUnwrappingCoGroupOperator.java | 6 +- .../PlanUnwrappingReduceGroupOperator.java | 14 +-- .../translation/TupleUnwrappingIterator.java | 7 +- .../record/functions/CoGroupFunction.java | 4 +- .../java/record/functions/ReduceFunction.java | 7 +- .../DeltaIterationTranslationTest.java | 4 +- .../type/extractor/TypeExtractorTest.java | 5 +- .../CoGroupWithSolutionSetFirstDriver.java | 6 +- .../CoGroupWithSolutionSetSecondDriver.java | 5 +- .../sort/CombiningUnilateralSortMerger.java | 7 +- .../sort/FixedLengthRecordSorter.java | 2 +- .../operators/sort/InMemorySorter.java | 2 +- .../runtime/operators/sort/MergeIterator.java | 17 +-- .../operators/sort/MergeMatchIterator.java | 1 - .../sort/SortMergeCoGroupIterator.java | 20 ++- .../operators/util/CoGroupTaskIterator.java | 9 +- .../flink/runtime/util/EmptyIterator.java | 11 +- .../runtime/util/KeyGroupedIterator.java | 7 +- .../util/MutableToRegularIteratorWrapper.java | 7 +- .../runtime/util/SingleElementIterator.java | 7 +- .../operators/CoGroupTaskExternalITCase.java | 13 +- .../runtime/operators/CoGroupTaskTest.java | 22 ++-- .../runtime/operators/CombineTaskTest.java | 28 ++--- .../operators/ReduceTaskExternalITCase.java | 32 +++-- .../runtime/operators/ReduceTaskTest.java | 39 +++--- .../operators/chaining/ChainTaskTest.java | 7 +- .../drivers/AllGroupReduceDriverTest.java | 16 +-- .../drivers/GroupReduceDriverTest.java | 16 +-- .../CombiningUnilateralSortMergerITCase.java | 16 +-- .../sort/SortMergeCoGroupIteratorITCase.java | 4 +- .../api/scala/functions/CoGroupFunction.scala | 14 +-- .../api/scala/functions/ReduceFunction.scala | 22 ++-- .../api/scala/operators/CoGroupOperator.scala | 18 +-- .../api/scala/operators/ReduceOperator.scala | 21 ++-- .../test/accumulators/AccumulatorITCase.java | 12 +- .../AccumulatorIterativeITCase.java | 8 +- .../BulkIterationWithAllReducerITCase.java | 8 +- .../CoGroupConnectedComponentsITCase.java | 12 +- ...oGroupConnectedComponentsSecondITCase.java | 12 +- .../DependencyConnectedComponentsITCase.java | 19 ++- ...erationTerminationWithTerminationTail.java | 8 +- .../IterationTerminationWithTwoTails.java | 9 +- .../IterationWithAllReducerITCase.java | 6 +- .../IterationWithChainingITCase.java | 119 +++++++++--------- .../iterative/IterationWithUnionITCase.java | 8 +- ...ntsWithParametrizableAggregatorITCase.java | 16 +-- ...tsWithParametrizableConvergenceITCase.java | 11 +- .../IterationWithChainingNepheleITCase.java | 7 +- .../CustomCompensatableDotProductCoGroup.java | 14 +-- .../CustomRankCombiner.java | 13 +- .../CompensatableDotProductCoGroup.java | 14 +-- .../test/javaApiOperators/CoGroupITCase.java | 94 ++++++-------- .../javaApiOperators/GroupReduceITCase.java | 102 ++++++--------- .../flink/test/operators/CoGroupITCase.java | 40 ++---- .../flink/test/operators/ReduceITCase.java | 32 ++--- .../GroupOrderReduceITCase.java | 10 +- .../recordJobs/graph/ComputeEdgeDegrees.java | 13 +- .../graph/ConnectedComponentsWithCoGroup.java | 13 +- .../graph/DeltaPageRankWithInitialDeltas.java | 9 +- .../EnumTrianglesOnEdgesWithDegrees.java | 8 +- .../graph/EnumTrianglesRdfFoaf.java | 11 +- .../test/recordJobs/graph/PairwiseSP.java | 34 +++-- .../test/recordJobs/graph/SimplePageRank.java | 8 +- .../graph/WorksetConnectedComponents.java | 10 +- .../graph/pageRankUtil/DotProductCoGroup.java | 14 +-- .../recordJobs/kmeans/KMeansBroadcast.java | 12 +- .../recordJobs/kmeans/KMeansSingleStep.java | 12 +- .../kmeans/udfs/FindNearestCenter.java | 12 +- .../kmeans/udfs/RecomputeClusterCenter.java | 13 +- .../recordJobs/relational/MergeOnlyJoin.java | 9 +- .../recordJobs/relational/TPCHQuery10.java | 15 +-- .../recordJobs/relational/TPCHQuery3.java | 10 +- .../recordJobs/relational/TPCHQuery4.java | 13 +- .../relational/TPCHQueryAsterix.java | 23 ++-- .../recordJobs/relational/WebLogAnalysis.java | 12 +- .../query1Util/GroupByReturnFlag.java | 8 +- .../query9Util/AmountAggregate.java | 15 +-- .../test/recordJobs/sort/ReduceGroupSort.java | 10 +- .../test/recordJobs/wordcount/WordCount.java | 10 +- .../wordcount/WordCountAccumulators.java | 9 +- 108 files changed, 677 insertions(+), 909 deletions(-) diff --git a/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java b/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java index 9cdaef0acdf04..a6f69b036b4c9 100644 --- a/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java +++ b/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.io.Serializable; -import java.util.Iterator; import java.util.Random; import org.apache.flink.api.common.Plan; @@ -80,14 +79,14 @@ public static final class ConcatenatingReducer extends ReduceFunction implements private final Record result = new Record(2); @Override - public void reduce(Iterator records, Collector out) throws Exception { - Record r = records.next(); + public void reduce(Iterable records, Collector out) throws Exception { + Record r = records.iterator().next(); int num = r.getField(1, IntValue.class).getValue(); String names = r.getField(0, SUser.class).datum().getFavoriteColor().toString(); - while (records.hasNext()) { - r = records.next(); + for (Record next : records) { + r = next; names += " - " + r.getField(0, SUser.class).datum().getFavoriteColor().toString(); } diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java index ea9edffbc1c4d..dccee7c25ecc8 100644 --- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java +++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.avro.reflect.Nullable; @@ -194,16 +193,16 @@ public static class MyCoGrouper extends CoGroupFunction { private static final long serialVersionUID = 1L; @Override - public void coGroup(Iterator records1, Iterator records2, Collector out) + public void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception { Record r1 = null; - if (records1.hasNext()) { - r1 = records1.next(); + for (Record next : records1) { + r1 = next; } Record r2 = null; - if (records2.hasNext()) { - r2 = records2.next(); + for (Record next : records2) { + r2 = next; } if (r1 != null) { diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java index 25caf0c944d99..372eedd96f707 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.hadoopcompatibility.mapred.record.example; import java.io.Serializable; -import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.Plan; @@ -100,11 +98,12 @@ public static class CountWords extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -114,7 +113,7 @@ public void reduce(Iterator records, Collector out) throws Excep } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java index a3cd3d57634b6..d7f508054b42a 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.hadoopcompatibility.mapred.record.example; import java.io.Serializable; -import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.Plan; @@ -98,11 +96,12 @@ public static class CountWords extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -112,7 +111,7 @@ public void reduce(Iterator records, Collector out) throws Excep } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java index bb84ceaa5282a..6062021a79e00 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java @@ -413,24 +413,28 @@ private VertexUpdateUdf(VertexUpdateFunction ve } @Override - public void coGroup(Iterator> messages, Iterator> vertex, + public void coGroup(Iterable> messages, Iterable> vertex, Collector> out) throws Exception { - if (vertex.hasNext()) { - Tuple2 vertexState = vertex.next(); + final Iterator> vertexIter = vertex.iterator(); + + if (vertexIter.hasNext()) { + Tuple2 vertexState = vertexIter.next(); @SuppressWarnings("unchecked") - Iterator> downcastIter = (Iterator>) (Iterator) messages; + Iterator> downcastIter = (Iterator>) (Iterator) messages.iterator(); messageIter.setSource(downcastIter); vertexUpdateFunction.setOutput(vertexState, out); vertexUpdateFunction.updateVertex(vertexState.f0, vertexState.f1, messageIter); - } else { - if (messages.hasNext()) { + } + else { + final Iterator> messageIter = messages.iterator(); + if (messageIter.hasNext()) { String message = "Target vertex does not exist!."; try { - Tuple2 next = messages.next(); + Tuple2 next = messageIter.next(); message = "Target vertex '" + next.f0 + "' does not exist!."; } catch (Throwable t) {} throw new Exception(message); @@ -481,13 +485,15 @@ private MessagingUdfNoEdgeValues(MessagingFunction> edges, - Iterator> state, Collector> out) + public void coGroup(Iterable> edges, + Iterable> state, Collector> out) throws Exception { - if (state.hasNext()) { - Tuple2 newVertexState = state.next(); - messagingFunction.set((Iterator) edges, out); + final Iterator> stateIter = state.iterator(); + + if (stateIter.hasNext()) { + Tuple2 newVertexState = stateIter.next(); + messagingFunction.set((Iterator) edges.iterator(), out); messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); } } @@ -534,13 +540,15 @@ private MessagingUdfWithEdgeValues(MessagingFunction> edges, - Iterator> state, Collector> out) + public void coGroup(Iterable> edges, + Iterable> state, Collector> out) throws Exception { - if (state.hasNext()) { - Tuple2 newVertexState = state.next(); - messagingFunction.set((Iterator) edges, out); + final Iterator> stateIter = state.iterator(); + + if (stateIter.hasNext()) { + Tuple2 newVertexState = stateIter.next(); + messagingFunction.set((Iterator) edges.iterator(), out); messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); } } diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java index 3a58afcfc0659..5f973ff104cd4 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java @@ -160,20 +160,23 @@ public static final class VertexUpdateDriver, V extends Value, private MessageIterator messageIter; @Override - public void coGroup(Iterator messages, Iterator vertex, Collector out) throws Exception { + public void coGroup(Iterable messages, Iterable vertex, Collector out) throws Exception { - if (vertex.hasNext()) { - Record first = vertex.next(); + final Iterator vertexIter = vertex.iterator(); + + if (vertexIter.hasNext()) { + Record first = vertexIter.next(); first.getFieldInto(0, vertexKey); first.getFieldInto(1, vertexValue); - messageIter.setSource(messages); + messageIter.setSource(messages.iterator()); vertexUpdateFunction.setOutput(first, out); vertexUpdateFunction.updateVertex(vertexKey, vertexValue, messageIter); } else { - if (messages.hasNext()) { + final Iterator messageIter = messages.iterator(); + if (messageIter.hasNext()) { String message = "Target vertex does not exist!."; try { - Record next = messages.next(); + Record next = messageIter.next(); next.getFieldInto(0, vertexKey); message = "Target vertex '" + vertexKey + "' does not exist!."; } catch (Throwable t) {} @@ -233,12 +236,13 @@ public static final class MessagingDriver, V extends Value, M e private V vertexValue; @Override - public void coGroup(Iterator edges, Iterator state, Collector out) throws Exception { - if (state.hasNext()) { - Record first = state.next(); + public void coGroup(Iterable edges, Iterable state, Collector out) throws Exception { + final Iterator stateIter = state.iterator(); + if (stateIter.hasNext()) { + Record first = stateIter.next(); first.getFieldInto(0, vertexKey); first.getFieldInto(1, vertexValue); - messagingFunction.set(edges, out); + messagingFunction.set(edges.iterator(), out); messagingFunction.sendMessages(vertexKey, vertexValue); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java index 1c30545d07e4c..de35efa232faa 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.compiler; -import java.util.Iterator; - import org.junit.Assert; import org.junit.Test; import org.apache.flink.api.common.Plan; @@ -45,8 +42,7 @@ public class CoGroupSolutionSetFirstTest extends CompilerTestBase { public static class SimpleCGroup extends CoGroupFunction, Tuple1, Tuple1> { @Override - public void coGroup(Iterator> first, Iterator> second, Collector> out) throws Exception { - } + public void coGroup(Iterable> first, Iterable> second, Collector> out) {} } public static class SimpleMap extends MapFunction, Tuple1> { diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java index f3e513a97baf3..2419e05af1ba1 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.compiler; -import java.util.Iterator; - import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.util.FieldList; import org.apache.flink.api.java.functions.GroupReduceFunction; @@ -51,7 +48,7 @@ public void testAllGroupReduceNoCombiner() { DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source"); data.reduceGroup(new GroupReduceFunction() { - public void reduce(Iterator values, Collector out) {} + public void reduce(Iterable values, Collector out) {} }).name("reducer") .print().name("sink"); @@ -95,7 +92,7 @@ public void testAllReduceWithCombiner() { DataSet data = env.generateSequence(1, 8000000).name("source"); ReduceGroupOperator reduced = data.reduceGroup(new GroupReduceFunction() { - public void reduce(Iterator values, Collector out) {} + public void reduce(Iterable values, Collector out) {} }).name("reducer"); reduced.setCombinable(true); @@ -148,7 +145,7 @@ public void testGroupedReduceWithFieldPositionKeyNonCombinable() { data .groupBy(1) .reduceGroup(new GroupReduceFunction, Tuple2>() { - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} }).name("reducer") .print().name("sink"); @@ -197,7 +194,7 @@ public void testGroupedReduceWithFieldPositionKeyCombinable() { ReduceGroupOperator, Tuple2> reduced = data .groupBy(1) .reduceGroup(new GroupReduceFunction, Tuple2>() { - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} }).name("reducer"); reduced.setCombinable(true); @@ -256,7 +253,7 @@ public void testGroupedReduceWithSelectorFunctionKeyNoncombinable() { public String getKey(Tuple2 value) { return value.f0; } }) .reduceGroup(new GroupReduceFunction, Tuple2>() { - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} }).name("reducer") .print().name("sink"); @@ -314,7 +311,7 @@ public void testGroupedReduceWithSelectorFunctionKeyCombinable() { public String getKey(Tuple2 value) { return value.f0; } }) .reduceGroup(new GroupReduceFunction, Tuple2>() { - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} }).name("reducer"); reduced.setCombinable(true); diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java index d3c3e3f643c20..1445436a442a4 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java @@ -16,15 +16,12 @@ * limitations under the License. */ - package org.apache.flink.compiler; import static org.junit.Assert.*; import org.junit.Test; -import java.util.Iterator; - import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.DeltaIteration; @@ -287,7 +284,7 @@ public Tuple2 map(Tuple2 value) throws Exception { public static final class Reduce101 extends GroupReduceFunction, Tuple1> { @Override - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} } @ConstantFields("0") diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java index e04256ced92f7..acdf992f73216 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java @@ -16,15 +16,12 @@ * limitations under the License. */ - package org.apache.flink.compiler; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.util.Iterator; - import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.util.FieldList; @@ -283,7 +280,7 @@ public Tuple3 join(Tuple3 first, Tuple3> nextWorkset = joinedWithSolutionSet.groupBy(1, 2) .reduceGroup(new GroupReduceFunction, Tuple3>() { - public void reduce(Iterator> values, Collector> out) {} + public void reduce(Iterable> values, Collector> out) {} }) .name(NEXT_WORKSET_REDUCER_NAME) .withConstantSet("1->1","2->2","0->0"); diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java index fe61f25dfa480..3b4c35187811b 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.compiler.testfunctions; -import java.util.Iterator; - import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; import org.apache.flink.util.Collector; @@ -32,9 +29,9 @@ public class IdentityGroupReducer extends GroupReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator values, Collector out) { - while (values.hasNext()) { - out.collect(values.next()); + public void reduce(Iterable values, Collector out) { + for (T next : values) { + out.collect(next); } } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java index 26db00e00b22d..1a4df49bcedfa 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.compiler.testfunctions; -import java.util.Iterator; - import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.GroupReduceFunction.Combinable; import org.apache.flink.util.Collector; @@ -32,7 +29,7 @@ public class Top1GroupReducer extends GroupReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator values, Collector out) { - out.collect(values.next()); + public void reduce(Iterable values, Collector out) { + out.collect(values.iterator().next()); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java index 6ef165102c21d..9d32af87bc97a 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.compiler.util; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.types.Record; @@ -30,13 +28,13 @@ public class DummyCoGroupStub extends CoGroupFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void coGroup(Iterator records1, Iterator records2, Collector out) { - while (records1.hasNext()) { - out.collect(records1.next()); + public void coGroup(Iterable records1, Iterable records2, Collector out) { + for (Record r : records1) { + out.collect(r); } - while (records2.hasNext()) { - out.collect(records2.next()); + for (Record r : records2) { + out.collect(r); } } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java b/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java index 3f3242387fee8..9cd3b0f068be1 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.compiler.util; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept; @@ -32,9 +30,9 @@ public final class IdentityReduce extends ReduceFunction implements Serializable private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { - while (records.hasNext()) { - out.collect(records.next()); + public void reduce(Iterable records, Collector out) throws Exception { + for (Record r : records) { + out.collect(r); } } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java index 59669a242a509..0d7b28f6923d8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java @@ -19,8 +19,6 @@ package org.apache.flink.api.common.functions; -import java.util.Iterator; - import org.apache.flink.util.Collector; @@ -35,6 +33,6 @@ public interface GenericCoGrouper extends Function { * @param records2 The records from the second input which were paired with the key. * @param out A collector that collects all output pairs. */ - void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; + void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java index 8dfe758b68f28..2c48019aebfb9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCombine.java @@ -19,7 +19,6 @@ package org.apache.flink.api.common.functions; -import java.util.Iterator; import org.apache.flink.util.Collector; @@ -28,5 +27,11 @@ */ public interface GenericCombine extends Function { - void combine(Iterator records, Collector out) throws Exception; + /** + * @param records The records from the current group. + * @param out The collector used to emit result records. + * @throws Exception The method may throw exceptions, which will cause the task to fail + * and may trigger the recovery logic. + */ + void combine(Iterable records, Collector out) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java index e8d9910b433ce..76531308b6874 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericGroupReduce.java @@ -16,20 +16,17 @@ * limitations under the License. */ - package org.apache.flink.api.common.functions; -import java.util.Iterator; - import org.apache.flink.util.Collector; - /** * - * @param Incoming types - * @param Outgoing types + * @param The type of the incoming records. + * @param The type of the outgoing records. */ public interface GenericGroupReduce extends Function { + /** * * The central function to be implemented for a reducer. The function receives per call one @@ -40,5 +37,5 @@ public interface GenericGroupReduce extends Function { * @param out The collector to hand results to. * @throws Exception */ - void reduce(Iterator records, Collector out) throws Exception; + void reduce(Iterable records, Collector out) throws Exception; } diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java index fba18fcecbcc5..27b7e9ff31d28 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java @@ -19,7 +19,6 @@ package org.apache.flink.example.java.graph; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.operators.Order; @@ -35,7 +34,7 @@ import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Triad; /** - * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. + * Triangle enumeration is a pre-processing step to find closely connected parts in graphs. * A triangle consists of three edges that connect three vertices with each other. * *

@@ -154,22 +153,22 @@ private static class TriadBuilder extends GroupReduceFunction { private final Triad outTriad = new Triad(); @Override - public void reduce(Iterator edges, Collector out) throws Exception { + public void reduce(Iterable edges, Collector out) throws Exception { // clear vertex list vertices.clear(); // read first edge - Edge firstEdge = edges.next(); + Edge firstEdge = edges.iterator().next(); outTriad.setFirstVertex(firstEdge.getFirstVertex()); vertices.add(firstEdge.getSecondVertex()); // build and emit triads - while(edges.hasNext()) { - Integer higherVertexId = edges.next().getSecondVertex(); + for (Edge e : edges) { + Integer higherVertexId = e.getSecondVertex(); // combine vertex with all previously read vertices - for(Integer lowerVertexId : vertices) { + for (Integer lowerVertexId : vertices) { outTriad.setSecondVertex(lowerVertexId); outTriad.setThirdVertex(higherVertexId); out.collect(outTriad); diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java index 265ce752f7423..39c92eddae681 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java @@ -19,7 +19,6 @@ package org.apache.flink.example.java.graph; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.operators.Order; @@ -38,7 +37,7 @@ import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Triad; /** - * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. + * Triangle enumeration is a pre-processing step to find closely connected parts in graphs. * A triangle consists of three edges that connect three vertices with each other. * *

@@ -166,18 +165,18 @@ private static class DegreeCounter extends GroupReduceFunction edges, Collector out) throws Exception { + public void reduce(Iterable edges, Collector out) { otherVertices.clear(); // get first edge - Edge edge = edges.next(); + Edge edge = edges.iterator().next(); Integer groupVertex = edge.getFirstVertex(); this.otherVertices.add(edge.getSecondVertex()); // get all other edges (assumes edges are sorted by second vertex) - while(edges.hasNext()) { - edge = edges.next(); + for (Edge e : edges) { + edge = e; Integer otherVertex = edge.getSecondVertex(); // collect unique vertices if(!otherVertices.contains(otherVertex) && otherVertex != groupVertex) { @@ -274,19 +273,19 @@ private static class TriadBuilder extends GroupReduceFunction { private final Triad outTriad = new Triad(); @Override - public void reduce(Iterator edges, Collector out) throws Exception { + public void reduce(Iterable edges, Collector out) throws Exception { // clear vertex list vertices.clear(); // read first edge - Edge firstEdge = edges.next(); + Edge firstEdge = edges.iterator().next(); outTriad.setFirstVertex(firstEdge.getFirstVertex()); vertices.add(firstEdge.getSecondVertex()); // build and emit triads - while(edges.hasNext()) { - Integer higherVertexId = edges.next().getSecondVertex(); + for (Edge e : edges) { + Integer higherVertexId = e.getSecondVertex(); // combine vertex with all previously read vertices for(Integer lowerVertexId : vertices) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java index 18eba5d3deca5..68c5606a19b3c 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java @@ -21,7 +21,6 @@ import static org.apache.flink.api.java.aggregation.Aggregations.SUM; import java.util.ArrayList; -import java.util.Iterator; import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.functions.FlatMapFunction; @@ -163,12 +162,11 @@ public static final class BuildOutgoingEdgeList extends GroupReduceFunction neighbors = new ArrayList(); @Override - public void reduce(Iterator> values, Collector> out) { + public void reduce(Iterable> values, Collector> out) { neighbors.clear(); Long id = 0L; - while (values.hasNext()) { - Tuple2 n = values.next(); + for (Tuple2 n : values) { id = n.f0; neighbors.add(n.f1); } diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java index d8d8b62f75c62..bd554e3b5d31c 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java @@ -23,8 +23,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; -import java.util.Iterator; - import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.functions.JoinFunction; @@ -70,8 +68,8 @@ public Tuple2 join(Tuple2 left, Tuple2 right .groupBy(0, 1) .reduceGroup(new GroupReduceFunction, Tuple2>() { @Override - public void reduce(Iterator> values, Collector> out) throws Exception { - out.collect(values.next()); + public void reduce(Iterable> values, Collector> out) throws Exception { + out.collect(values.iterator().next()); } }); diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java index 2649d24fe0867..dfc23e024ddb5 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.example.java.relational; -import java.util.Iterator; - import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.functions.FilterFunction; import org.apache.flink.api.java.tuple.Tuple1; @@ -249,12 +246,12 @@ public static class AntiJoinVisits extends CoGroupFunction> ranks, Iterator> visits, Collector> out) { + public void coGroup(Iterable> ranks, Iterable> visits, Collector> out) { // Check if there is a entry in the visits relation - if (!visits.hasNext()) { - while (ranks.hasNext()) { + if (!visits.iterator().hasNext()) { + for (Tuple3 next : ranks) { // Emit all rank pairs - out.collect(ranks.next()); + out.collect(next); } } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java index 201794abce4a8..fbc8dd4492629 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/CoGroupFunction.java @@ -18,8 +18,6 @@ package org.apache.flink.api.java.functions; -import java.util.Iterator; - import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.util.Collector; @@ -69,6 +67,6 @@ public abstract class CoGroupFunction extends AbstractFunction im * to fail and may trigger recovery. */ @Override - public abstract void coGroup(Iterator first, Iterator second, Collector out) throws Exception; + public abstract void coGroup(Iterable first, Iterable second, Collector out) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java index 01ae9c1addbef..1148a82c4d511 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceFunction.java @@ -22,7 +22,6 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import java.util.Iterator; import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCombine; @@ -44,7 +43,7 @@ * *

* GroupReduceFunctions may be "combinable", in which case they can pre-reduce partial groups in order to - * reduce the data volume early. See the {@link #combine(Iterator, Collector)} function for details. + * reduce the data volume early. See the {@link #combine(Iterable, Collector)} function for details. *

* Like all functions, the GroupReduceFunction needs to be serializable, as defined in {@link java.io.Serializable}. * @@ -66,7 +65,7 @@ public abstract class GroupReduceFunction extends AbstractFunction impl * to fail and may trigger recovery. */ @Override - public abstract void reduce(Iterator values, Collector out) throws Exception; + public abstract void reduce(Iterable values, Collector out) throws Exception; /** * The combine methods pre-reduces elements. It may be called on subsets of the data @@ -81,7 +80,7 @@ public abstract class GroupReduceFunction extends AbstractFunction impl *

* Since the reduce function will be called on the result of this method, it is important that this * method returns the same data type as it consumes. By default, this method only calls the - * {@link #reduce(Iterator, Collector)} method. If the behavior in the pre-reducing is different + * {@link #reduce(Iterable, Collector)} method. If the behavior in the pre-reducing is different * from the final reduce function (for example because the reduce function changes the data type), * this method must be overwritten, or the execution will fail. * @@ -92,7 +91,7 @@ public abstract class GroupReduceFunction extends AbstractFunction impl * to fail and may trigger recovery. */ @Override - public void combine(Iterator values, Collector out) throws Exception { + public void combine(Iterable values, Collector out) throws Exception { @SuppressWarnings("unchecked") Collector c = (Collector) out; reduce(values, c); @@ -102,7 +101,7 @@ public void combine(Iterator values, Collector out) throws Exception { /** * This annotation can be added to classes that extend {@link GroupReduceFunction}, in oder to mark - * them as "combinable". The system may call the {@link GroupReduceFunction#combine(Iterator, Collector)} + * them as "combinable". The system may call the {@link GroupReduceFunction#combine(Iterable, Collector)} * method on such functions, to pre-reduce the data before transferring it over the network to * the actual group reduce operation. *

diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java index 6cb397b93d4cd..cd87246da3349 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java @@ -28,13 +28,13 @@ public abstract class GroupReduceIterator extends GroupReduceFunction reduceGroup(Iterator values) throws Exception; + public abstract Iterator reduceGroup(Iterable values) throws Exception; // ------------------------------------------------------------------------------------------- @Override - public final void reduce(Iterator values, Collector out) throws Exception { + public final void reduce(Iterable values, Collector out) throws Exception { for (Iterator iter = reduceGroup(values); iter.hasNext(); ) { out.collect(iter.next()); } 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 ca6ed940950c4..420311d912daa 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 @@ -271,13 +271,14 @@ public void open(Configuration parameters) throws Exception { } @Override - public void reduce(Iterator values, Collector out) { + public void reduce(Iterable records, Collector out) { final AggregationFunction[] aggFunctions = this.aggFunctions; final int[] fieldPositions = this.fieldPositions; // aggregators are initialized from before T current = null; + final Iterator values = records.iterator(); while (values.hasNext()) { current = values.next(); 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 8748556da0b99..075d5431689e2 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 @@ -300,7 +300,7 @@ private static PlanUnwrappingCoGroupOperator tr /** * Intermediate step of a CoGroup transformation.
* To continue the CoGroup transformation, select the grouping key of the first input {@link DataSet} by calling - * {@link CoGroupOperatorSets#where()} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(KeySelector)}. + * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(int, 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. * @param The type of the second input DataSet of the CoGroup transformation. @@ -327,7 +327,7 @@ public CoGroupOperatorSets(DataSet input1, DataSet input2) { * @param field0 The first index of the Tuple fields of the first co-grouped DataSets that should be used as key * @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 CoGroupOperatorSetsPredicate#equalTo()} to continue the CoGroup. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int, int...)} to continue the CoGroup. * * @see Tuple * @see DataSet @@ -347,7 +347,7 @@ public CoGroupOperatorSetsPredicate where(int field0, int... fields) { * @param field0 The first field of the Tuple fields of the first co-grouped DataSets that should be used as key * @param fields The fields of the first co-grouped DataSets that should be used as keys. * @return An incomplete CoGroup transformation. - * Call {@link CoGroupOperatorSetsPredicate#equalTo()} to continue the CoGroup. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int, int...)} to continue the CoGroup. * * @see Tuple * @see DataSet @@ -366,7 +366,7 @@ public CoGroupOperatorSetsPredicate where(String field0, String... fields) { * * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is grouped. * @return An incomplete CoGroup transformation. - * Call {@link CoGroupOperatorSetsPredicate#equalTo()} to continue the CoGroup. + * Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int, int...)} to continue the CoGroup. * * @see KeySelector * @see DataSet @@ -380,7 +380,7 @@ 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 - * {@link CoGroupOperatorSetsPredicate#equalTo(int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(KeySelector)}. + * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int, int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(KeySelector)}. * */ public final class CoGroupOperatorSetsPredicate { 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 cb7db063b57e9..aa4a924a2b609 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 @@ -18,8 +18,6 @@ package org.apache.flink.api.java.operators; -import java.util.Iterator; - import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.api.common.functions.GenericMap; @@ -143,9 +141,8 @@ public static final class DistinctFunction extends GroupReduceFunction private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator values, Collector out) - throws Exception { - out.collect(values.next()); + public void reduce(Iterable values, Collector out) { + out.collect(values.iterator().next()); } } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java index 20bd3b0bff9a8..909548350bf46 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java @@ -79,9 +79,9 @@ private TupleUnwrappingCoGrouper(CoGroupFunction wrapped) { @Override - public void coGroup(Iterator> records1, Iterator> records2, Collector out) throws Exception { - iter1.set(records1); - iter2.set(records2); + public void coGroup(Iterable> records1, Iterable> records2, Collector out) throws Exception { + iter1.set(records1.iterator()); + iter2.set(records2.iterator()); this.wrappedFunction.coGroup(iter1, iter2, out); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java index 5a59664186d85..2c1f845937e69 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java @@ -18,8 +18,6 @@ package org.apache.flink.api.java.operators.translation; -import java.util.Iterator; - import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -66,14 +64,14 @@ private TupleUnwrappingCombinableGroupReducer(GroupReduceFunction wrapp @Override - public void reduce(Iterator> values, Collector out) throws Exception { - iter.set(values); + public void reduce(Iterable> values, Collector out) throws Exception { + iter.set(values.iterator()); this.wrappedFunction.reduce(iter, out); } @Override - public void combine(Iterator> values, Collector> out) throws Exception { - iter.set(values); + public void combine(Iterable> values, Collector> out) throws Exception { + iter.set(values.iterator()); coll.set(out); this.wrappedFunction.combine(iter, coll); } @@ -99,8 +97,8 @@ private TupleUnwrappingNonCombinableGroupReducer(GroupReduceFunction wr @Override - public void reduce(Iterator> values, Collector out) throws Exception { - iter.set(values); + public void reduce(Iterable> values, Collector out) throws Exception { + iter.set(values.iterator()); this.wrappedFunction.reduce(iter, out); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java index 1f2c208ed1e8d..47074282aaca6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java @@ -26,7 +26,7 @@ * An iterator that reads 2-tuples (key value pairs) and returns only the values (second field). * The iterator also tracks the keys, as the pairs flow though it. */ -public class TupleUnwrappingIterator implements Iterator, java.io.Serializable { +public class TupleUnwrappingIterator implements Iterator, Iterable, java.io.Serializable { private static final long serialVersionUID = 1L; @@ -57,4 +57,9 @@ public T next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java index 633adabb69f82..933ae919cfb86 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java @@ -19,8 +19,6 @@ package org.apache.flink.api.java.record.functions; -import java.util.Iterator; - import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.types.Record; @@ -47,6 +45,6 @@ public abstract class CoGroupFunction extends AbstractFunction implements Generi * decide whether to retry the task execution. */ @Override - public abstract void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; + public abstract void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java index 4b1dbb3ca0ee0..b7acd1e2bde2f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.api.java.record.functions; -import java.util.Iterator; - import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; @@ -48,7 +45,7 @@ public abstract class ReduceFunction extends AbstractFunction implements Generic * decide whether to retry the reduce execution. */ @Override - public abstract void reduce(Iterator records, Collector out) throws Exception; + public abstract void reduce(Iterable records, Collector out) throws Exception; /** * No default implementation provided. @@ -72,7 +69,7 @@ public abstract class ReduceFunction extends AbstractFunction implements Generic * decide whether to retry the combiner execution. */ @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { // to be implemented, if the reducer should use a combiner. Note that the combining method // is only used, if the stub class is further annotated with the annotation // @ReduceOperator.Combinable diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java index db795d9edb52e..e1b7dde5db940 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java @@ -240,7 +240,7 @@ public T map(T value) throws Exception { public static class SolutionWorksetCoGroup1 extends CoGroupFunction, Tuple3, Tuple3> { @Override - public void coGroup(Iterator> first, Iterator> second, + public void coGroup(Iterable> first, Iterable> second, Collector> out) { } } @@ -248,7 +248,7 @@ public void coGroup(Iterator> first, Iterator, Tuple2, Tuple3> { @Override - public void coGroup(Iterator> second, Iterator> first, + public void coGroup(Iterable> second, Iterable> first, Collector> out) { } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java index b284052ca1f30..2b12fe2843f75 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java @@ -21,7 +21,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.Iterator; import org.apache.flink.api.common.functions.GenericMap; import org.apache.flink.api.common.functions.RuntimeContext; @@ -72,7 +71,7 @@ public void testBasicType() { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator values, Collector out) throws Exception { + public void reduce(Iterable values, Collector out) throws Exception { // nothing to do } }; @@ -1084,7 +1083,7 @@ public void testBasicArray() { private static final long serialVersionUID = 1L; @Override - public void coGroup(Iterator first, Iterator second, Collector out) throws Exception { + public void coGroup(Iterable first, Iterable second, Collector out) throws Exception { // nothing to do } }; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java index 9d066186bf113..c725b867342db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java @@ -16,10 +16,9 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; -import java.util.Iterator; +import java.util.Collections; import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -29,7 +28,6 @@ import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker; import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask; import org.apache.flink.runtime.operators.hash.CompactingHashTable; -import org.apache.flink.runtime.util.EmptyIterator; import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; @@ -132,7 +130,7 @@ public void run() throws Exception { final KeyGroupedIterator probeSideInput = new KeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); final SingleElementIterator siIter = new SingleElementIterator(); - final Iterator emptySolutionSide = EmptyIterator.get(); + final Iterable emptySolutionSide = Collections.emptySet(); final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java index 80fa855f477ad..6727c17202e5b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; -import java.util.Iterator; - import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; @@ -132,7 +129,7 @@ public void run() throws Exception { final KeyGroupedIterator probeSideInput = new KeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); final SingleElementIterator siIter = new SingleElementIterator(); - final Iterator emptySolutionSide = EmptyIterator.get(); + final Iterable emptySolutionSide = EmptyIterator.get(); final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 06d0ac72a807f..76326d423270d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -502,7 +502,7 @@ protected ChannelWithBlockCount mergeChannels(List channe * This class implements an iterator over values from a sort buffer. The iterator returns the values of a given * interval. */ - private static final class CombineValueIterator implements Iterator { + private static final class CombineValueIterator implements Iterator, Iterable { private final InMemorySorter buffer; // the buffer from which values are returned @@ -563,6 +563,11 @@ public E next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } }; // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java index f5750764a56c1..ee2a2c9ac3150 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java @@ -410,7 +410,7 @@ record = comparator.readWithKeyDenormalization(record, inView); * * @param output The output view to write the records to. * @param start The logical start position of the subset. - * @param len The number of elements to write. + * @param num The number of elements to write. * @throws IOException Thrown, if an I/O exception occurred writing to the output view. */ @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java index a018def8e67f8..6f754908b6ebb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java @@ -103,7 +103,7 @@ public interface InMemorySorter extends IndexedSortable { * * @param output The output view to write the records to. * @param start The logical start position of the subset. - * @param len The number of elements to write. + * @param num The number of elements to write. * @throws IOException Thrown, if an I/O exception occurred writing to the output view. */ public void writeToOutput(final ChannelWriterOutputView output, final int start, int num) throws IOException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java index 9ef2ad7240e85..6881cdf3d4e3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java @@ -34,24 +34,19 @@ * to the next smallest element logarithmic in complexity, with respect to the * number of streams to be merged. * The order among the elements is established using the methods from the - * {@link TypeSerializer} class, specifically {@link TypeSerializer#setReference(Object)} - * and {@link TypeSerializer#compareToReference(TypeSerializer)}. - * - * @see TypeSerializer - * @see TypeSerializer#setReference(Object) - * @see TypeSerializer#compareToReference(TypeSerializer) - * + * {@link TypeComparator} class, specifically {@link TypeComparator#setReference(Object)} + * and {@link TypeComparator#compareToReference(TypeComparator)}. */ -public class MergeIterator implements MutableObjectIterator -{ +public class MergeIterator implements MutableObjectIterator { + private final PartialOrderPriorityQueue> heap; // heap over the head elements of the stream private final TypeSerializer serializer; /** * @param iterators - * @param accessors The accessors used to establish an order among the elements. - * The accessors will not be used directly, but a duplicate will be used. + * @param serializer + * @param comparator * @throws IOException */ public MergeIterator(List> iterators, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java index 308e3339e2de5..0806126bf0097 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.sort; import java.io.IOException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java index dcd336110844f..f6803fc677ffe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java @@ -16,17 +16,15 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.sort; import java.io.IOException; -import java.util.Iterator; +import java.util.Collections; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.CoGroupTaskIterator; -import org.apache.flink.runtime.util.EmptyIterator; import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.util.MutableObjectIterator; @@ -41,9 +39,9 @@ private static enum MatchStatus { private MatchStatus matchStatus; - private Iterator firstReturn; + private Iterable firstReturn; - private Iterator secondReturn; + private Iterable secondReturn; private TypePairComparator comp; @@ -73,13 +71,13 @@ public void close() {} @Override - public Iterator getValues1() { + public Iterable getValues1() { return this.firstReturn; } @Override - public Iterator getValues2() { + public Iterable getValues2() { return this.secondReturn; } @@ -117,7 +115,7 @@ public boolean next() throws IOException { } else if (firstEmpty && !secondEmpty) { // input1 is empty, input2 not - this.firstReturn = EmptyIterator.get(); + this.firstReturn = Collections.emptySet(); this.secondReturn = this.iterator2.getValues(); this.matchStatus = MatchStatus.FIRST_EMPTY; return true; @@ -125,7 +123,7 @@ else if (firstEmpty && !secondEmpty) { else if (!firstEmpty && secondEmpty) { // input1 is not empty, input 2 is empty this.firstReturn = this.iterator1.getValues(); - this.secondReturn = EmptyIterator.get(); + this.secondReturn = Collections.emptySet(); this.matchStatus = MatchStatus.SECOND_EMPTY; return true; } @@ -142,12 +140,12 @@ else if (!firstEmpty && secondEmpty) { else if (0 < comp) { // key1 goes first this.firstReturn = this.iterator1.getValues(); - this.secondReturn = EmptyIterator.get(); + this.secondReturn = Collections.emptySet(); this.matchStatus = MatchStatus.SECOND_REMAINED; } else { // key 2 goes first - this.firstReturn = EmptyIterator.get(); + this.firstReturn = Collections.emptySet(); this.secondReturn = this.iterator2.getValues(); this.matchStatus = MatchStatus.FIRST_REMAINED; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java index a20490fe99504..e137e27764058 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.operators.util; import java.io.IOException; -import java.util.Iterator; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; @@ -30,8 +29,8 @@ * @param The generic type of the first input's data type. * @param The generic type of the second input's data type. */ -public interface CoGroupTaskIterator -{ +public interface CoGroupTaskIterator { + /** * General-purpose open method. * @@ -63,12 +62,12 @@ public interface CoGroupTaskIterator * * @return an iterable over the left input values for the current key. */ - Iterator getValues1(); + Iterable getValues1(); /** * Returns an iterable over the left input values for the current key. * * @return an iterable over the left input values for the current key. */ - Iterator getValues2(); + Iterable getValues2(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyIterator.java index b1628d7e76654..caafaea78bf73 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyIterator.java @@ -25,7 +25,7 @@ /** * An empty iterator that never returns anything. */ -public final class EmptyIterator implements Iterator { +public final class EmptyIterator implements Iterator, Iterable { /** * The singleton instance. @@ -38,9 +38,9 @@ public final class EmptyIterator implements Iterator { * @param The type of the objects (not) returned by the iterator. * @return An instance of the iterator. */ - public static Iterator get() { + public static EmptyIterator get() { @SuppressWarnings("unchecked") - Iterator iter = (Iterator) INSTANCE; + EmptyIterator iter = (EmptyIterator) INSTANCE; return iter; } @@ -73,4 +73,9 @@ public E next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java index 0873baf587fb0..f583593b18cf1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java @@ -151,7 +151,7 @@ public ValuesIterator getValues() { // -------------------------------------------------------------------------------------------- - public final class ValuesIterator implements Iterator + public final class ValuesIterator implements Iterator, Iterable { private final TypeSerializer serializer = KeyGroupedIterator.this.serializer; private final TypeComparator comparator = KeyGroupedIterator.this.comparator; @@ -221,5 +221,10 @@ public E next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java index e707a50db43b4..0558e47ea1f2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java @@ -32,7 +32,7 @@ * whenever hasNext() returns (possibly with false), the previous obtained record is * still valid and cannot have been overwritten internally. */ -public class MutableToRegularIteratorWrapper implements Iterator { +public class MutableToRegularIteratorWrapper implements Iterator, Iterable { private final MutableObjectIterator source; @@ -85,4 +85,9 @@ public T next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SingleElementIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SingleElementIterator.java index 222c5b841534c..f7eb41ac14907 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SingleElementIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SingleElementIterator.java @@ -27,7 +27,7 @@ * * @param The generic type of the iterator. */ -public final class SingleElementIterator implements Iterator { +public final class SingleElementIterator implements Iterator, Iterable { private E current; private boolean available = false; @@ -61,4 +61,9 @@ public E next() { public void remove() { throw new UnsupportedOperationException(); } + + @Override + public Iterator iterator() { + return this; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java index e8024f33adba5..9624676d0d7d7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; -import java.util.Iterator; - import junit.framework.Assert; import org.apache.flink.api.common.functions.GenericCoGrouper; @@ -92,20 +89,18 @@ public static final class MockCoGroupStub extends CoGroupFunction { private final Record res = new Record(); + @SuppressWarnings("unused") @Override - public void coGroup(Iterator records1, Iterator records2, Collector out) - { + public void coGroup(Iterable records1, Iterable records2, Collector out) { int val1Cnt = 0; int val2Cnt = 0; - while (records1.hasNext()) { + for (Record r : records1) { val1Cnt++; - records1.next(); } - while (records2.hasNext()) { + for (Record r : records2) { val2Cnt++; - records2.next(); } if (val1Cnt == 0) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java index a453cf6b6432d..c06ce0ee8a31e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java @@ -16,10 +16,8 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; -import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; import junit.framework.Assert; @@ -409,18 +407,14 @@ public static class MockFailingCoGroupStub extends CoGroupFunction { private int cnt = 0; @Override - public void coGroup(Iterator records1, - Iterator records2, Collector out) throws RuntimeException - { + public void coGroup(Iterable records1, Iterable records2, Collector out) { int val1Cnt = 0; - while (records1.hasNext()) { + for (@SuppressWarnings("unused") Record r : records1) { val1Cnt++; - records1.next(); } - while (records2.hasNext()) { - Record record2 = records2.next(); + for (Record record2 : records2) { if (val1Cnt == 0) { if(++this.cnt>=10) { @@ -446,22 +440,20 @@ public void coGroup(Iterator records1, public static final class MockDelayingCoGroupStub extends CoGroupFunction { private static final long serialVersionUID = 1L; + @SuppressWarnings("unused") @Override - public void coGroup(Iterator records1, - Iterator records2, Collector out) { + public void coGroup(Iterable records1, Iterable records2, Collector out) { - while (records1.hasNext()) { + for (Record r : records1) { try { Thread.sleep(100); } catch (InterruptedException e) { } - records1.next(); } - while (records2.hasNext()) { + for (Record r : records2) { try { Thread.sleep(100); } catch (InterruptedException e) { } - records2.next(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java index 27fc11be2a815..ad29bf31ca175 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import java.util.ArrayList; -import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; import junit.framework.Assert; @@ -171,12 +169,12 @@ public static class MockCombiningReduceStub extends ReduceFunction { private final IntValue theInteger = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.theInteger); sum += this.theInteger.getValue(); @@ -187,7 +185,7 @@ public void reduce(Iterator records, Collector out) } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { reduce(records, out); } } @@ -203,12 +201,12 @@ public static final class MockFailingCombiningReduceStub extends ReduceFunction private final IntValue combineValue = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.value); sum += this.value.getValue(); @@ -220,12 +218,12 @@ public void reduce(Iterator records, Collector out) } @Override - public void combine(Iterator records, Collector out) - throws Exception { + public void combine(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.combineValue); sum += this.combineValue.getValue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java index afcbf9704f1be..b14f0d3d860a0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import junit.framework.Assert; @@ -43,8 +41,8 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class ReduceTaskExternalITCase extends DriverTestBase> -{ +public class ReduceTaskExternalITCase extends DriverTestBase> { + private static final Log LOG = LogFactory.getLog(ReduceTaskExternalITCase.class); @SuppressWarnings("unchecked") @@ -222,12 +220,12 @@ public static class MockReduceStub extends ReduceFunction { private final IntValue value = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int cnt = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; cnt++; } element.getField(0, this.key); @@ -246,12 +244,12 @@ public static class MockCombiningReduceStub extends ReduceFunction { private final IntValue combineValue = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.value); sum += this.value.getValue(); @@ -263,12 +261,12 @@ public void reduce(Iterator records, Collector out) } @Override - public void combine(Iterator records, Collector out) - throws Exception { + public void combine(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.combineValue); sum += this.combineValue.getValue(); @@ -278,7 +276,5 @@ public void combine(Iterator records, Collector out) element.setField(1, this.combineValue); out.collect(element); } - } - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java index 6d8ec94658c60..c4858f470f025 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -279,12 +277,12 @@ public static class MockReduceStub extends ReduceFunction { private final IntValue value = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int cnt = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; cnt++; } element.getField(0, this.key); @@ -303,12 +301,12 @@ public static class MockCombiningReduceStub extends ReduceFunction { private final IntValue combineValue = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.value); sum += this.value.getValue(); @@ -320,12 +318,12 @@ public void reduce(Iterator records, Collector out) } @Override - public void combine(Iterator records, Collector out) - throws Exception { + public void combine(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; element.getField(1, this.combineValue); sum += this.combineValue.getValue(); @@ -347,12 +345,12 @@ public static class MockFailingReduceStub extends ReduceFunction { private final IntValue value = new IntValue(); @Override - public void reduce(Iterator records, Collector out) - throws Exception { + public void reduce(Iterable records, Collector out) { Record element = null; int valCnt = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; valCnt++; } @@ -371,12 +369,11 @@ public static class MockDelayingReduceStub extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) { - while(records.hasNext()) { + public void reduce(Iterable records, Collector out) { + for (@SuppressWarnings("unused") Record r : records) { try { Thread.sleep(100); } catch (InterruptedException e) {} - records.next(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java index aabaa0370711a..0827c7ee33529 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.operators.chaining; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.functions.GenericCollectorMap; @@ -196,12 +195,12 @@ public static final class MockFailingCombineStub extends ReduceFunction { private int cnt = 0; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { if (++this.cnt >= 5) { throw new RuntimeException("Expected Test Exception"); } - while(records.hasNext()) { - out.collect(records.next()); + for (Record rec : records) { + out.collect(rec); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java index 8c8715e728750..f13adab51a459 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.drivers; import java.util.Arrays; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.functions.GenericGroupReduce; @@ -155,11 +153,10 @@ public void testAllReduceDriverMutable() { public static final class ConcatSumReducer extends GroupReduceFunction, Tuple2> { @Override - public void reduce(Iterator> values, Collector> out) { - Tuple2 current = values.next(); + public void reduce(Iterable> values, Collector> out) { + Tuple2 current = values.iterator().next(); - while (values.hasNext()) { - Tuple2 next = values.next(); + for (Tuple2 next : values) { next.f0 = current.f0 + next.f0; next.f1 = current.f1 + next.f1; current = next; @@ -172,11 +169,10 @@ public void reduce(Iterator> values, Collector, Tuple2> { @Override - public void reduce(Iterator> values, Collector> out) { - Tuple2 current = values.next(); + public void reduce(Iterable> values, Collector> out) { + Tuple2 current = values.iterator().next(); - while (values.hasNext()) { - Tuple2 next = values.next(); + for (Tuple2 next : values) { next.f0.append(current.f0); next.f1.setValue(current.f1.getValue() + next.f1.getValue()); current = next; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java index e5a01d60fe826..92c0edcfc9a14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java @@ -16,10 +16,8 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.drivers; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.functions.GenericGroupReduce; @@ -152,11 +150,10 @@ public void testAllReduceDriverMutable() { public static final class ConcatSumReducer extends GroupReduceFunction, Tuple2> { @Override - public void reduce(Iterator> values, Collector> out) { - Tuple2 current = values.next(); + public void reduce(Iterable> values, Collector> out) { + Tuple2 current = values.iterator().next(); - while (values.hasNext()) { - Tuple2 next = values.next(); + for (Tuple2 next : values) { next.f0 = current.f0 + next.f0; next.f1 = current.f1 + next.f1; current = next; @@ -169,11 +166,10 @@ public void reduce(Iterator> values, Collector, Tuple2> { @Override - public void reduce(Iterator> values, Collector> out) { - Tuple2 current = values.next(); + public void reduce(Iterable> values, Collector> out) { + Tuple2 current = values.iterator().next(); - while (values.hasNext()) { - Tuple2 next = values.next(); + for (Tuple2 next : values) { next.f0.append(current.f0); next.f1.setValue(current.f1.getValue() + next.f1.getValue()); current = next; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java index 5ddaa9fccfe21..8e3e1b4a40c18 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java @@ -284,11 +284,11 @@ public static class TestCountCombiner extends ReduceFunction { @Override - public void combine(Iterator values, Collector out) { + public void combine(Iterable values, Collector out) { Record rec = null; int cnt = 0; - while (values.hasNext()) { - rec = values.next(); + for (Record next : values) { + rec = next; cnt += rec.getField(1, IntValue.class).getValue(); } @@ -298,7 +298,7 @@ public void combine(Iterator values, Collector out) { } @Override - public void reduce(Iterator values, Collector out) {} + public void reduce(Iterable values, Collector out) {} @Override public void open(Configuration parameters) throws Exception { @@ -319,11 +319,11 @@ public static class TestCountCombiner2 extends ReduceFunction { public volatile boolean closed = false; @Override - public void combine(Iterator values, Collector out) { + public void combine(Iterable values, Collector out) { Record rec = null; int cnt = 0; - while (values.hasNext()) { - rec = values.next(); + for (Record next : values) { + rec = next; cnt += Integer.parseInt(rec.getField(1, TestData.Value.class).toString()); } @@ -331,7 +331,7 @@ public void combine(Iterator values, Collector out) { } @Override - public void reduce(Iterator values, Collector out) { + public void reduce(Iterable values, Collector out) { // yo, nothing, mon } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java index a66fb5dac7327..48a4b91317cdb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java @@ -116,8 +116,8 @@ public void testMerge() { final TestData.Key key = new TestData.Key(); while (iterator.next()) { - Iterator iter1 = iterator.getValues1(); - Iterator iter2 = iterator.getValues2(); + Iterator iter1 = iterator.getValues1().iterator(); + Iterator iter2 = iterator.getValues2().iterator(); TestData.Value v1 = null; TestData.Value v2 = null; diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala index 6c7e93b285dc2..82d71a5b5c403 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.functions -import java.util.{Iterator => JIterator} +import java.lang.{Iterable => JIterable} import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT} import org.apache.flink.api.scala.analysis.UDF2 @@ -53,9 +53,9 @@ abstract class CoGroupFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends } abstract class CoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Out] { - override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords) - val firstRightRecord = rightIterator.initialize(rightRecords) + override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) + val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) @@ -72,11 +72,11 @@ abstract class CoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGr } abstract class FlatCoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Iterator[Out]] { - override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords) + override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo) - val firstRightRecord = rightIterator.initialize(rightRecords) + val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) val output = apply(leftIterator, rightIterator) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala index 6ea8cbf103587..84765a5788795 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala @@ -19,10 +19,10 @@ package org.apache.flink.api.scala.functions -import java.util.{ Iterator => JIterator } - import scala.Iterator +import java.lang.{Iterable => JIterable} + import org.apache.flink.api.scala.analysis.{UDTSerializer, FieldSelector, UDT} import org.apache.flink.api.scala.analysis.UDF1 @@ -47,12 +47,12 @@ abstract class ReduceFunctionBase[In: UDT, Out: UDT] extends JReduceFunction wit abstract class ReduceFunction[In: UDT] extends ReduceFunctionBase[In, In] with Function2[In, In, In] { - override def combine(records: JIterator[Record], out: Collector[Record]) = { + override def combine(records: JIterable[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(records: JIterator[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records) + override def reduce(records: JIterable[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records.iterator()) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = reduceIterator.reduce(apply) @@ -63,8 +63,8 @@ abstract class ReduceFunction[In: UDT] extends ReduceFunctionBase[In, In] with F } abstract class GroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] { - override def reduce(records: JIterator[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records) + override def reduce(records: JIterable[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records.iterator()) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = apply(reduceIterator) @@ -75,8 +75,8 @@ abstract class GroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase } abstract class CombinableGroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] { - override def combine(records: JIterator[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records) + override def combine(records: JIterable[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records.iterator()) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = combine(reduceIterator) @@ -85,8 +85,8 @@ abstract class CombinableGroupReduceFunction[In: UDT, Out: UDT] extends ReduceFu out.collect(reduceRecord) } - override def reduce(records: JIterator[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records) + override def reduce(records: JIterable[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records.iterator()) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = reduce(reduceIterator) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala index 4c263073ecbf0..f6164966b3a3a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala @@ -16,13 +16,13 @@ * limitations under the License. */ - package org.apache.flink.api.scala.operators import language.experimental.macros import scala.reflect.macros.Context import java.util.{ Iterator => JIterator } +import java.lang.{ Iterable => JIterable } import org.apache.flink.types.Record import org.apache.flink.util.Collector @@ -106,11 +106,13 @@ object CoGroupMacros { implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice + new CoGroupFunctionBase[LeftIn, RightIn, Out] { - override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { + + override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords) - val firstRightRecord = rightIterator.initialize(rightRecords) + val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) + val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) @@ -177,10 +179,12 @@ object CoGroupMacros { implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice + new CoGroupFunctionBase[LeftIn, RightIn, Out] { - override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords) - val firstRightRecord = rightIterator.initialize(rightRecords) + + override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) + val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala index cf3a96c0eb850..1fa62bc1cc762 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala @@ -23,6 +23,7 @@ import language.experimental.macros import scala.language.reflectiveCalls import scala.reflect.macros.Context +import java.lang.{ Iterable => JIterable } import java.util.{ Iterator => JIterator } import org.apache.flink.api.scala.DataSet @@ -126,11 +127,12 @@ object ReduceMacros { implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice new ReduceFunctionBase[In, In] { - override def combine(records: JIterator[Record], out: Collector[Record]) = { + override def combine(records: JIterable[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(records: JIterator[Record], out: Collector[Record]) = { + override def reduce(recordsIterable: JIterable[Record], out: Collector[Record]) = { + val records: JIterator[Record] = recordsIterable.iterator() if (records.hasNext) { val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) @@ -191,7 +193,9 @@ object ReduceMacros { implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice new ReduceFunctionBase[In, Out] { - override def reduce(records: JIterator[Record], out: Collector[Record]) = { + override def reduce(recordsIterable: JIterable[Record], out: Collector[Record]) = { + val records: JIterator[Record] = recordsIterable.iterator() + if (records.hasNext) { val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) @@ -248,12 +252,12 @@ object ReduceMacros { implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice new ReduceFunctionBase[In, In] { - override def combine(records: JIterator[Record], out: Collector[Record]) = { + override def combine(records: JIterable[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(records: JIterator[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records) + override def reduce(records: JIterable[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records.iterator()) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = fun.splice.apply(reduceIterator) @@ -319,8 +323,9 @@ object ReduceMacros { this.countPosition = udf.getOutputLength - 1; } - override def reduce(records: JIterator[Record], result: Collector[Record]) : Unit = { + override def reduce(recordsIterable: JIterable[Record], result: Collector[Record]) : Unit = { + val records: JIterator[Record] = recordsIterable.iterator() var record : Record = null var counter: Int = 0 while (records.hasNext()) { @@ -334,7 +339,7 @@ object ReduceMacros { result.collect(record) } - override def combine(records: JIterator[Record], result: Collector[Record]) : Unit = { + override def combine(records: JIterable[Record], result: Collector[Record]) : Unit = { reduce(records, result) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index e76f86dd4f8cd..208c429844088 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -256,22 +255,23 @@ public void open(Configuration parameters) throws Exception { } @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { reduceCalls.add(1); reduceInternal(records, out); } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { combineCalls.add(1); reduceInternal(records, out); } - private void reduceInternal(Iterator records, Collector out) { + private void reduceInternal(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for (Record next : records) { + element = next; IntValue i = element.getField(1, IntValue.class); sum += i.getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java index bf6c37d3d8746..2495dd3b24342 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java @@ -21,7 +21,6 @@ import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.accumulators.IntCounter; @@ -124,11 +123,12 @@ public void open(Configuration parameters) throws Exception { } @Override - public void reduce(Iterator it, Collector out) { + public void reduce(Iterable records, Collector out) { // Compute the sum int sum = 0; - while (it.hasNext()) { - Integer value = Integer.parseInt(it.next().getField(0, StringValue.class).getValue()); + + for (Record r : records) { + Integer value = Integer.parseInt(r.getField(0, StringValue.class).getValue()); sum += value; testCounter.add(value); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java index e17c089a58cf6..7f9fcda525be7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.java.functions.GroupReduceFunction; @@ -72,15 +70,13 @@ public void open(Configuration parameters) { } @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable records, Collector out) { if (bcValue == null) { return; } final int x = bcValue; - while (records.hasNext()) { - int y = records.next(); - + for (Integer y : records) { if (y > x) { out.collect(y); return; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java index 37695b2d794e8..d35f886e30a0e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.io.BufferedReader; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.CoGroupFunction; @@ -94,17 +92,17 @@ public static final class MinIdAndUpdate extends CoGroupFunction implements Seri private final LongValue newComponentId = new LongValue(); @Override - public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { - if (!current.hasNext()) { + public void coGroup(Iterable candidates, Iterable current, Collector out) throws Exception { + if (!current.iterator().hasNext()) { throw new Exception("Error: Id not encountered before."); } - Record old = current.next(); + Record old = current.iterator().next(); long oldId = old.getField(1, LongValue.class).getValue(); long minimumComponentID = Long.MAX_VALUE; - while (candidates.hasNext()) { - long candidateComponentID = candidates.next().getField(1, LongValue.class).getValue(); + for (Record candidate : candidates) { + long candidateComponentID = candidate.getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java index 31540dc42a8c5..cb087bee88aeb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.java.functions.CoGroupFunction; @@ -116,17 +114,17 @@ public void flatMap(String value, Collector> out) throws Exce public static final class MinIdAndUpdate extends CoGroupFunction, Tuple2, Tuple2> { @Override - public void coGroup(Iterator> candidates, Iterator> current, Collector> out) { - if (!current.hasNext()) { + public void coGroup(Iterable> candidates, Iterable> current, Collector> out) { + if (!current.iterator().hasNext()) { throw new RuntimeException("Error: Id not encountered before."); } - Tuple2 old = current.next(); + Tuple2 old = current.iterator().next(); long minimumComponentID = Long.MAX_VALUE; - while (candidates.hasNext()) { - long candidateComponentID = candidates.next().f1; + for (Tuple2 candidate : candidates) { + long candidateComponentID = candidate.f1; if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java index be24662bb3e5a..0db1b81291008 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.java.functions.FlatMapFunction; @@ -171,8 +169,8 @@ public static final class RemoveDuplicatesReduce extends GroupReduceFunction values, Collector out) throws Exception { - out.collect(values.next()); + public void reduce(Iterable values, Collector out) { + out.collect(values.iterator().next()); } } @@ -208,15 +206,14 @@ public static final class MinimumReduce extends GroupReduceFunction final Tuple2 resultVertex = new Tuple2(); @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { - final Tuple2 first = values.next(); + final Tuple2 first = values.iterator().next(); final Long vertexId = first.f0; Long minimumCompId = first.f1; - while ( values.hasNext() ) { - Long candidateCompId = values.next().f1; + for ( Tuple2 value : values ) { + Long candidateCompId = value.f1; if ( candidateCompId < minimumCompId ) { minimumCompId = candidateCompId; } @@ -236,8 +233,8 @@ public static final class MinimumIdFilter extends FlatMapFunction @Override public void flatMap( Tuple2, Tuple2> vertexWithNewAndOldId, - Collector> out) throws Exception { - + Collector> out) + { if ( vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1 ) { out.collect(vertexWithNewAndOldId.f0); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java index cb2501945bfda..3eb4775c97dea 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -106,11 +104,11 @@ public static final class SumReducer extends ReduceFunction implements Serializa private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator it, Collector out) { + public void reduce(Iterable it, Collector out) { // Compute the sum int sum = 0; - while (it.hasNext()) { - sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; + for (Record r : it) { + sum += Integer.parseInt(r.getField(0, StringValue.class).getValue()) + 1; } out.collect(new Record(new StringValue(Integer.toString(sum)))); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java index 6a487c41d302a..641540c77f531 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -107,11 +105,12 @@ static final class SumReducer extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator it, Collector out) { + public void reduce(Iterable it, Collector out) { // Compute the sum int sum = 0; - while (it.hasNext()) { - sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; + + for (Record r : it) { + sum += Integer.parseInt(r.getField(0, StringValue.class).getValue()) + 1; } out.collect(new Record(new StringValue(Integer.toString(sum)))); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java index c81b32a0bfe62..b35b008448022 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.ReduceFunction; @@ -102,8 +100,8 @@ public static final class PickOneReducer extends ReduceFunction implements Seria private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator it, Collector out) { - out.collect(it.next()); + public void reduce(Iterable it, Collector out) { + out.collect(it.iterator().next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java index b62d85a1324cd..2a0831d78bbb0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -44,85 +43,81 @@ @RunWith(Parameterized.class) public class IterationWithChainingITCase extends RecordAPITestBase { - private static final String DATA_POINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; + private static final String DATA_POINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; - private String dataPath; - private String resultPath; + private String dataPath; + private String resultPath; - public IterationWithChainingITCase(Configuration config) { - super(config); + public IterationWithChainingITCase(Configuration config) { + super(config); setTaskManagerNumSlots(DOP); - } + } - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("data_points.txt", DATA_POINTS); - resultPath = getTempFilePath("result"); - } + @Override + protected void preSubmit() throws Exception { + dataPath = createTempFile("data_points.txt", DATA_POINTS); + resultPath = getTempFilePath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(DATA_POINTS, resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(DATA_POINTS, resultPath); + } + @Override + protected Plan getTestJob() { + Plan plan = getTestPlan(config.getInteger("ChainedMapperITCase#NoSubtasks", 1), dataPath, resultPath); + return plan; + } - @Override - protected Plan getTestJob() { - Plan plan = getTestPlan(config.getInteger("ChainedMapperITCase#NoSubtasks", 1), dataPath, resultPath); - return plan; - } + @Parameters + public static Collection getConfigurations() { + Configuration config1 = new Configuration(); + config1.setInteger("ChainedMapperITCase#NoSubtasks", DOP); + return toParameterList(config1); + } - @Parameters - public static Collection getConfigurations() { - Configuration config1 = new Configuration(); - config1.setInteger("ChainedMapperITCase#NoSubtasks", DOP); - return toParameterList(config1); - } + public static final class IdentityMapper extends MapFunction implements Serializable { - public static final class IdentityMapper extends MapFunction implements Serializable { + private static final long serialVersionUID = 1L; - private static final long serialVersionUID = 1L; + @Override + public void map(Record rec, Collector out) { + out.collect(rec); + } + } - @Override - public void map(Record rec, Collector out) { - out.collect(rec); - } - } + public static final class DummyReducer extends ReduceFunction implements Serializable { - public static final class DummyReducer extends ReduceFunction implements Serializable { + private static final long serialVersionUID = 1L; - private static final long serialVersionUID = 1L; + @Override + public void reduce(Iterable it, Collector out) { + for (Record r : it) { + out.collect(r); + } + } + } - @Override - public void reduce(Iterator it, Collector out) { - while (it.hasNext()) { - out.collect(it.next()); - } - } - } + static Plan getTestPlan(int numSubTasks, String input, String output) { - static Plan getTestPlan(int numSubTasks, String input, String output) { + FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input"); + initialInput.setDegreeOfParallelism(1); - FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input"); - initialInput.setDegreeOfParallelism(1); + BulkIteration iteration = new BulkIteration("Loop"); + iteration.setInput(initialInput); + iteration.setMaximumNumberOfIterations(2); - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(2); + ReduceOperator dummyReduce = ReduceOperator.builder(new DummyReducer(), IntValue.class, 0).input(iteration.getPartialSolution()) + .name("Reduce something").build(); - ReduceOperator dummyReduce = ReduceOperator.builder(new DummyReducer(), IntValue.class, 0) - .input(iteration.getPartialSolution()) - .name("Reduce something") - .build(); + MapOperator dummyMap = MapOperator.builder(new IdentityMapper()).input(dummyReduce).build(); + iteration.setNextPartialSolution(dummyMap); + FileDataSink finalResult = new FileDataSink(new PointOutFormat(), output, iteration, "Output"); - MapOperator dummyMap = MapOperator.builder(new IdentityMapper()).input(dummyReduce).build(); - iteration.setNextPartialSolution(dummyMap); - - FileDataSink finalResult = new FileDataSink(new PointOutFormat(), output, iteration, "Output"); - - Plan plan = new Plan(finalResult, "Iteration with chained map test"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } + Plan plan = new Plan(finalResult, "Iteration with chained map test"); + plan.setDefaultParallelism(numSubTasks); + return plan; + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java index aa83e9b2dc0d0..0fd3d2c235a18 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.iterative; import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -111,9 +109,9 @@ static final class DummyReducer extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator it, Collector out) { - while (it.hasNext()) { - out.collect(it.next()); + public void reduce(Iterable it, Collector out) { + for (Record r : it) { + out.collect(r); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java index 00428d5e64982..4146475ee9e50 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative.aggregators; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.aggregators.LongSumAggregator; @@ -39,11 +37,8 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.IterativeDataSet; - /** - * - * Connected Components test case that uses a parametrizable aggregator - * + * Connected Components test case that uses a parameterizable aggregator */ public class ConnectedComponentsWithParametrizableAggregatorITCase extends JavaProgramTestBase { @@ -168,15 +163,14 @@ public static final class MinimumReduce extends GroupReduceFunction final Tuple2 resultVertex = new Tuple2(); @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { - final Tuple2 first = values.next(); + final Tuple2 first = values.iterator().next(); final Long vertexId = first.f0; Long minimumCompId = first.f1; - while (values.hasNext()) { - Long candidateCompId = values.next().f1; + for (Tuple2 value: values) { + Long candidateCompId = value.f1; if (candidateCompId < minimumCompId) { minimumCompId = candidateCompId; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java index fa1676ff5d7b1..f8396618c07dd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.iterative.aggregators; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; @@ -160,15 +158,14 @@ public static final class MinimumReduce extends GroupReduceFunction final Tuple2 resultVertex = new Tuple2(); @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { - final Tuple2 first = values.next(); + final Tuple2 first = values.iterator().next(); final Long vertexId = first.f0; Long minimumCompId = first.f1; - while (values.hasNext()) { - Long candidateCompId = values.next().f1; + for (Tuple2 value : values) { + Long candidateCompId = value.f1; if (candidateCompId < minimumCompId) { minimumCompId = candidateCompId; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java index 448b7bd9e7b44..0e2c7138cce7c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java @@ -19,7 +19,6 @@ package org.apache.flink.test.iterative.nephele; import java.util.Collection; -import java.util.Iterator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; @@ -278,9 +277,9 @@ public static final class DummyReducer extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator it, Collector out) { - while (it.hasNext()) { - out.collect(it.next()); + public void reduce(Iterable it, Collector out) { + for (Record r :it) { + out.collect(r); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index 0475a4f6d91c5..e4dbcc871dc4c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -16,10 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.iterative.nephele.customdanglingpagerank; -import java.util.Iterator; import java.util.Set; import org.apache.flink.api.common.functions.AbstractFunction; @@ -83,20 +81,20 @@ public void open(Configuration parameters) throws Exception { } @Override - public void coGroup(Iterator currentPageRankIterator, Iterator partialRanks, + public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, Collector collector) { - if (!currentPageRankIterator.hasNext()) { - long missingVertex = partialRanks.next().getVertexID(); + if (!currentPageRankIterator.iterator().hasNext()) { + long missingVertex = partialRanks.iterator().next().getVertexID(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - VertexWithRankAndDangling currentPageRank = currentPageRankIterator.next(); + VertexWithRankAndDangling currentPageRank = currentPageRankIterator.iterator().next(); long edges = 0; double summedRank = 0; - while (partialRanks.hasNext()) { - summedRank += partialRanks.next().getRank(); + for (VertexWithRank pr :partialRanks) { + summedRank += pr.getRank(); edges++; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java index 8af92474b8cec..b9e2391c6985f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java @@ -18,15 +18,12 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank; -import java.util.Iterator; - import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank; import org.apache.flink.util.Collector; - public class CustomRankCombiner extends AbstractFunction implements GenericGroupReduce, GenericCombine { @@ -35,18 +32,18 @@ public class CustomRankCombiner extends AbstractFunction implements GenericGroup private final VertexWithRank accumulator = new VertexWithRank(); @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { throw new UnsupportedOperationException(); } @Override - public void combine(Iterator records, Collector out) throws Exception { - VertexWithRank next = records.next(); + public void combine(Iterable records, Collector out) throws Exception { + VertexWithRank next = records.iterator().next(); this.accumulator.setVertexID(next.getVertexID()); double rank = next.getRank(); - while (records.hasNext()) { - rank += records.next().getRank(); + for (VertexWithRank r : records) { + rank += r.getRank(); } this.accumulator.setRank(rank); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java index ab3dea91aeb5e..508462c8437c0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java @@ -16,10 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.iterative.nephele.danglingpagerank; -import java.util.Iterator; import java.util.Set; import org.apache.flink.api.java.record.functions.CoGroupFunction; @@ -90,19 +88,19 @@ public void open(Configuration parameters) { } @Override - public void coGroup(Iterator currentPageRankIterator, Iterator partialRanks, Collector collector) { + public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, Collector collector) { - if (!currentPageRankIterator.hasNext()) { - long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue(); + if (!currentPageRankIterator.iterator().hasNext()) { + long missingVertex = partialRanks.iterator().next().getField(0, LongValue.class).getValue(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - Record currentPageRank = currentPageRankIterator.next(); + Record currentPageRank = currentPageRankIterator.iterator().next(); long edges = 0; double summedRank = 0; - while (partialRanks.hasNext()) { - summedRank += partialRanks.next().getField(1, doubleInstance).getValue(); + for (Record pr : partialRanks) { + summedRank += pr.getField(1, doubleInstance).getValue(); edges++; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java index b914c1c5c1007..dd9b289d0cc16 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java @@ -21,7 +21,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedList; import org.apache.flink.api.java.functions.CoGroupFunction; @@ -307,21 +306,19 @@ public static class Tuple5CoGroup extends CoGroupFunction> first, - Iterator> second, - Collector> out) throws Exception { - + Iterable> first, + Iterable> second, + Collector> out) + { int sum = 0; int id = 0; - while(first.hasNext()) { - Tuple5 element = first.next(); + for ( Tuple5 element : first ) { sum += element.f2; id = element.f0; } - while(second.hasNext()) { - Tuple5 element = second.next(); + for ( Tuple5 element : second ) { sum += element.f2; id = element.f0; } @@ -335,27 +332,22 @@ public static class CustomTypeCoGroup extends CoGroupFunction first, - Iterator second, Collector out) - throws Exception { + public void coGroup(Iterable first, Iterable second, Collector out) { CustomType o = new CustomType(0,0,"test"); - while(first.hasNext()) { - CustomType element = first.next(); + for ( CustomType element : first ) { o.myInt = element.myInt; o.myLong += element.myLong; } - while(second.hasNext()) { - CustomType element = second.next(); + for ( CustomType element : second ) { o.myInt = element.myInt; o.myLong += element.myLong; } out.collect(o); } - } public static class MixedCoGroup extends CoGroupFunction, CustomType, Tuple3> { @@ -364,21 +356,19 @@ public static class MixedCoGroup extends CoGroupFunction> first, - Iterator second, + Iterable> first, + Iterable second, Collector> out) throws Exception { long sum = 0; int id = 0; - while(first.hasNext()) { - Tuple5 element = first.next(); + for ( Tuple5 element : first ) { sum += element.f0; id = element.f2; } - while(second.hasNext()) { - CustomType element = second.next(); + for (CustomType element : second) { id = element.myInt; sum += element.myLong; } @@ -393,20 +383,18 @@ public static class MixedCoGroup2 extends CoGroupFunction first, - Iterator> second, - Collector out) throws Exception { - + public void coGroup(Iterable first, + Iterable> second, + Collector out) + { CustomType o = new CustomType(0,0,"test"); - while(first.hasNext()) { - CustomType element = first.next(); + for (CustomType element : first) { o.myInt = element.myInt; o.myLong += element.myLong; } - while(second.hasNext()) { - Tuple5 element = second.next(); + for (Tuple5 element : second) { o.myInt = element.f2; o.myLong += element.f0; } @@ -422,14 +410,14 @@ public static class Tuple3ReturnLeft extends CoGroupFunction> first, - Iterator> second, - Collector> out) throws Exception { - - while(first.hasNext()) { - Tuple3 element = first.next(); - if(element.f0 < 6) + public void coGroup(Iterable> first, + Iterable> second, + Collector> out) + { + for (Tuple3 element : first) { + if(element.f0 < 6) { out.collect(element); + } } } } @@ -440,20 +428,16 @@ public static class Tuple5ReturnRight extends CoGroupFunction> first, - Iterator> second, + Iterable> first, + Iterable> second, Collector> out) - throws Exception { - - while(second.hasNext()) { - Tuple5 element = second.next(); - if(element.f0 < 4) + { + for (Tuple5 element : second) { + if(element.f0 < 4) { out.collect(element); + } } - } - - } public static class Tuple5CoGroupBC extends CoGroupFunction, Tuple5, Tuple3> { @@ -476,21 +460,19 @@ public void open(Configuration config) { @Override public void coGroup( - Iterator> first, - Iterator> second, - Collector> out) throws Exception { - + Iterable> first, + Iterable> second, + Collector> out) + { int sum = 0; int id = 0; - while(first.hasNext()) { - Tuple5 element = first.next(); + for (Tuple5 element : first) { sum += element.f2; id = element.f0; } - while(second.hasNext()) { - Tuple5 element = second.next(); + for (Tuple5 element : second) { sum += element.f2; id = element.f0; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index 6556b5ed89c27..d34f205a9d6eb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -21,7 +21,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedList; import org.apache.flink.api.common.operators.Order; @@ -417,16 +416,13 @@ public Integer getKey(CustomType in) { public static class Tuple3GroupReduce extends GroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; - @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { int i = 0; long l = 0l; - while(values.hasNext()) { - Tuple3 t = values.next(); + for (Tuple3 t : values) { i += t.f0; l = t.f1; } @@ -441,24 +437,21 @@ public static class Tuple3SortedGroupReduce extends GroupReduceFunction> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { - Tuple3 t = values.next(); + Tuple3 t = values.iterator().next(); int sum = t.f0; long key = t.f1; String concat = t.f2; - while(values.hasNext()) { - t = values.next(); + for (Tuple3 next : values) { - sum += t.f0; - concat += "-"+t.f2; + sum += next.f0; + concat += "-" + next.f2; } out.collect(new Tuple3(sum, key, concat)); - } } @@ -467,16 +460,14 @@ public static class Tuple5GroupReduce extends GroupReduceFunction> values, + Iterable> values, Collector> out) - throws Exception { - + { int i = 0; long l = 0l; long l2 = 0l; - while(values.hasNext()) { - Tuple5 t = values.next(); + for ( Tuple5 t : values ) { i = t.f0; l += t.f1; l2 = t.f4; @@ -491,19 +482,17 @@ public static class CustomTypeGroupReduce extends GroupReduceFunction values, - Collector out) throws Exception { + public void reduce(Iterable values, Collector out) { CustomType o = new CustomType(); - CustomType c = values.next(); + CustomType c = values.iterator().next(); o.myString = "Hello!"; o.myInt = c.myInt; o.myLong = c.myLong; - while(values.hasNext()) { - c = values.next(); - o.myLong += c.myLong; + for ( CustomType next : values) { + o.myLong += next.myLong; } @@ -516,11 +505,9 @@ public static class InputReturningTuple3GroupReduce extends GroupReduceFunction< private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { if(t.f0 < 4) { t.f2 = "Hi!"; @@ -538,14 +525,12 @@ public static class AllAddingTuple3GroupReduce extends GroupReduceFunction> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { int i = 0; long l = 0l; - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { i += t.f0; l += t.f1; } @@ -558,21 +543,19 @@ public static class AllAddingCustomTypeGroupReduce extends GroupReduceFunction values, - Collector out) throws Exception { + public void reduce(Iterable values, Collector out) { CustomType o = new CustomType(); - CustomType c = values.next(); + CustomType c = values.iterator().next(); o.myString = "Hello!"; o.myInt = c.myInt; o.myLong = c.myLong; - while(values.hasNext()) { - c = values.next(); - o.myInt += c.myInt; - o.myLong += c.myLong; + for (CustomType next : values) { + o.myInt += next.myInt; + o.myLong += next.myLong; } out.collect(o); @@ -596,14 +579,12 @@ public void open(Configuration config) { } @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { int i = 0; long l = 0l; - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { i += t.f0; l = t.f1; } @@ -618,12 +599,11 @@ public static class Tuple3GroupReduceWithCombine extends GroupReduceFunction> values, Collector> out) throws Exception { + public void combine(Iterable> values, Collector> out) { Tuple3 o = new Tuple3(0, 0l, ""); - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { o.f0 += t.f0; o.f1 = t.f1; o.f2 = "test"+o.f1; @@ -633,14 +613,12 @@ public void combine(Iterator> values, Collector> values, - Collector> out) throws Exception { + public void reduce(Iterable> values, Collector> out) { int i = 0; String s = ""; - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { i += t.f0; s = t.f2; } @@ -655,12 +633,11 @@ public static class Tuple3AllGroupReduceWithCombine extends GroupReduceFunction< private static final long serialVersionUID = 1L; @Override - public void combine(Iterator> values, Collector> out) { + public void combine(Iterable> values, Collector> out) { Tuple3 o = new Tuple3(0, 0l, ""); - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { o.f0 += t.f0; o.f1 += t.f1; o.f2 += "test"; @@ -670,13 +647,12 @@ public void combine(Iterator> values, Collector> values, Collector> out) { + public void reduce(Iterable> values, Collector> out) { int i = 0; String s = ""; - while(values.hasNext()) { - Tuple3 t = values.next(); + for ( Tuple3 t : values ) { i += t.f0 + t.f1; s += t.f2; } @@ -691,12 +667,11 @@ public static class CustomTypeGroupReduceWithCombine extends GroupReduceFunction private static final long serialVersionUID = 1L; @Override - public void combine(Iterator values, Collector out) throws Exception { + public void combine(Iterable values, Collector out) throws Exception { CustomType o = new CustomType(); - while(values.hasNext()) { - CustomType c = values.next(); + for ( CustomType c : values ) { o.myInt = c.myInt; o.myLong += c.myLong; o.myString = "test"+c.myInt; @@ -706,13 +681,11 @@ public void combine(Iterator values, Collector out) thro } @Override - public void reduce(Iterator values, - Collector out) throws Exception { + public void reduce(Iterable values, Collector out) { CustomType o = new CustomType(0, 0, ""); - while(values.hasNext()) { - CustomType c = values.next(); + for ( CustomType c : values) { o.myInt = c.myInt; o.myLong += c.myLong; o.myString = c.myString; @@ -727,6 +700,5 @@ public static final class IdentityMapper extends MapFunction { @Override public T map(T value) { return value; } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java index 5a895d3423373..32724561c8e07 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java @@ -16,18 +16,14 @@ * limitations under the License. */ - package org.apache.flink.test.operators; import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedList; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.record.io.DelimitedInputFormat; @@ -46,13 +42,9 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -/** - */ @RunWith(Parameterized.class) public class CoGroupITCase extends RecordAPITestBase { - private static final Log LOG = LogFactory.getLog(CoGroupITCase.class); - String leftInPath = null; String rightInPath = null; String resultPath = null; @@ -89,8 +81,6 @@ public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) target.setField(0, keyString); target.setField(1, valueString); - LOG.debug("Read in: [" + keyString.getValue() + "," + valueString.getValue() + "]"); - return target; } @@ -112,9 +102,6 @@ public void writeRecord(Record record) throws IOException { this.buffer.append('\n'); byte[] bytes = this.buffer.toString().getBytes(); - - LOG.debug("Writing out: [" + keyString.toString() + "," + valueInteger.getValue() + "]"); - this.stream.write(bytes); } } @@ -124,33 +111,28 @@ public static class TestCoGrouper extends CoGroupFunction implements Serializabl private StringValue keyString = new StringValue(); private StringValue valueString = new StringValue(); - private Record record = new Record(); @Override - public void coGroup(Iterator records1, Iterator records2, Collector out) { - // TODO Auto-generated method stub + public void coGroup(Iterable records1, Iterable records2, Collector out) { + Record record = null; int sum = 0; - LOG.debug("Start iterating over input1"); - while (records1.hasNext()) { - record = records1.next(); + + for (Record next : records1) { + record = next; keyString = record.getField(0, keyString); valueString = record.getField(1, valueString); sum += Integer.parseInt(valueString.getValue()); - - LOG.debug("Processed: [" + keyString.getValue() + "," + valueString.getValue() + "]"); } - LOG.debug("Start iterating over input2"); - while (records2.hasNext()) { - record = records2.next(); + + + for (Record next : records2) { + record = next; keyString = record.getField(0, keyString); valueString = record.getField(1, valueString); sum -= Integer.parseInt(valueString.getValue()); - - LOG.debug("Processed: [" + keyString.getValue() + "," + valueString.getValue() + "]"); } record.setField(1, new IntValue(sum)); - LOG.debug("Finished"); out.collect(record); } @@ -197,9 +179,9 @@ public static Collection getConfigurations() throws FileNotFoundExcept LinkedList tConfigs = new LinkedList(); - String[] localStrategies = { PactCompiler.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE}; + String[] localStrategies = { PactCompiler.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE }; - String[] shipStrategies = { PactCompiler.HINT_SHIP_STRATEGY_REPARTITION_HASH, }; + String[] shipStrategies = { PactCompiler.HINT_SHIP_STRATEGY_REPARTITION_HASH }; for (String localStrategy : localStrategies) { for (String shipStrategy : shipStrategies) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java index b06cf474a6474..c60e647cf81ff 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.operators; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.io.DelimitedInputFormat; @@ -48,13 +45,10 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedList; @RunWith(Parameterized.class) public class ReduceITCase extends RecordAPITestBase { - - private static final Log LOG = LogFactory.getLog(ReduceITCase.class); String inPath = null; String resultPath = null; @@ -83,17 +77,14 @@ public static class TestReducer extends ReduceFunction implements Serializable { private StringValue combineValue = new StringValue(); @Override - public void combine(Iterator records, Collector out) throws Exception { - + public void combine(Iterable records, Collector out) { + Record record = null; int sum = 0; - Record record = new Record(); - while (records.hasNext()) { - record = records.next(); + + for (Record next : records) { + record = next; combineValue = record.getField(1, combineValue); sum += Integer.parseInt(combineValue.toString()); - - LOG.debug("Processed: [" + record.getField(0, StringValue.class).toString() + - "," + combineValue.toString() + "]"); } combineValue.setValue(sum + ""); record.setField(1, combineValue); @@ -101,17 +92,14 @@ record = records.next(); } @Override - public void reduce(Iterator records, Collector out) throws Exception { - + public void reduce(Iterable records, Collector out) { + Record record = null; int sum = 0; - Record record = new Record(); - while (records.hasNext()) { - record = records.next(); + + for (Record next : records) { + record = next; reduceValue = record.getField(1, reduceValue); sum += Integer.parseInt(reduceValue.toString()); - - LOG.debug("Processed: [" + record.getField(0, StringValue.class).toString() + - "," + reduceValue.toString() + "]"); } record.setField(1, new IntValue(sum)); out.collect(record); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java index 60b75121c58c1..057212c726016 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobTests; import java.io.Serializable; import java.util.Collection; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.Order; @@ -119,11 +117,11 @@ public static final class CheckingReducer extends ReduceFunction implements Seri private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { - int lastValue = records.next().getField(1, IntValue.class).getValue(); + public void reduce(Iterable records, Collector out) throws Exception { + int lastValue = records.iterator().next().getField(1, IntValue.class).getValue(); - while (records.hasNext()) { - int nextValue = records.next().getField(1, IntValue.class).getValue(); + for (Record r : records) { + int nextValue = r.getField(1, IntValue.class).getValue(); if (nextValue < lastValue) { throw new Exception("Group Order is violated!"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java index 6ab9c3cad3149..cf2e7d1a18155 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; import java.util.Arrays; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -84,14 +82,13 @@ public static final class CountEdges extends ReduceFunction implements Serializa private int[] vals = new int[1024]; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) { int[] vals = this.vals; int len = 0; int key = -1; // collect all values - while (records.hasNext()) { - final Record rec = records.next(); + for (Record rec : records) { final int id = rec.getField(1, IntValue.class).getValue(); if (key == -1) { key = rec.getField(0, IntValue.class).getValue(); @@ -156,13 +153,13 @@ public static final class JoinCountsAndUniquify extends ReduceFunction implement private final IntValue count2 = new IntValue(); @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) { Record rec = null; int c1 = 0, c2 = 0; int numValues = 0; - while (records.hasNext()) { - rec = records.next(); + for (Record next : records) { + rec = next; final int f1 = rec.getField(2, IntValue.class).getValue(); final int f2 = rec.getField(3, IntValue.class).getValue(); c1 += f1; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java index fe51914b34a0e..831173cb4e540 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -57,17 +55,18 @@ public static final class MinIdAndUpdate extends CoGroupFunction implements Seri private final LongValue newComponentId = new LongValue(); @Override - public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { - if (!current.hasNext()) { + public void coGroup(Iterable candidates, Iterable current, Collector out) throws Exception { + if (!current.iterator().hasNext()) { throw new Exception("Error: Id not encountered before."); } - Record old = current.next(); + + Record old = current.iterator().next(); long oldId = old.getField(1, LongValue.class).getValue(); long minimumComponentID = Long.MAX_VALUE; - while (candidates.hasNext()) { - long candidateComponentID = candidates.next().getField(1, LongValue.class).getValue(); + for (Record candidate : candidates) { + long candidateComponentID = candidate.getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java index e5e552d4c7d56..e8a9f0b2c9a18 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; -import java.util.Iterator; - import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; import org.apache.flink.api.common.ProgramDescription; @@ -74,14 +71,14 @@ public static final class UpdateRankReduceDelta extends ReduceFunction { private final DoubleValue newRank = new DoubleValue(); @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable records, Collector out) { double rankSum = 0.0; double rank; Record rec = null; - while (records.hasNext()) { - rec = records.next(); + for (Record next : records) { + rec = next; rank = rec.getField(1, DoubleValue.class).getValue(); rankSum += rank; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java index 8362c29da440d..98bf5c7d91e21 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -90,12 +88,10 @@ public static final class BuildTriads extends ReduceFunction implements Serializ private int[] edgeCache = new int[1024]; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { int len = 0; - Record rec = null; - while (records.hasNext()) { - rec = records.next(); + for (Record rec : records) { final int e1 = rec.getField(1, IntValue.class).getValue(); for (int i = 0; i < len; i++) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java index 5873581ab628c..9c7707efc1dfa 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; import java.util.ArrayList; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -175,9 +173,9 @@ public BuildTriads() { } @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { // read the first edge - final Record rec = records.next(); + final Record rec = records.iterator().next(); // read the matching vertex rec.getFieldInto(0, this.matchVertex); // read the non-matching vertex and add it to the list @@ -188,10 +186,7 @@ public void reduce(Iterator records, Collector out) throws Excep int numEdges = 1; // while there are more edges - while (records.hasNext()) { - - // read the next edge - final Record next = records.next(); + for (Record next : records) { final StringValue myVertex; // obtain an object to store the non-matching vertex diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java index c57cf6aab5802..578ef54edbdd7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java @@ -16,14 +16,12 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.Set; import java.util.StringTokenizer; @@ -286,32 +284,32 @@ public static class FindShortestPath extends CoGroupFunction implements Serializ private final IntValue minLength = new IntValue(); @Override - public void coGroup(Iterator inputRecords, Iterator concatRecords, Collector out) { + public void coGroup(Iterable inputRecords, Iterable concatRecords, Collector out) { // init minimum length and minimum path - Record pathRec = null; + Record initial = null; StringValue path = null; - if(inputRecords.hasNext()) { + + if (inputRecords.iterator().hasNext()) { // path is in input paths - pathRec = inputRecords.next(); + initial = inputRecords.iterator().next(); } else { // path must be in concat paths - pathRec = concatRecords.next(); + initial = concatRecords.iterator().next(); } // get from node (common for all paths) - StringValue fromNode = pathRec.getField(0, StringValue.class); + StringValue fromNode = initial.getField(0, StringValue.class); // get to node (common for all paths) - StringValue toNode = pathRec.getField(1, StringValue.class); + StringValue toNode = initial.getField(1, StringValue.class); // get length of path - minLength.setValue(pathRec.getField(2, IntValue.class).getValue()); + minLength.setValue(initial.getField(2, IntValue.class).getValue()); // store path and hop count - path = new StringValue(pathRec.getField(4, StringValue.class)); + path = new StringValue(initial.getField(4, StringValue.class)); shortestPaths.add(path); - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); + hopCnts.put(path, new IntValue(initial.getField(3, IntValue.class).getValue())); // find shortest path of all input paths - while (inputRecords.hasNext()) { - pathRec = inputRecords.next(); + for (Record pathRec : inputRecords) { IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { @@ -320,7 +318,8 @@ public void coGroup(Iterator inputRecords, Iterator concatRecord if(shortestPaths.add(path)) { hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } - } else if (length.getValue() < minLength.getValue()) { + } + else if (length.getValue() < minLength.getValue()) { // path has minimum length minLength.setValue(length.getValue()); // clear lists @@ -334,8 +333,7 @@ public void coGroup(Iterator inputRecords, Iterator concatRecord } // find shortest path of all input and concatenated paths - while (concatRecords.hasNext()) { - pathRec = concatRecords.next(); + for (Record pathRec : concatRecords) { IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { @@ -367,7 +365,7 @@ public void coGroup(Iterator inputRecords, Iterator concatRecord outputRecord.setField(4, shortestPath); out.collect(outputRecord); } - + hopCnts.clear(); shortestPaths.clear(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java index 68b89f3981b5b..cab33d9c03cfa 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -90,12 +88,12 @@ public static final class AggregatingReduce extends ReduceFunction implements Se private final DoubleValue sum = new DoubleValue(); @Override - public void reduce(Iterator pageWithPartialRank, Collector out) throws Exception { + public void reduce(Iterable pageWithPartialRank, Collector out) throws Exception { Record rec = null; double rankSum = 0.0; - while (pageWithPartialRank.hasNext()) { - rec = pageWithPartialRank.next(); + for (Record next : pageWithPartialRank) { + rec = next; rankSum += rec.getField(1, DoubleValue.class).getValue(); } sum.setValue(rankSum); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java index 52f14c6123df1..e4e8cc1f7128a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -91,15 +89,15 @@ public static final class MinimumComponentIDReduce extends ReduceFunction implem private final LongValue minComponentId = new LongValue(); @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable records, Collector out) { - final Record first = records.next(); + final Record first = records.iterator().next(); final long vertexID = first.getField(0, LongValue.class).getValue(); long minimumComponentID = first.getField(1, LongValue.class).getValue(); - while (records.hasNext()) { - long candidateComponentID = records.next().getField(1, LongValue.class).getValue(); + for (Record r : records) { + long candidateComponentID = r.getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java index 6efa2bf558d8f..f5a70efc85b2c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.graph.pageRankUtil; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; @@ -91,20 +89,20 @@ public void open(Configuration parameters) throws Exception { } @Override - public void coGroup(Iterator currentPageRankIterator, Iterator partialRanks, + public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, Collector collector) { - if (!currentPageRankIterator.hasNext()) { - long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue(); + if (!currentPageRankIterator.iterator().hasNext()) { + long missingVertex = partialRanks.iterator().next().getField(0, LongValue.class).getValue(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - Record currentPageRank = currentPageRankIterator.next(); + Record currentPageRank = currentPageRankIterator.iterator().next(); long edges = 0; double summedRank = 0; - while (partialRanks.hasNext()) { - summedRank += partialRanks.next().getField(1, doubleInstance).getValue(); + for (Record r : partialRanks) { + summedRank += r.getField(1, doubleInstance).getValue(); edges++; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java index 66c8aaed9d5bb..80da9590fe340 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java @@ -16,14 +16,12 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.kmeans; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.Plan; @@ -255,7 +253,7 @@ public static final class RecomputeClusterCenter extends ReduceFunction { * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterator points, Collector out) { + public void reduce(Iterable points, Collector out) { Record sum = sumPointsAndCount(points); sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); out.collect(sum); @@ -265,18 +263,18 @@ public void reduce(Iterator points, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterator points, Collector out) { + public void combine(Iterable points, Collector out) { out.collect(sumPointsAndCount(points)); } - private final Record sumPointsAndCount(Iterator dataPoints) { + private final Record sumPointsAndCount(Iterable dataPoints) { Record next = null; p.clear(); int count = 0; // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); + for (Record n : dataPoints) { + next = n; p.add(next.getField(1, Point.class)); count += next.getField(2, IntValue.class).getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java index 24539861ccdef..d1fd0db64c38d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java @@ -16,14 +16,12 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.kmeans; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.Plan; @@ -236,7 +234,7 @@ public static final class RecomputeClusterCenter extends ReduceFunction { * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterator points, Collector out) { + public void reduce(Iterable points, Collector out) { Record sum = sumPointsAndCount(points); sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); out.collect(sum); @@ -246,18 +244,18 @@ public void reduce(Iterator points, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterator points, Collector out) { + public void combine(Iterable points, Collector out) { out.collect(sumPointsAndCount(points)); } - private final Record sumPointsAndCount(Iterator dataPoints) { + private final Record sumPointsAndCount(Iterable dataPoints) { Record next = null; p.clear(); int count = 0; // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); + for (Record n : dataPoints) { + next = n; p.add(next.getField(1, Point.class)); count += next.getField(2, IntValue.class).getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java index c4b974b21841b..3fe8bfe57043f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java @@ -19,7 +19,6 @@ package org.apache.flink.test.recordJobs.kmeans.udfs; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; @@ -54,14 +53,12 @@ public class FindNearestCenter extends ReduceFunction implements Serializable { * 2: constant(1) (to enable combinable average computation in the following reducer) */ @Override - public void reduce(Iterator pointsWithDistance, Collector out) { + public void reduce(Iterable pointsWithDistance, Collector out) { double nearestDistance = Double.MAX_VALUE; int nearestClusterId = 0; // check all cluster centers - while (pointsWithDistance.hasNext()) { - Record res = pointsWithDistance.next(); - + for ( Record res : pointsWithDistance) { double distance = res.getField(3, DoubleValue.class).getValue(); // compare distances @@ -92,12 +89,11 @@ public void reduce(Iterator pointsWithDistance, Collector out) { * cluster centers. */ @Override - public void combine(Iterator pointsWithDistance, Collector out) { + public void combine(Iterable pointsWithDistance, Collector out) { double nearestDistance = Double.MAX_VALUE; // check all cluster centers - while (pointsWithDistance.hasNext()) { - Record res = pointsWithDistance.next(); + for (Record res : pointsWithDistance) { double distance = res.getField(3, DoubleValue.class).getValue(); // compare distances diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java index 4cc61f50438ba..7a137536bc7c3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java @@ -19,7 +19,6 @@ package org.apache.flink.test.recordJobs.kmeans.udfs; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; @@ -48,7 +47,7 @@ public class RecomputeClusterCenter extends ReduceFunction implements Serializab * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterator dataPoints, Collector out) { + public void reduce(Iterable dataPoints, Collector out) { Record next = null; // initialize coordinate vector sum and count @@ -57,8 +56,8 @@ public void reduce(Iterator dataPoints, Collector out) { int count = 0; // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); + for (Record n : dataPoints) { + next = n; // get the coordinates and the count from the record double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); @@ -94,7 +93,7 @@ public void reduce(Iterator dataPoints, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterator dataPoints, Collector out) { + public void combine(Iterable dataPoints, Collector out) { Record next = null; @@ -104,8 +103,8 @@ public void combine(Iterator dataPoints, Collector out) { int count = 0; // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); + for (Record n : dataPoints) { + next = n; // get the coordinates and the count from the record double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java index 78e2c73415077..573c87642e635 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; -import java.util.Iterator; - import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; import org.apache.flink.api.java.record.functions.JoinFunction; @@ -57,9 +54,9 @@ public static class DummyReduce extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator values, Collector out) { - while (values.hasNext()) { - out.collect(values.next()); + public void reduce(Iterable values, Collector out) { + for (Record r : values) { + out.collect(r); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java index 378c2b67cc62a..e2cf6936b6413 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java @@ -16,13 +16,11 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; import java.io.IOException; import java.text.DecimalFormat; import java.text.DecimalFormatSymbols; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -200,17 +198,16 @@ public void join(Record colRecord, Record nation, Collector out) throws } @ReduceOperator.Combinable - public static class Sum extends ReduceFunction - { + public static class Sum extends ReduceFunction { + private final DoubleValue d = new DoubleValue(); @Override - public void reduce(Iterator records, Collector out) throws Exception - { + public void reduce(Iterable records, Collector out) { Record record = null; double sum = 0; - while (records.hasNext()) { - record = records.next(); + for (Record next : records) { + record = next; sum += record.getField(2, DoubleValue.class).getValue(); } @@ -220,7 +217,7 @@ record = records.next(); } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) { reduce(records,out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java index 15640c0b57eb7..6da83be955066 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -175,12 +173,12 @@ public static class AggLiO extends ReduceFunction implements Serializable { * 2:SUM(EXTENDEDPRICE) */ @Override - public void reduce(Iterator values, Collector out) { + public void reduce(Iterable values, Collector out) { Record rec = null; double partExtendedPriceSum = 0; - while (values.hasNext()) { - rec = values.next(); + for (Record next : values) { + rec = next; partExtendedPriceSum += rec.getField(2, DoubleValue.class).getValue(); } @@ -193,7 +191,7 @@ public void reduce(Iterator values, Collector out) { * Creates partial sums on the price attribute for each data batch. */ @Override - public void combine(Iterator values, Collector out) { + public void combine(Iterable values, Collector out) { reduce(values, out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java index 0db22cb2b1a59..c83feeb6f520e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; import java.text.ParseException; @@ -24,7 +23,6 @@ import java.util.Calendar; import java.util.Date; import java.util.GregorianCalendar; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -180,17 +178,16 @@ public void join(Record order, Record line, Collector out) public static class CountAgg extends ReduceFunction { @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { long count = 0; Record rec = null; - while(records.hasNext()) { - rec = records.next(); - count++; + for (Record next : records) { + rec = next; + count++; } - if(rec != null) - { + if(rec != null) { Tuple tuple = new Tuple(); tuple.addAttribute("" + count); rec.setField(1, tuple); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java index ffb0f2e3ce7ed..6e2a72afcb44a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -97,7 +95,6 @@ public static class AggCO extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; private final IntValue integer = new IntValue(); - private Record record = new Record(); /** * Output Schema: @@ -106,29 +103,23 @@ public static class AggCO extends ReduceFunction implements Serializable { * */ @Override - public void reduce(Iterator records, Collector out) - throws Exception { - + public void reduce(Iterable records, Collector out) { + Record record = null; int count = 0; - while (records.hasNext()) { - record = records.next(); - count+=record.getField(0, integer).getValue(); + for (Record next : records) { + record = next; + count += record.getField(0, integer).getValue(); } integer.setValue(count); record.setField(0, integer); out.collect(record); } - - /** - * Computes partial counts - */ - public void combine(Iterator records, Collector out) - throws Exception { + + public void combine(Iterable records, Collector out) { reduce(records, out); } - } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java index d80dc17622125..0df4aeaa50c6b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -221,12 +219,12 @@ public static class AntiJoinVisits extends CoGroupFunction implements Serializab * 2: AVG_DURATION */ @Override - public void coGroup(Iterator ranks, Iterator visits, Collector out) { + public void coGroup(Iterable ranks, Iterable visits, Collector out) { // Check if there is a entry in the visits relation - if (!visits.hasNext()) { - while (ranks.hasNext()) { - // Emit all rank pairs - out.collect(ranks.next()); + if (!visits.iterator().hasNext()) { + // Emit all rank pairs + for (Record r : ranks) { + out.collect(r); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java index f77ea61dc90cb..d576401e21d29 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java @@ -16,11 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational.query1Util; -import java.util.Iterator; - import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.test.recordJobs.util.Tuple; import org.apache.flink.types.LongValue; @@ -33,7 +30,7 @@ public class GroupByReturnFlag extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) { Record outRecord = new Record(); Tuple returnTuple = new Tuple(); @@ -41,8 +38,7 @@ public void reduce(Iterator records, Collector out) throws Excep double extendedPriceSum = 0.0; boolean first = true; - while(records.hasNext()) { - Record rec = records.next(); + for (Record rec : records) { Tuple t = rec.getField(1, Tuple.class); if(first) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java index 49b385e1e1fee..12c29764df050 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java @@ -16,12 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.relational.query9Util; - -import java.util.Iterator; - import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; @@ -43,15 +39,13 @@ public class AmountAggregate extends ReduceFunction { * Value: amount * */ - @Override - public void reduce(Iterator records, Collector out) throws Exception - { + public void reduce(Iterable records, Collector out) { Record record = null; float amount = 0; - while (records.hasNext()) { - record = records.next(); + for (Record next : records) { + record = next; StringValue value = record.getField(1, StringValue.class); amount += Float.parseFloat(value.toString()); } @@ -66,8 +60,7 @@ record = records.next(); * Creates partial sums of "amount" for each data batch: */ @Override - public void combine(Iterator records, Collector out) throws Exception - { + public void combine(Iterable records, Collector out) { reduce(records, out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java index d41353daa3858..964b7b018027c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.sort; import java.io.Serializable; -import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -56,9 +54,9 @@ public static class IdentityReducer extends ReduceFunction implements Serializab private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable records, Collector out) { - Record next = records.next(); + Record next = records.iterator().next(); // Increments the first field of the first record of the reduce group by 100 and emit it IntValue incrVal = next.getField(0, IntValue.class); @@ -67,8 +65,8 @@ public void reduce(Iterator records, Collector out) { out.collect(next); // emit all remaining records - while (records.hasNext()) { - out.collect(records.next()); + for (Record r : records){ + out.collect(r); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java index dcad7ac35f534..3958d1e2ae642 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java @@ -16,10 +16,8 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.wordcount; -import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.JobExecutionResult; @@ -89,11 +87,11 @@ public static class CountWords extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterator records, Collector out) throws Exception { + public void reduce(Iterable records, Collector out) throws Exception { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + for (Record next : records) { + element = next; int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -103,7 +101,7 @@ public void reduce(Iterator records, Collector out) throws Excep } @Override - public void combine(Iterator records, Collector out) throws Exception { + public void combine(Iterable records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java index 0240272f43180..d875f3d83c762 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.test.recordJobs.wordcount; import java.io.IOException; import java.io.Serializable; -import java.util.Iterator; import java.util.Set; import java.util.StringTokenizer; @@ -128,11 +126,12 @@ public static class CountWords extends ReduceFunction implements Serializable { private final IntValue cnt = new IntValue(); @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable records, Collector out) { Record element = null; int sum = 0; - while (records.hasNext()) { - element = records.next(); + + for ( Record next : records ) { + element = next; IntValue i = element.getField(1, IntValue.class); sum += i.getValue(); } From b7ffa6f350f4af467071993ce246f96b54720719 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 30 Jul 2014 13:26:46 +0200 Subject: [PATCH 2/3] [FLINK-1023] Retain backwards compatibility in old record API --- .../example/ReflectiveAvroTypeExample.java | 9 +- .../api/avro/AvroWithEmptyArrayITCase.java | 12 +- .../mapred/record/example/WordCount.java | 9 +- .../example/WordCountWithOutputFormat.java | 9 +- .../spargel/java/record/SpargelIteration.java | 24 +-- .../flink/compiler/util/DummyCoGroupStub.java | 11 +- .../flink/compiler/util/IdentityReduce.java | 7 +- .../flink/api/java/ExecutionEnvironment.java | 21 ++ .../translation/WrappingFunction.java | 6 +- .../record/functions/CoGroupFunction.java | 10 +- .../java/record/functions/ReduceFunction.java | 12 +- .../record/operators/CoGroupOperator.java | 61 +++++- .../java/record/operators/ReduceOperator.java | 73 +++++-- .../record/ReduceWrappingFunctionTest.java | 185 ++++++++++++++++++ .../operators/CachedMatchTaskTest.java | 21 +- .../operators/CoGroupTaskExternalITCase.java | 4 +- .../runtime/operators/CoGroupTaskTest.java | 10 +- .../runtime/operators/CombineTaskTest.java | 6 +- .../operators/ReduceTaskExternalITCase.java | 6 +- .../runtime/operators/ReduceTaskTest.java | 10 +- .../operators/chaining/ChainTaskTest.java | 10 +- .../CombiningUnilateralSortMergerITCase.java | 6 +- .../test/accumulators/AccumulatorITCase.java | 11 +- .../AccumulatorIterativeITCase.java | 6 +- .../CoGroupConnectedComponentsITCase.java | 10 +- ...erationTerminationWithTerminationTail.java | 7 +- .../IterationTerminationWithTwoTails.java | 7 +- .../IterationWithAllReducerITCase.java | 5 +- .../IterationWithChainingITCase.java | 7 +- .../iterative/IterationWithUnionITCase.java | 73 ++----- .../IterationWithChainingNepheleITCase.java | 4 +- .../CompensatableDotProductCoGroup.java | 12 +- .../flink/test/operators/CoGroupITCase.java | 11 +- .../flink/test/operators/ReduceITCase.java | 13 +- .../GroupOrderReduceITCase.java | 9 +- .../recordJobs/graph/ComputeEdgeDegrees.java | 13 +- .../graph/ConnectedComponentsWithCoGroup.java | 13 +- .../graph/DeltaPageRankWithInitialDeltas.java | 9 +- .../EnumTrianglesOnEdgesWithDegrees.java | 8 +- .../graph/EnumTrianglesRdfFoaf.java | 11 +- .../test/recordJobs/graph/PairwiseSP.java | 34 ++-- .../test/recordJobs/graph/SimplePageRank.java | 8 +- .../graph/WorksetConnectedComponents.java | 10 +- .../graph/pageRankUtil/DotProductCoGroup.java | 14 +- .../recordJobs/kmeans/KMeansBroadcast.java | 12 +- .../recordJobs/kmeans/KMeansSingleStep.java | 12 +- .../kmeans/udfs/FindNearestCenter.java | 12 +- .../kmeans/udfs/RecomputeClusterCenter.java | 13 +- .../recordJobs/relational/MergeOnlyJoin.java | 9 +- .../recordJobs/relational/TPCHQuery10.java | 15 +- .../recordJobs/relational/TPCHQuery3.java | 10 +- .../recordJobs/relational/TPCHQuery4.java | 13 +- .../relational/TPCHQueryAsterix.java | 23 ++- .../recordJobs/relational/WebLogAnalysis.java | 12 +- .../query1Util/GroupByReturnFlag.java | 8 +- .../query9Util/AmountAggregate.java | 15 +- .../test/recordJobs/sort/ReduceGroupSort.java | 10 +- .../test/recordJobs/wordcount/WordCount.java | 10 +- .../wordcount/WordCountAccumulators.java | 9 +- 59 files changed, 680 insertions(+), 320 deletions(-) create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java diff --git a/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java b/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java index a6f69b036b4c9..9cdaef0acdf04 100644 --- a/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java +++ b/flink-addons/flink-avro/src/main/java/org/apache/flink/api/java/record/io/avro/example/ReflectiveAvroTypeExample.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.Serializable; +import java.util.Iterator; import java.util.Random; import org.apache.flink.api.common.Plan; @@ -79,14 +80,14 @@ public static final class ConcatenatingReducer extends ReduceFunction implements private final Record result = new Record(2); @Override - public void reduce(Iterable records, Collector out) throws Exception { - Record r = records.iterator().next(); + public void reduce(Iterator records, Collector out) throws Exception { + Record r = records.next(); int num = r.getField(1, IntValue.class).getValue(); String names = r.getField(0, SUser.class).datum().getFavoriteColor().toString(); - for (Record next : records) { - r = next; + while (records.hasNext()) { + r = records.next(); names += " - " + r.getField(0, SUser.class).datum().getFavoriteColor().toString(); } diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java index dccee7c25ecc8..89db1fa92be5d 100644 --- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java +++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroWithEmptyArrayITCase.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.avro.reflect.Nullable; @@ -193,16 +194,15 @@ public static class MyCoGrouper extends CoGroupFunction { private static final long serialVersionUID = 1L; @Override - public void coGroup(Iterable records1, Iterable records2, Collector out) - throws Exception { + public void coGroup(Iterator records1, Iterator records2, Collector out) { Record r1 = null; - for (Record next : records1) { - r1 = next; + while (records1.hasNext()) { + r1 = records1.next(); } Record r2 = null; - for (Record next : records2) { - r2 = next; + while (records2.hasNext()) { + r2 = records2.next(); } if (r1 != null) { diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java index 372eedd96f707..88b18921a2340 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java @@ -19,6 +19,7 @@ package org.apache.flink.hadoopcompatibility.mapred.record.example; import java.io.Serializable; +import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.Plan; @@ -98,12 +99,12 @@ public static class CountWords extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { Record element = null; int sum = 0; - for (Record next : records) { - element = next; + while (records.hasNext()) { + element = records.next(); int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -113,7 +114,7 @@ public void reduce(Iterable records, Collector out) throws Excep } @Override - public void combine(Iterable records, Collector out) throws Exception { + public void combine(Iterator records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java index d7f508054b42a..8aaf8e515c628 100644 --- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java +++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java @@ -19,6 +19,7 @@ package org.apache.flink.hadoopcompatibility.mapred.record.example; import java.io.Serializable; +import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.Plan; @@ -96,12 +97,12 @@ public static class CountWords extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { Record element = null; int sum = 0; - for (Record next : records) { - element = next; + while (records.hasNext()) { + element = records.next(); int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -111,7 +112,7 @@ public void reduce(Iterable records, Collector out) throws Excep } @Override - public void combine(Iterable records, Collector out) throws Exception { + public void combine(Iterator records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java index 5f973ff104cd4..96bc79967cfd8 100644 --- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java +++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java @@ -160,23 +160,20 @@ public static final class VertexUpdateDriver, V extends Value, private MessageIterator messageIter; @Override - public void coGroup(Iterable messages, Iterable vertex, Collector out) throws Exception { - - final Iterator vertexIter = vertex.iterator(); + public void coGroup(Iterator messages, Iterator vertex, Collector out) throws Exception { - if (vertexIter.hasNext()) { - Record first = vertexIter.next(); + if (vertex.hasNext()) { + Record first = vertex.next(); first.getFieldInto(0, vertexKey); first.getFieldInto(1, vertexValue); - messageIter.setSource(messages.iterator()); + messageIter.setSource(messages); vertexUpdateFunction.setOutput(first, out); vertexUpdateFunction.updateVertex(vertexKey, vertexValue, messageIter); } else { - final Iterator messageIter = messages.iterator(); - if (messageIter.hasNext()) { + if (messages.hasNext()) { String message = "Target vertex does not exist!."; try { - Record next = messageIter.next(); + Record next = messages.next(); next.getFieldInto(0, vertexKey); message = "Target vertex '" + vertexKey + "' does not exist!."; } catch (Throwable t) {} @@ -236,13 +233,12 @@ public static final class MessagingDriver, V extends Value, M e private V vertexValue; @Override - public void coGroup(Iterable edges, Iterable state, Collector out) throws Exception { - final Iterator stateIter = state.iterator(); - if (stateIter.hasNext()) { - Record first = stateIter.next(); + public void coGroup(Iterator edges, Iterator state, Collector out) throws Exception { + if (state.hasNext()) { + Record first = state.next(); first.getFieldInto(0, vertexKey); first.getFieldInto(1, vertexValue); - messagingFunction.set(edges.iterator(), out); + messagingFunction.set(edges, out); messagingFunction.sendMessages(vertexKey, vertexValue); } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java index 9d32af87bc97a..13cd37bdb11d4 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/util/DummyCoGroupStub.java @@ -19,6 +19,7 @@ package org.apache.flink.compiler.util; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.types.Record; @@ -28,13 +29,13 @@ public class DummyCoGroupStub extends CoGroupFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void coGroup(Iterable records1, Iterable records2, Collector out) { - for (Record r : records1) { - out.collect(r); + public void coGroup(Iterator records1, Iterator records2, Collector out) { + while (records1.hasNext()) { + out.collect(records1.next()); } - for (Record r : records2) { - out.collect(r); + while (records2.hasNext()) { + out.collect(records2.next()); } } } diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java b/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java index 9cd3b0f068be1..b78a850a44afa 100644 --- a/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java +++ b/flink-compiler/src/test/java/org/apache/flink/compiler/util/IdentityReduce.java @@ -19,6 +19,7 @@ package org.apache.flink.compiler.util; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept; @@ -30,9 +31,9 @@ public final class IdentityReduce extends ReduceFunction implements Serializable private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) throws Exception { - for (Record r : records) { - out.collect(r); + public void reduce(Iterator records, Collector out) throws Exception { + while (records.hasNext()) { + out.collect(records.next()); } } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index d00fb4743323a..085e9bbcbf73b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; +import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.api.java.io.CsvReader; @@ -253,6 +254,26 @@ public DataSource readTextFileWithValue(String filePath, String cha public CsvReader readCsvFile(String filePath) { return new CsvReader(filePath, this); } + + // ------------------------------------ File Input Format ----------------------------------------- + + public DataSource readFile(FileInputFormat inputFormat, String filePath) { + if (inputFormat == null) { + throw new IllegalArgumentException("InputFormat must not be null."); + } + if (filePath == null) { + throw new IllegalArgumentException("The file path must not be null."); + } + + inputFormat.setFilePath(new Path(filePath)); + try { + return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); + } + catch (Exception e) { + throw new InvalidProgramException("The type returned by the input format could not be automatically determined. " + + "Please specify the TypeInformation of the produced type explicitly."); + } + } // ----------------------------------- Generic Input Format --------------------------------------- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index c98df6b373777..7640c2b52cb47 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -40,13 +40,16 @@ public abstract class WrappingFunction extends Abstr private static final long serialVersionUID = 1L; - protected final T wrappedFunction; + protected T wrappedFunction; protected WrappingFunction(T wrappedFunction) { this.wrappedFunction = wrappedFunction; } + public T getWrappedFunction() { + return wrappedFunction; + } @Override public void open(Configuration parameters) throws Exception { @@ -170,6 +173,5 @@ public > T getIterationAggregator(String name) { public T getPreviousIterationAggregate(String name) { return ((IterationRuntimeContext) context).getPreviousIterationAggregate(name); } - } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java index 933ae919cfb86..8f2e3fd17f40f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/CoGroupFunction.java @@ -16,18 +16,18 @@ * limitations under the License. */ - package org.apache.flink.api.java.record.functions; +import java.util.Iterator; + import org.apache.flink.api.common.functions.AbstractFunction; -import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; /** * The CoGroupFunction is the base class for functions that are invoked by a {@link org.apache.flink.api.java.operators.CoGroupOperator}. */ -public abstract class CoGroupFunction extends AbstractFunction implements GenericCoGrouper { +public abstract class CoGroupFunction extends AbstractFunction { private static final long serialVersionUID = 1L; @@ -44,7 +44,5 @@ public abstract class CoGroupFunction extends AbstractFunction implements Generi * runtime catches an exception, it aborts the task and lets the fail-over logic * decide whether to retry the task execution. */ - @Override - public abstract void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception; - + public abstract void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java index b7acd1e2bde2f..15f79ce8e0ccc 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/ReduceFunction.java @@ -18,9 +18,9 @@ package org.apache.flink.api.java.record.functions; +import java.util.Iterator; + import org.apache.flink.api.common.functions.AbstractFunction; -import org.apache.flink.api.common.functions.GenericCombine; -import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -28,7 +28,7 @@ * The ReduceFunction must be extended to provide a reducer implementation, as invoked by a * {@link org.apache.flink.api.java.operators.ReduceOperator}. */ -public abstract class ReduceFunction extends AbstractFunction implements GenericGroupReduce, GenericCombine { +public abstract class ReduceFunction extends AbstractFunction { private static final long serialVersionUID = 1L; @@ -44,8 +44,7 @@ public abstract class ReduceFunction extends AbstractFunction implements Generic * runtime catches an exception, it aborts the reduce task and lets the fail-over logic * decide whether to retry the reduce execution. */ - @Override - public abstract void reduce(Iterable records, Collector out) throws Exception; + public abstract void reduce(Iterator records, Collector out) throws Exception; /** * No default implementation provided. @@ -68,8 +67,7 @@ public abstract class ReduceFunction extends AbstractFunction implements Generic * runtime catches an exception, it aborts the combine task and lets the fail-over logic * decide whether to retry the combiner execution. */ - @Override - public void combine(Iterable records, Collector out) throws Exception { + public void combine(Iterator records, Collector out) throws Exception { // to be implemented, if the reducer should use a combiner. Note that the combining method // is only used, if the stub class is further annotated with the annotation // @ReduceOperator.Combinable diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/CoGroupOperator.java index a8cedebbbaf65..b3d9378501960 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/CoGroupOperator.java @@ -19,23 +19,29 @@ package org.apache.flink.api.java.record.operators; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.commons.lang3.Validate; +import org.apache.flink.api.common.functions.GenericCoGrouper; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.RecordOperator; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; +import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation; import org.apache.flink.types.Key; import org.apache.flink.types.Record; +import org.apache.flink.util.Collector; +import org.apache.flink.util.InstantiationUtil; /** * CoGroupOperator that applies a {@link CoGroupFunction} to groups of records sharing @@ -43,7 +49,7 @@ * * @see CoGroupFunction */ -public class CoGroupOperator extends CoGroupOperatorBase implements RecordOperator { +public class CoGroupOperator extends CoGroupOperatorBase> implements RecordOperator { /** * The types of the keys that the operator groups on. @@ -61,7 +67,8 @@ public class CoGroupOperator extends CoGroupOperatorBase> keyClass, int keyColumn1, int keyColumn2) { - return new Builder(new UserCodeObjectWrapper(udf), keyClass, keyColumn1, keyColumn2); + WrappingCoGroupFunction wrapper = new WrappingCoGroupFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn1, keyColumn2); } /** @@ -75,7 +82,8 @@ public static Builder builder(CoGroupFunction udf, Class> keyCl public static Builder builder(Class udf, Class> keyClass, int keyColumn1, int keyColumn2) { - return new Builder(new UserCodeClassWrapper(udf), keyClass, keyColumn1, keyColumn2); + WrappingCoGroupFunction wrapper = new WrappingClassCoGroupFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn1, keyColumn2); } /** @@ -96,7 +104,9 @@ protected CoGroupOperator(Builder builder) { setBroadcastVariables(builder.broadcastInputs); setGroupOrderForInputOne(builder.secondaryOrder1); setGroupOrderForInputTwo(builder.secondaryOrder2); - setSemanticProperties(FunctionAnnotation.readDualConstantAnnotations(builder.udf)); + + CoGroupFunction function = ((WrappingCoGroupFunction) builder.udf.getUserCodeObject()).getWrappedFunction(); + setSemanticProperties(FunctionAnnotation.readDualConstantAnnotations(new UserCodeObjectWrapper(function))); } // -------------------------------------------------------------------------------------------- @@ -115,7 +125,7 @@ public Class>[] getKeyClasses() { public static class Builder { /* The required parameters */ - private final UserCodeWrapper udf; + private final UserCodeWrapper> udf; private final List>> keyClasses; private final List keyColumns1; private final List keyColumns2; @@ -136,7 +146,7 @@ public static class Builder { * @param keyColumn1 The position of the key in the first input's records. * @param keyColumn2 The position of the key in the second input's records. */ - protected Builder(UserCodeWrapper udf, Class> keyClass, + protected Builder(UserCodeWrapper> udf, Class> keyClass, int keyColumn1, int keyColumn2) { this.udf = udf; @@ -157,7 +167,7 @@ protected Builder(UserCodeWrapper udf, Class> * * @param udf The {@link CoGroupFunction} implementation for this CoGroup operator. */ - protected Builder(UserCodeWrapper udf) { + protected Builder(UserCodeWrapper> udf) { this.udf = udf; this.keyClasses = new ArrayList>>(); this.keyColumns1 = new ArrayList(); @@ -338,4 +348,39 @@ public CoGroupOperator build() { return new CoGroupOperator(this); } } + + // ============================================================================================ + + public static class WrappingCoGroupFunction extends WrappingFunction implements GenericCoGrouper { + + private static final long serialVersionUID = 1L; + + public WrappingCoGroupFunction(CoGroupFunction coGrouper) { + super(coGrouper); + } + + + @Override + public void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception { + this.wrappedFunction.coGroup(records1.iterator(), records2.iterator(), out); + } + } + + public static final class WrappingClassCoGroupFunction extends WrappingCoGroupFunction { + + private static final long serialVersionUID = 1L; + + public WrappingClassCoGroupFunction(Class reducer) { + super(InstantiationUtil.instantiate(reducer)); + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeObject(wrappedFunction.getClass()); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + Class clazz = (Class) in.readObject(); + this.wrappedFunction = (CoGroupFunction) InstantiationUtil.instantiate(clazz); + } + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java index 3860fbeec93d7..13289ec61dd79 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java @@ -16,9 +16,11 @@ * limitations under the License. */ - package org.apache.flink.api.java.record.operators; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; @@ -29,24 +31,28 @@ import java.util.Map; import org.apache.commons.lang3.Validate; +import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.GenericGroupReduce; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.RecordOperator; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; +import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.types.Key; import org.apache.flink.types.Record; +import org.apache.flink.util.Collector; +import org.apache.flink.util.InstantiationUtil; /** * ReduceOperator evaluating a {@link ReduceFunction} over each group of records that share the same key. * * @see ReduceFunction */ -public class ReduceOperator extends GroupReduceOperatorBase implements RecordOperator { +public class ReduceOperator extends GroupReduceOperatorBase> implements RecordOperator { private static final String DEFAULT_NAME = ""; // the default name for contracts @@ -63,7 +69,8 @@ public class ReduceOperator extends GroupReduceOperatorBase(udf)); + WrappingReduceFunction wrapper = new WrappingReduceFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper)); } /** @@ -74,7 +81,8 @@ public static Builder builder(ReduceFunction udf) { * @param keyColumn The position of the key. */ public static Builder builder(ReduceFunction udf, Class> keyClass, int keyColumn) { - return new Builder(new UserCodeObjectWrapper(udf), keyClass, keyColumn); + WrappingReduceFunction wrapper = new WrappingReduceFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn); } /** @@ -83,7 +91,8 @@ public static Builder builder(ReduceFunction udf, Class> keyCla * @param udf The {@link ReduceFunction} implementation for this Reduce contract. */ public static Builder builder(Class udf) { - return new Builder(new UserCodeClassWrapper(udf)); + WrappingReduceFunction wrapper = new WrappingClassReduceFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper)); } /** @@ -94,7 +103,8 @@ public static Builder builder(Class udf) { * @param keyColumn The position of the key. */ public static Builder builder(Class udf, Class> keyClass, int keyColumn) { - return new Builder(new UserCodeClassWrapper(udf), keyClass, keyColumn); + WrappingReduceFunction wrapper = new WrappingClassReduceFunction(udf); + return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn); } /** @@ -111,7 +121,9 @@ protected ReduceOperator(Builder builder) { setGroupOrder(builder.secondaryOrder); setBroadcastVariables(builder.broadcastInputs); - setSemanticProperties(FunctionAnnotation.readSingleConstantAnnotations(builder.udf)); + + ReduceFunction function = ((WrappingReduceFunction) builder.udf.getUserCodeObject()).getWrappedFunction(); + setSemanticProperties(FunctionAnnotation.readSingleConstantAnnotations(new UserCodeObjectWrapper(function))); } // -------------------------------------------------------------------------------------------- @@ -178,7 +190,7 @@ public boolean isCombinable() { public static class Builder { /* The required parameters */ - private final UserCodeWrapper udf; + private final UserCodeWrapper> udf; private final List>> keyClasses; private final List keyColumns; @@ -193,7 +205,7 @@ public static class Builder { * * @param udf The {@link ReduceFunction} implementation for this Reduce contract. */ - private Builder(UserCodeWrapper udf) { + private Builder(UserCodeWrapper> udf) { this.udf = udf; this.keyClasses = new ArrayList>>(); this.keyColumns = new ArrayList(); @@ -208,7 +220,7 @@ private Builder(UserCodeWrapper udf) { * @param keyClass The class of the key data type. * @param keyColumn The position of the key. */ - private Builder(UserCodeWrapper udf, Class> keyClass, int keyColumn) { + private Builder(UserCodeWrapper> udf, Class> keyClass, int keyColumn) { this.udf = udf; this.keyClasses = new ArrayList>>(); this.keyClasses.add(keyClass); @@ -330,4 +342,43 @@ public ReduceOperator build() { return new ReduceOperator(this); } } + + // ============================================================================================ + + public static class WrappingReduceFunction extends WrappingFunction implements GenericGroupReduce, GenericCombine { + + private static final long serialVersionUID = 1L; + + public WrappingReduceFunction(ReduceFunction reducer) { + super(reducer); + } + + @Override + public final void reduce(Iterable records, Collector out) throws Exception { + this.wrappedFunction.reduce(records.iterator(), out); + } + + @Override + public final void combine(Iterable records, Collector out) throws Exception { + this.wrappedFunction.combine(records.iterator(), out); + } + } + + public static final class WrappingClassReduceFunction extends WrappingReduceFunction { + + private static final long serialVersionUID = 1L; + + public WrappingClassReduceFunction(Class reducer) { + super(InstantiationUtil.instantiate(reducer)); + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeObject(wrappedFunction.getClass()); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + Class clazz = (Class) in.readObject(); + this.wrappedFunction = (ReduceFunction) InstantiationUtil.instantiate(clazz); + } + } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java new file mode 100644 index 0000000000000..78331cef5ea1d --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java @@ -0,0 +1,185 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.record; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.lang3.SerializationUtils; +import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.GenericCombine; +import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.operators.util.UserCodeWrapper; +import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.record.operators.ReduceOperator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.Record; +import org.apache.flink.util.Collector; +import org.junit.Test; + +@SuppressWarnings("serial") +public class ReduceWrappingFunctionTest { + + @SuppressWarnings("unchecked") + @Test + public void testWrappedReduceObject() { + try { + AtomicInteger methodCounter = new AtomicInteger(); + + ReduceOperator reduceOp = ReduceOperator.builder(new TestReduceFunction(methodCounter)).build(); + + AbstractFunction reducer = (AbstractFunction) reduceOp.getUserCodeWrapper().getUserCodeObject(); + + // test the method invocations + reducer.close(); + reducer.open(new Configuration()); + assertEquals(2, methodCounter.get()); + + // prepare the reduce / combine tests + final List target = new ArrayList(); + Collector collector = new Collector() { + @Override + public void collect(Record record) { + target.add(record); + } + @Override + public void close() {} + }; + + List source = new ArrayList(); + source.add(new Record(new IntValue(42), new LongValue(11))); + source.add(new Record(new IntValue(13), new LongValue(17))); + + // test reduce + ((GenericGroupReduce) reducer).reduce(source, collector); + assertEquals(2, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(0).getField(1, LongValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(17), target.get(1).getField(1, LongValue.class)); + target.clear(); + + // test combine + ((GenericCombine) reducer).combine(source, collector); + assertEquals(2, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(0).getField(1, LongValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(17), target.get(1).getField(1, LongValue.class)); + target.clear(); + + // test the serialization + SerializationUtils.clone(reducer); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testWrappedReduceClass() { + try { + ReduceOperator reduceOp = ReduceOperator.builder(TestReduceFunction.class).build(); + + UserCodeWrapper> udf = reduceOp.getUserCodeWrapper(); + UserCodeWrapper> copy = SerializationUtils.clone(udf); + GenericGroupReduce reducer = copy.getUserCodeObject(); + + // prepare the reduce / combine tests + final List target = new ArrayList(); + Collector collector = new Collector() { + @Override + public void collect(Record record) { + target.add(record); + } + @Override + public void close() {} + }; + + List source = new ArrayList(); + source.add(new Record(new IntValue(42), new LongValue(11))); + source.add(new Record(new IntValue(13), new LongValue(17))); + + // test reduce + ((GenericGroupReduce) reducer).reduce(source, collector); + assertEquals(2, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(0).getField(1, LongValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(17), target.get(1).getField(1, LongValue.class)); + target.clear(); + + // test combine + ((GenericCombine) reducer).combine(source, collector); + assertEquals(2, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(0).getField(1, LongValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(17), target.get(1).getField(1, LongValue.class)); + target.clear(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + + public static class TestReduceFunction extends ReduceFunction { + + private final AtomicInteger methodCounter; + + private TestReduceFunction(AtomicInteger methodCounter) { + this.methodCounter= methodCounter; + } + + public TestReduceFunction() { + methodCounter = new AtomicInteger(); + } + + @Override + public void reduce(Iterator records, Collector out) throws Exception { + while (records.hasNext()) { + out.collect(records.next()); + } + } + + @Override + public void close() throws Exception { + methodCounter.incrementAndGet(); + super.close(); + } + + @Override + public void open(Configuration parameters) throws Exception { + methodCounter.incrementAndGet(); + super.open(parameters); + } + }; +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java index e91e1007e15c3..bfcd03fbcc404 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.flink.api.common.functions.GenericJoiner; -import org.apache.flink.api.java.record.functions.JoinFunction; +import org.apache.flink.api.java.functions.JoinFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.runtime.operators.BuildFirstCachedMatchDriver; @@ -462,31 +462,31 @@ public void run() { // ================================================================================================= - public static final class MockMatchStub extends JoinFunction { + public static final class MockMatchStub extends JoinFunction { private static final long serialVersionUID = 1L; @Override - public void join(Record record1, Record record2, Collector out) throws Exception { - out.collect(record1); + public Record join(Record record1, Record record2) throws Exception { + return record1; } } - public static final class MockFailingMatchStub extends JoinFunction { + public static final class MockFailingMatchStub extends JoinFunction { private static final long serialVersionUID = 1L; private int cnt = 0; @Override - public void join(Record record1, Record record2, Collector out) { + public Record join(Record first, Record second) throws Exception { if (++this.cnt >= 10) { throw new ExpectedTestException(); } - out.collect(record1); + return first; } } - public static final class MockDelayingMatchStub extends JoinFunction { + public static final class MockDelayingMatchStub extends JoinFunction { private static final long serialVersionUID = 1L; @Override @@ -495,5 +495,10 @@ public void join(Record record1, Record record2, Collector out) { Thread.sleep(100); } catch (InterruptedException e) { } } + + @Override + public Record join(Record first, Record second){ + return null; + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java index 9624676d0d7d7..27fca6f98d41b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java @@ -21,7 +21,7 @@ import junit.framework.Assert; import org.apache.flink.api.common.functions.GenericCoGrouper; -import org.apache.flink.api.java.record.functions.CoGroupFunction; +import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.runtime.operators.CoGroupDriver; @@ -84,7 +84,7 @@ public void testExternalSortCoGroupTask() { Assert.assertEquals("Wrong result set size.", expCnt, this.output.getNumberOfRecords()); } - public static final class MockCoGroupStub extends CoGroupFunction { + public static final class MockCoGroupStub extends CoGroupFunction { private static final long serialVersionUID = 1L; private final Record res = new Record(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java index c06ce0ee8a31e..c55d221e85ae5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java @@ -23,7 +23,7 @@ import junit.framework.Assert; import org.apache.flink.api.common.functions.GenericCoGrouper; -import org.apache.flink.api.java.record.functions.CoGroupFunction; +import org.apache.flink.api.java.functions.CoGroupFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.runtime.operators.CoGroupDriver; @@ -40,8 +40,8 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class CoGroupTaskTest extends DriverTestBase> -{ +public class CoGroupTaskTest extends DriverTestBase> { + private static final long SORT_MEM = 3*1024*1024; @SuppressWarnings("unchecked") @@ -401,7 +401,7 @@ public void run() { Assert.assertTrue("Test threw an exception even though it was properly canceled.", success.get()); } - public static class MockFailingCoGroupStub extends CoGroupFunction { + public static class MockFailingCoGroupStub extends CoGroupFunction { private static final long serialVersionUID = 1L; private int cnt = 0; @@ -437,7 +437,7 @@ public void coGroup(Iterable records1, Iterable records2, Collec } - public static final class MockDelayingCoGroupStub extends CoGroupFunction { + public static final class MockDelayingCoGroupStub extends CoGroupFunction { private static final long serialVersionUID = 1L; @SuppressWarnings("unused") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java index ad29bf31ca175..ec67f81c2add7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java @@ -24,7 +24,7 @@ import junit.framework.Assert; import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.runtime.operators.DriverStrategy; @@ -163,7 +163,7 @@ public void run() { } @Combinable - public static class MockCombiningReduceStub extends ReduceFunction { + public static class MockCombiningReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue theInteger = new IntValue(); @@ -191,7 +191,7 @@ public void combine(Iterable records, Collector out) throws Exce } @Combinable - public static final class MockFailingCombiningReduceStub extends ReduceFunction { + public static final class MockFailingCombiningReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private int cnt = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java index b14f0d3d860a0..652a1b2f07dc8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java @@ -26,7 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; @@ -213,7 +213,7 @@ public void testMultiLevelMergeCombiningReduceTask() { } - public static class MockReduceStub extends ReduceFunction { + public static class MockReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue key = new IntValue(); @@ -236,7 +236,7 @@ public void reduce(Iterable records, Collector out) { } @Combinable - public static class MockCombiningReduceStub extends ReduceFunction { + public static class MockCombiningReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue key = new IntValue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java index c4858f470f025..63a126efef12d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java @@ -27,7 +27,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.flink.api.common.functions.GenericGroupReduce; -import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; @@ -270,7 +270,7 @@ public void run() { } - public static class MockReduceStub extends ReduceFunction { + public static class MockReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue key = new IntValue(); @@ -293,7 +293,7 @@ public void reduce(Iterable records, Collector out) { } @Combinable - public static class MockCombiningReduceStub extends ReduceFunction { + public static class MockCombiningReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue key = new IntValue(); @@ -336,7 +336,7 @@ public void combine(Iterable records, Collector out) { } - public static class MockFailingReduceStub extends ReduceFunction { + public static class MockFailingReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private int cnt = 0; @@ -365,7 +365,7 @@ public void reduce(Iterable records, Collector out) { } } - public static class MockDelayingReduceStub extends ReduceFunction { + public static class MockDelayingReduceStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java index 0827c7ee33529..b33381b6f7a5d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.chaining; import java.util.ArrayList; @@ -24,7 +23,7 @@ import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; -import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparatorFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; import org.apache.flink.configuration.Configuration; @@ -189,7 +188,7 @@ public void testFailingMapTask() { } } - public static final class MockFailingCombineStub extends ReduceFunction { + public static final class MockFailingCombineStub extends GroupReduceFunction { private static final long serialVersionUID = 1L; private int cnt = 0; @@ -199,8 +198,9 @@ public void reduce(Iterable records, Collector out) throws Excep if (++this.cnt >= 5) { throw new RuntimeException("Expected Test Exception"); } - for (Record rec : records) { - out.collect(rec); + + for (Record r : records) { + out.collect(r); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java index 8e3e1b4a40c18..8925667db3289 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java @@ -32,7 +32,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.api.java.record.functions.ReduceFunction; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; import org.apache.flink.configuration.Configuration; @@ -273,7 +273,7 @@ public void testSortAndValidate() throws Exception // -------------------------------------------------------------------------------------------- - public static class TestCountCombiner extends ReduceFunction { + public static class TestCountCombiner extends GroupReduceFunction { private static final long serialVersionUID = 1L; private final IntValue count = new IntValue(); @@ -311,7 +311,7 @@ public void close() throws Exception { } } - public static class TestCountCombiner2 extends ReduceFunction { + public static class TestCountCombiner2 extends GroupReduceFunction { private static final long serialVersionUID = 1L; public volatile boolean opened = false; diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index 208c429844088..5497004bd44be 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -255,23 +256,23 @@ public void open(Configuration parameters) throws Exception { } @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { reduceCalls.add(1); reduceInternal(records, out); } @Override - public void combine(Iterable records, Collector out) throws Exception { + public void combine(Iterator records, Collector out) throws Exception { combineCalls.add(1); reduceInternal(records, out); } - private void reduceInternal(Iterable records, Collector out) { + private void reduceInternal(Iterator records, Collector out) { Record element = null; int sum = 0; - for (Record next : records) { - element = next; + while (records.hasNext()) { + element = records.next(); IntValue i = element.getField(1, IntValue.class); sum += i.getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java index 2495dd3b24342..197a7454a46b0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.accumulators.IntCounter; @@ -123,11 +124,12 @@ public void open(Configuration parameters) throws Exception { } @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { // Compute the sum int sum = 0; - for (Record r : records) { + while (records.hasNext()) { + Record r = records.next(); Integer value = Integer.parseInt(r.getField(0, StringValue.class).getValue()); sum += value; testCounter.add(value); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java index d35f886e30a0e..f2a43a8c0105e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java @@ -20,6 +20,7 @@ import java.io.BufferedReader; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.CoGroupFunction; @@ -92,16 +93,17 @@ public static final class MinIdAndUpdate extends CoGroupFunction implements Seri private final LongValue newComponentId = new LongValue(); @Override - public void coGroup(Iterable candidates, Iterable current, Collector out) throws Exception { - if (!current.iterator().hasNext()) { + public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { + if (!current.hasNext()) { throw new Exception("Error: Id not encountered before."); } - Record old = current.iterator().next(); + Record old = current.next(); long oldId = old.getField(1, LongValue.class).getValue(); long minimumComponentID = Long.MAX_VALUE; - for (Record candidate : candidates) { + while (candidates.hasNext()) { + Record candidate = candidates.next(); long candidateComponentID = candidate.getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java index 3eb4775c97dea..3dc0bdffefb4b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java @@ -19,6 +19,7 @@ package org.apache.flink.test.iterative; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -104,11 +105,11 @@ public static final class SumReducer extends ReduceFunction implements Serializa private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable it, Collector out) { + public void reduce(Iterator it, Collector out) { // Compute the sum int sum = 0; - for (Record r : it) { - sum += Integer.parseInt(r.getField(0, StringValue.class).getValue()) + 1; + while (it.hasNext()) { + sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; } out.collect(new Record(new StringValue(Integer.toString(sum)))); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java index 641540c77f531..2c777f5ceb931 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java @@ -19,6 +19,7 @@ package org.apache.flink.test.iterative; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -105,12 +106,12 @@ static final class SumReducer extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable it, Collector out) { + public void reduce(Iterator it, Collector out) { // Compute the sum int sum = 0; - for (Record r : it) { - sum += Integer.parseInt(r.getField(0, StringValue.class).getValue()) + 1; + while (it.hasNext()) { + sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; } out.collect(new Record(new StringValue(Integer.toString(sum)))); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java index b35b008448022..b3145dceaf32c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.iterative; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.ReduceFunction; @@ -100,8 +101,8 @@ public static final class PickOneReducer extends ReduceFunction implements Seria private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable it, Collector out) { - out.collect(it.iterator().next()); + public void reduce(Iterator it, Collector out) { + out.collect(it.next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java index 2a0831d78bbb0..d00602f565080 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.record.functions.MapFunction; @@ -92,9 +93,9 @@ public static final class DummyReducer extends ReduceFunction implements Seriali private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable it, Collector out) { - for (Record r : it) { - out.collect(r); + public void reduce(Iterator it, Collector out) { + while (it.hasNext()) { + out.collect(it.next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java index 0fd3d2c235a18..a5faab7f4ee88 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java @@ -19,38 +19,25 @@ package org.apache.flink.test.iterative; import java.io.Serializable; -import java.util.Collection; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.IterativeDataSet; +import org.apache.flink.api.java.functions.GroupReduceFunction; +import org.apache.flink.api.java.functions.MapFunction; import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat; import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat; -import org.apache.flink.test.util.RecordAPITestBase; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; -@RunWith(Parameterized.class) -public class IterationWithUnionITCase extends RecordAPITestBase { +public class IterationWithUnionITCase extends JavaProgramTestBase { private static final String DATAPOINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; protected String dataPath; protected String resultPath; - - public IterationWithUnionITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(DOP); - } @Override protected void preSubmit() throws Exception { @@ -64,48 +51,30 @@ protected void postSubmit() throws Exception { } @Override - protected Plan getTestJob() { - return getPlan(config.getInteger("IterationWithUnionITCase#NumSubtasks", 1), dataPath, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config1 = new Configuration(); - config1.setInteger("IterationWithUnionITCase#NumSubtasks", DOP); - - return toParameterList(config1); - } - - private static Plan getPlan(int numSubTasks, String input, String output) { - FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input"); - initialInput.setDegreeOfParallelism(1); + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(2); - - @SuppressWarnings("unchecked") - MapOperator map2 = MapOperator.builder(new IdentityMapper()).input(iteration.getPartialSolution(), iteration.getPartialSolution()).name("map").build(); + DataSet initialInput = env.readFile(new PointInFormat(), this.dataPath).setParallelism(1); - iteration.setNextPartialSolution(map2); - - FileDataSink finalResult = new FileDataSink(new PointOutFormat(), output, iteration, "Output"); - - Plan plan = new Plan(finalResult, "Iteration with union test"); - plan.setDefaultParallelism(numSubTasks); - return plan; + IterativeDataSet iteration = initialInput.iterate(2); + + DataSet result = iteration.union(iteration).map(new IdentityMapper()); + + iteration.closeWith(result).write(new PointOutFormat(), this.resultPath); + + env.execute(); } - static final class IdentityMapper extends MapFunction implements Serializable { + static final class IdentityMapper extends MapFunction implements Serializable { private static final long serialVersionUID = 1L; @Override - public void map(Record rec, Collector out) { - out.collect(rec); + public Record map(Record rec) { + return rec; } } - static final class DummyReducer extends ReduceFunction implements Serializable { + static final class DummyReducer extends GroupReduceFunction { private static final long serialVersionUID = 1L; @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java index 0e2c7138cce7c..b1612444c7adb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java @@ -23,8 +23,8 @@ import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; +import org.apache.flink.api.java.functions.GroupReduceFunction; import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.io.FileOutputFormat; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparatorFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; @@ -272,7 +272,7 @@ public void map(Record rec, Collector out) { } } - public static final class DummyReducer extends ReduceFunction { + public static final class DummyReducer extends GroupReduceFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java index 508462c8437c0..c51cf96de5526 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java @@ -18,6 +18,7 @@ package org.apache.flink.test.iterative.nephele.danglingpagerank; +import java.util.Iterator; import java.util.Set; import org.apache.flink.api.java.record.functions.CoGroupFunction; @@ -88,18 +89,19 @@ public void open(Configuration parameters) { } @Override - public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, Collector collector) { + public void coGroup(Iterator currentPageRankIterator, Iterator partialRanks, Collector collector) { - if (!currentPageRankIterator.iterator().hasNext()) { - long missingVertex = partialRanks.iterator().next().getField(0, LongValue.class).getValue(); + if (!currentPageRankIterator.hasNext()) { + long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - Record currentPageRank = currentPageRankIterator.iterator().next(); + Record currentPageRank = currentPageRankIterator.next(); long edges = 0; double summedRank = 0; - for (Record pr : partialRanks) { + while (partialRanks.hasNext()) { + Record pr = partialRanks.next(); summedRank += pr.getField(1, doubleInstance).getValue(); edges++; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java index 32724561c8e07..865c550d792e6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import java.util.LinkedList; import org.apache.flink.api.common.Plan; @@ -113,21 +114,21 @@ public static class TestCoGrouper extends CoGroupFunction implements Serializabl private StringValue valueString = new StringValue(); @Override - public void coGroup(Iterable records1, Iterable records2, Collector out) { + public void coGroup(Iterator records1, Iterator records2, Collector out) { Record record = null; int sum = 0; - for (Record next : records1) { - record = next; + while (records1.hasNext()) { + record = records1.next(); keyString = record.getField(0, keyString); valueString = record.getField(1, valueString); sum += Integer.parseInt(valueString.getValue()); } - for (Record next : records2) { - record = next; + while (records2.hasNext()) { + record = records2.next(); keyString = record.getField(0, keyString); valueString = record.getField(1, valueString); sum -= Integer.parseInt(valueString.getValue()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java index c60e647cf81ff..2711417e843fc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java @@ -45,6 +45,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import java.util.LinkedList; @RunWith(Parameterized.class) @@ -77,12 +78,12 @@ public static class TestReducer extends ReduceFunction implements Serializable { private StringValue combineValue = new StringValue(); @Override - public void combine(Iterable records, Collector out) { + public void combine(Iterator records, Collector out) { Record record = null; int sum = 0; - for (Record next : records) { - record = next; + while (records.hasNext()) { + record = records.next(); combineValue = record.getField(1, combineValue); sum += Integer.parseInt(combineValue.toString()); } @@ -92,12 +93,12 @@ record = next; } @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { Record record = null; int sum = 0; - for (Record next : records) { - record = next; + while (records.hasNext()) { + record = records.next(); reduceValue = record.getField(1, reduceValue); sum += Integer.parseInt(reduceValue.toString()); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java index 057212c726016..0b843098cc360 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Collection; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.Order; @@ -117,11 +118,11 @@ public static final class CheckingReducer extends ReduceFunction implements Seri private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) throws Exception { - int lastValue = records.iterator().next().getField(1, IntValue.class).getValue(); + public void reduce(Iterator records, Collector out) throws Exception { + int lastValue = records.next().getField(1, IntValue.class).getValue(); - for (Record r : records) { - int nextValue = r.getField(1, IntValue.class).getValue(); + while (records.hasNext()) { + int nextValue = records.next().getField(1, IntValue.class).getValue(); if (nextValue < lastValue) { throw new Exception("Group Order is violated!"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java index cf2e7d1a18155..6ab9c3cad3149 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java @@ -16,10 +16,12 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; import java.util.Arrays; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -82,13 +84,14 @@ public static final class CountEdges extends ReduceFunction implements Serializa private int[] vals = new int[1024]; @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) throws Exception { int[] vals = this.vals; int len = 0; int key = -1; // collect all values - for (Record rec : records) { + while (records.hasNext()) { + final Record rec = records.next(); final int id = rec.getField(1, IntValue.class).getValue(); if (key == -1) { key = rec.getField(0, IntValue.class).getValue(); @@ -153,13 +156,13 @@ public static final class JoinCountsAndUniquify extends ReduceFunction implement private final IntValue count2 = new IntValue(); @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) throws Exception { Record rec = null; int c1 = 0, c2 = 0; int numValues = 0; - for (Record next : records) { - rec = next; + while (records.hasNext()) { + rec = records.next(); final int f1 = rec.getField(2, IntValue.class).getValue(); final int f2 = rec.getField(3, IntValue.class).getValue(); c1 += f1; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java index 831173cb4e540..fe51914b34a0e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -55,18 +57,17 @@ public static final class MinIdAndUpdate extends CoGroupFunction implements Seri private final LongValue newComponentId = new LongValue(); @Override - public void coGroup(Iterable candidates, Iterable current, Collector out) throws Exception { - if (!current.iterator().hasNext()) { + public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { + if (!current.hasNext()) { throw new Exception("Error: Id not encountered before."); } - - Record old = current.iterator().next(); + Record old = current.next(); long oldId = old.getField(1, LongValue.class).getValue(); long minimumComponentID = Long.MAX_VALUE; - for (Record candidate : candidates) { - long candidateComponentID = candidate.getField(1, LongValue.class).getValue(); + while (candidates.hasNext()) { + long candidateComponentID = candidates.next().getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java index e8a9f0b2c9a18..e5e552d4c7d56 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java @@ -16,8 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; +import java.util.Iterator; + import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; import org.apache.flink.api.common.ProgramDescription; @@ -71,14 +74,14 @@ public static final class UpdateRankReduceDelta extends ReduceFunction { private final DoubleValue newRank = new DoubleValue(); @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { double rankSum = 0.0; double rank; Record rec = null; - for (Record next : records) { - rec = next; + while (records.hasNext()) { + rec = records.next(); rank = rec.getField(1, DoubleValue.class).getValue(); rankSum += rank; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java index 98bf5c7d91e21..8362c29da440d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -88,10 +90,12 @@ public static final class BuildTriads extends ReduceFunction implements Serializ private int[] edgeCache = new int[1024]; @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { int len = 0; - for (Record rec : records) { + Record rec = null; + while (records.hasNext()) { + rec = records.next(); final int e1 = rec.getField(1, IntValue.class).getValue(); for (int i = 0; i < len; i++) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java index 9c7707efc1dfa..5873581ab628c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java @@ -16,10 +16,12 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; import java.util.ArrayList; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -173,9 +175,9 @@ public BuildTriads() { } @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { // read the first edge - final Record rec = records.iterator().next(); + final Record rec = records.next(); // read the matching vertex rec.getFieldInto(0, this.matchVertex); // read the non-matching vertex and add it to the list @@ -186,7 +188,10 @@ public void reduce(Iterable records, Collector out) throws Excep int numEdges = 1; // while there are more edges - for (Record next : records) { + while (records.hasNext()) { + + // read the next edge + final Record next = records.next(); final StringValue myVertex; // obtain an object to store the non-matching vertex diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java index 578ef54edbdd7..c57cf6aab5802 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java @@ -16,12 +16,14 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.Map; import java.util.Set; import java.util.StringTokenizer; @@ -284,32 +286,32 @@ public static class FindShortestPath extends CoGroupFunction implements Serializ private final IntValue minLength = new IntValue(); @Override - public void coGroup(Iterable inputRecords, Iterable concatRecords, Collector out) { + public void coGroup(Iterator inputRecords, Iterator concatRecords, Collector out) { // init minimum length and minimum path - Record initial = null; + Record pathRec = null; StringValue path = null; - - if (inputRecords.iterator().hasNext()) { + if(inputRecords.hasNext()) { // path is in input paths - initial = inputRecords.iterator().next(); + pathRec = inputRecords.next(); } else { // path must be in concat paths - initial = concatRecords.iterator().next(); + pathRec = concatRecords.next(); } // get from node (common for all paths) - StringValue fromNode = initial.getField(0, StringValue.class); + StringValue fromNode = pathRec.getField(0, StringValue.class); // get to node (common for all paths) - StringValue toNode = initial.getField(1, StringValue.class); + StringValue toNode = pathRec.getField(1, StringValue.class); // get length of path - minLength.setValue(initial.getField(2, IntValue.class).getValue()); + minLength.setValue(pathRec.getField(2, IntValue.class).getValue()); // store path and hop count - path = new StringValue(initial.getField(4, StringValue.class)); + path = new StringValue(pathRec.getField(4, StringValue.class)); shortestPaths.add(path); - hopCnts.put(path, new IntValue(initial.getField(3, IntValue.class).getValue())); + hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); // find shortest path of all input paths - for (Record pathRec : inputRecords) { + while (inputRecords.hasNext()) { + pathRec = inputRecords.next(); IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { @@ -318,8 +320,7 @@ public void coGroup(Iterable inputRecords, Iterable concatRecord if(shortestPaths.add(path)) { hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); } - } - else if (length.getValue() < minLength.getValue()) { + } else if (length.getValue() < minLength.getValue()) { // path has minimum length minLength.setValue(length.getValue()); // clear lists @@ -333,7 +334,8 @@ else if (length.getValue() < minLength.getValue()) { } // find shortest path of all input and concatenated paths - for (Record pathRec : concatRecords) { + while (concatRecords.hasNext()) { + pathRec = concatRecords.next(); IntValue length = pathRec.getField(2, IntValue.class); if (length.getValue() == minLength.getValue()) { @@ -365,7 +367,7 @@ else if (length.getValue() < minLength.getValue()) { outputRecord.setField(4, shortestPath); out.collect(outputRecord); } - + hopCnts.clear(); shortestPaths.clear(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java index cab33d9c03cfa..68b89f3981b5b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -88,12 +90,12 @@ public static final class AggregatingReduce extends ReduceFunction implements Se private final DoubleValue sum = new DoubleValue(); @Override - public void reduce(Iterable pageWithPartialRank, Collector out) throws Exception { + public void reduce(Iterator pageWithPartialRank, Collector out) throws Exception { Record rec = null; double rankSum = 0.0; - for (Record next : pageWithPartialRank) { - rec = next; + while (pageWithPartialRank.hasNext()) { + rec = pageWithPartialRank.next(); rankSum += rec.getField(1, DoubleValue.class).getValue(); } sum.setValue(rankSum); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java index e4e8cc1f7128a..52f14c6123df1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -89,15 +91,15 @@ public static final class MinimumComponentIDReduce extends ReduceFunction implem private final LongValue minComponentId = new LongValue(); @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { - final Record first = records.iterator().next(); + final Record first = records.next(); final long vertexID = first.getField(0, LongValue.class).getValue(); long minimumComponentID = first.getField(1, LongValue.class).getValue(); - for (Record r : records) { - long candidateComponentID = r.getField(1, LongValue.class).getValue(); + while (records.hasNext()) { + long candidateComponentID = records.next().getField(1, LongValue.class).getValue(); if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java index f5a70efc85b2c..6efa2bf558d8f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.graph.pageRankUtil; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.java.record.functions.CoGroupFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; @@ -89,20 +91,20 @@ public void open(Configuration parameters) throws Exception { } @Override - public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, + public void coGroup(Iterator currentPageRankIterator, Iterator partialRanks, Collector collector) { - if (!currentPageRankIterator.iterator().hasNext()) { - long missingVertex = partialRanks.iterator().next().getField(0, LongValue.class).getValue(); + if (!currentPageRankIterator.hasNext()) { + long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - Record currentPageRank = currentPageRankIterator.iterator().next(); + Record currentPageRank = currentPageRankIterator.next(); long edges = 0; double summedRank = 0; - for (Record r : partialRanks) { - summedRank += r.getField(1, doubleInstance).getValue(); + while (partialRanks.hasNext()) { + summedRank += partialRanks.next().getField(1, doubleInstance).getValue(); edges++; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java index 80da9590fe340..66c8aaed9d5bb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java @@ -16,12 +16,14 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.kmeans; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.Plan; @@ -253,7 +255,7 @@ public static final class RecomputeClusterCenter extends ReduceFunction { * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterable points, Collector out) { + public void reduce(Iterator points, Collector out) { Record sum = sumPointsAndCount(points); sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); out.collect(sum); @@ -263,18 +265,18 @@ public void reduce(Iterable points, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterable points, Collector out) { + public void combine(Iterator points, Collector out) { out.collect(sumPointsAndCount(points)); } - private final Record sumPointsAndCount(Iterable dataPoints) { + private final Record sumPointsAndCount(Iterator dataPoints) { Record next = null; p.clear(); int count = 0; // compute coordinate vector sum and count - for (Record n : dataPoints) { - next = n; + while (dataPoints.hasNext()) { + next = dataPoints.next(); p.add(next.getField(1, Point.class)); count += next.getField(2, IntValue.class).getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java index d1fd0db64c38d..24539861ccdef 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java @@ -16,12 +16,14 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.kmeans; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.Plan; @@ -234,7 +236,7 @@ public static final class RecomputeClusterCenter extends ReduceFunction { * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterable points, Collector out) { + public void reduce(Iterator points, Collector out) { Record sum = sumPointsAndCount(points); sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); out.collect(sum); @@ -244,18 +246,18 @@ public void reduce(Iterable points, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterable points, Collector out) { + public void combine(Iterator points, Collector out) { out.collect(sumPointsAndCount(points)); } - private final Record sumPointsAndCount(Iterable dataPoints) { + private final Record sumPointsAndCount(Iterator dataPoints) { Record next = null; p.clear(); int count = 0; // compute coordinate vector sum and count - for (Record n : dataPoints) { - next = n; + while (dataPoints.hasNext()) { + next = dataPoints.next(); p.add(next.getField(1, Point.class)); count += next.getField(2, IntValue.class).getValue(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java index 3fe8bfe57043f..c4b974b21841b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java @@ -19,6 +19,7 @@ package org.apache.flink.test.recordJobs.kmeans.udfs; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; @@ -53,12 +54,14 @@ public class FindNearestCenter extends ReduceFunction implements Serializable { * 2: constant(1) (to enable combinable average computation in the following reducer) */ @Override - public void reduce(Iterable pointsWithDistance, Collector out) { + public void reduce(Iterator pointsWithDistance, Collector out) { double nearestDistance = Double.MAX_VALUE; int nearestClusterId = 0; // check all cluster centers - for ( Record res : pointsWithDistance) { + while (pointsWithDistance.hasNext()) { + Record res = pointsWithDistance.next(); + double distance = res.getField(3, DoubleValue.class).getValue(); // compare distances @@ -89,11 +92,12 @@ public void reduce(Iterable pointsWithDistance, Collector out) { * cluster centers. */ @Override - public void combine(Iterable pointsWithDistance, Collector out) { + public void combine(Iterator pointsWithDistance, Collector out) { double nearestDistance = Double.MAX_VALUE; // check all cluster centers - for (Record res : pointsWithDistance) { + while (pointsWithDistance.hasNext()) { + Record res = pointsWithDistance.next(); double distance = res.getField(3, DoubleValue.class).getValue(); // compare distances diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java index 7a137536bc7c3..4cc61f50438ba 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java @@ -19,6 +19,7 @@ package org.apache.flink.test.recordJobs.kmeans.udfs; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; @@ -47,7 +48,7 @@ public class RecomputeClusterCenter extends ReduceFunction implements Serializab * Compute the new position (coordinate vector) of a cluster center. */ @Override - public void reduce(Iterable dataPoints, Collector out) { + public void reduce(Iterator dataPoints, Collector out) { Record next = null; // initialize coordinate vector sum and count @@ -56,8 +57,8 @@ public void reduce(Iterable dataPoints, Collector out) { int count = 0; // compute coordinate vector sum and count - for (Record n : dataPoints) { - next = n; + while (dataPoints.hasNext()) { + next = dataPoints.next(); // get the coordinates and the count from the record double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); @@ -93,7 +94,7 @@ public void reduce(Iterable dataPoints, Collector out) { * Computes a pre-aggregated average value of a coordinate vector. */ @Override - public void combine(Iterable dataPoints, Collector out) { + public void combine(Iterator dataPoints, Collector out) { Record next = null; @@ -103,8 +104,8 @@ public void combine(Iterable dataPoints, Collector out) { int count = 0; // compute coordinate vector sum and count - for (Record n : dataPoints) { - next = n; + while (dataPoints.hasNext()) { + next = dataPoints.next(); // get the coordinates and the count from the record double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java index 573c87642e635..78e2c73415077 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java @@ -16,8 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; +import java.util.Iterator; + import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; import org.apache.flink.api.java.record.functions.JoinFunction; @@ -54,9 +57,9 @@ public static class DummyReduce extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable values, Collector out) { - for (Record r : values) { - out.collect(r); + public void reduce(Iterator values, Collector out) { + while (values.hasNext()) { + out.collect(values.next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java index e2cf6936b6413..378c2b67cc62a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java @@ -16,11 +16,13 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; import java.io.IOException; import java.text.DecimalFormat; import java.text.DecimalFormatSymbols; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -198,16 +200,17 @@ public void join(Record colRecord, Record nation, Collector out) throws } @ReduceOperator.Combinable - public static class Sum extends ReduceFunction { - + public static class Sum extends ReduceFunction + { private final DoubleValue d = new DoubleValue(); @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) throws Exception + { Record record = null; double sum = 0; - for (Record next : records) { - record = next; + while (records.hasNext()) { + record = records.next(); sum += record.getField(2, DoubleValue.class).getValue(); } @@ -217,7 +220,7 @@ record = next; } @Override - public void combine(Iterable records, Collector out) { + public void combine(Iterator records, Collector out) throws Exception { reduce(records,out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java index 6da83be955066..15640c0b57eb7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -173,12 +175,12 @@ public static class AggLiO extends ReduceFunction implements Serializable { * 2:SUM(EXTENDEDPRICE) */ @Override - public void reduce(Iterable values, Collector out) { + public void reduce(Iterator values, Collector out) { Record rec = null; double partExtendedPriceSum = 0; - for (Record next : values) { - rec = next; + while (values.hasNext()) { + rec = values.next(); partExtendedPriceSum += rec.getField(2, DoubleValue.class).getValue(); } @@ -191,7 +193,7 @@ public void reduce(Iterable values, Collector out) { * Creates partial sums on the price attribute for each data batch. */ @Override - public void combine(Iterable values, Collector out) { + public void combine(Iterator values, Collector out) { reduce(values, out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java index c83feeb6f520e..0db22cb2b1a59 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java @@ -16,6 +16,7 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; import java.text.ParseException; @@ -23,6 +24,7 @@ import java.util.Calendar; import java.util.Date; import java.util.GregorianCalendar; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -178,16 +180,17 @@ public void join(Record order, Record line, Collector out) public static class CountAgg extends ReduceFunction { @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { long count = 0; Record rec = null; - for (Record next : records) { - rec = next; - count++; + while(records.hasNext()) { + rec = records.next(); + count++; } - if(rec != null) { + if(rec != null) + { Tuple tuple = new Tuple(); tuple.addAttribute("" + count); rec.setField(1, tuple); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java index 6e2a72afcb44a..ffb0f2e3ce7ed 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -95,6 +97,7 @@ public static class AggCO extends ReduceFunction implements Serializable { private static final long serialVersionUID = 1L; private final IntValue integer = new IntValue(); + private Record record = new Record(); /** * Output Schema: @@ -103,23 +106,29 @@ public static class AggCO extends ReduceFunction implements Serializable { * */ @Override - public void reduce(Iterable records, Collector out) { - Record record = null; + public void reduce(Iterator records, Collector out) + throws Exception { + int count = 0; - for (Record next : records) { - record = next; - count += record.getField(0, integer).getValue(); + while (records.hasNext()) { + record = records.next(); + count+=record.getField(0, integer).getValue(); } integer.setValue(count); record.setField(0, integer); out.collect(record); } - - public void combine(Iterable records, Collector out) { + + /** + * Computes partial counts + */ + public void combine(Iterator records, Collector out) + throws Exception { reduce(records, out); } + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java index 0df4aeaa50c6b..d80dc17622125 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -219,12 +221,12 @@ public static class AntiJoinVisits extends CoGroupFunction implements Serializab * 2: AVG_DURATION */ @Override - public void coGroup(Iterable ranks, Iterable visits, Collector out) { + public void coGroup(Iterator ranks, Iterator visits, Collector out) { // Check if there is a entry in the visits relation - if (!visits.iterator().hasNext()) { - // Emit all rank pairs - for (Record r : ranks) { - out.collect(r); + if (!visits.hasNext()) { + while (ranks.hasNext()) { + // Emit all rank pairs + out.collect(ranks.next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java index d576401e21d29..f77ea61dc90cb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java @@ -16,8 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational.query1Util; +import java.util.Iterator; + import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.test.recordJobs.util.Tuple; import org.apache.flink.types.LongValue; @@ -30,7 +33,7 @@ public class GroupByReturnFlag extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) throws Exception { Record outRecord = new Record(); Tuple returnTuple = new Tuple(); @@ -38,7 +41,8 @@ public void reduce(Iterable records, Collector out) { double extendedPriceSum = 0.0; boolean first = true; - for (Record rec : records) { + while(records.hasNext()) { + Record rec = records.next(); Tuple t = rec.getField(1, Tuple.class); if(first) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java index 12c29764df050..49b385e1e1fee 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java @@ -16,8 +16,12 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.relational.query9Util; + +import java.util.Iterator; + import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; @@ -39,13 +43,15 @@ public class AmountAggregate extends ReduceFunction { * Value: amount * */ + @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) throws Exception + { Record record = null; float amount = 0; - for (Record next : records) { - record = next; + while (records.hasNext()) { + record = records.next(); StringValue value = record.getField(1, StringValue.class); amount += Float.parseFloat(value.toString()); } @@ -60,7 +66,8 @@ record = next; * Creates partial sums of "amount" for each data batch: */ @Override - public void combine(Iterable records, Collector out) { + public void combine(Iterator records, Collector out) throws Exception + { reduce(records, out); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java index 964b7b018027c..d41353daa3858 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java @@ -16,9 +16,11 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.sort; import java.io.Serializable; +import java.util.Iterator; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.Program; @@ -54,9 +56,9 @@ public static class IdentityReducer extends ReduceFunction implements Serializab private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { - Record next = records.iterator().next(); + Record next = records.next(); // Increments the first field of the first record of the reduce group by 100 and emit it IntValue incrVal = next.getField(0, IntValue.class); @@ -65,8 +67,8 @@ public void reduce(Iterable records, Collector out) { out.collect(next); // emit all remaining records - for (Record r : records){ - out.collect(r); + while (records.hasNext()) { + out.collect(records.next()); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java index 3958d1e2ae642..dcad7ac35f534 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java @@ -16,8 +16,10 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.wordcount; +import java.util.Iterator; import java.util.StringTokenizer; import org.apache.flink.api.common.JobExecutionResult; @@ -87,11 +89,11 @@ public static class CountWords extends ReduceFunction { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable records, Collector out) throws Exception { + public void reduce(Iterator records, Collector out) throws Exception { Record element = null; int sum = 0; - for (Record next : records) { - element = next; + while (records.hasNext()) { + element = records.next(); int cnt = element.getField(1, IntValue.class).getValue(); sum += cnt; } @@ -101,7 +103,7 @@ public void reduce(Iterable records, Collector out) throws Excep } @Override - public void combine(Iterable records, Collector out) throws Exception { + public void combine(Iterator records, Collector out) throws Exception { // the logic is the same as in the reduce function, so simply call the reduce method reduce(records, out); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java index d875f3d83c762..0240272f43180 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java @@ -16,10 +16,12 @@ * limitations under the License. */ + package org.apache.flink.test.recordJobs.wordcount; import java.io.IOException; import java.io.Serializable; +import java.util.Iterator; import java.util.Set; import java.util.StringTokenizer; @@ -126,12 +128,11 @@ public static class CountWords extends ReduceFunction implements Serializable { private final IntValue cnt = new IntValue(); @Override - public void reduce(Iterable records, Collector out) { + public void reduce(Iterator records, Collector out) { Record element = null; int sum = 0; - - for ( Record next : records ) { - element = next; + while (records.hasNext()) { + element = records.next(); IntValue i = element.getField(1, IntValue.class); sum += i.getValue(); } From 9ccf913fbe89d1b3fad0f8cf1bf6eaa1f2284125 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 30 Jul 2014 14:34:31 +0200 Subject: [PATCH 3/3] [FLINK-1023] Iterables over transient data throw an TraversableOnceException when iterated over again. --- .../common/functions/GenericCoGrouper.java | 3 - .../flink/util/TraversableOnceException.java | 28 +++ .../java/graph/EnumTrianglesBasic.java | 11 +- .../example/java/graph/EnumTrianglesOpt.java | 19 +- .../translation/TupleUnwrappingIterator.java | 10 +- .../java/record/operators/ReduceOperator.java | 62 +++-- .../record/CoGroupWrappingFunctionTest.java | 223 ++++++++++++++++++ .../record/ReduceWrappingFunctionTest.java | 67 +++++- .../runtime/operators/RegularPactTask.java | 1 - .../sort/CombiningUnilateralSortMerger.java | 11 +- .../runtime/util/KeyGroupedIterator.java | 20 +- .../util/MutableToRegularIteratorWrapper.java | 11 +- .../drivers/AllGroupReduceDriverTest.java | 4 +- .../drivers/GroupReduceDriverTest.java | 4 +- .../runtime/util/KeyGroupedIteratorTest.java | 43 +++- .../api/scala/functions/CoGroupFunction.scala | 14 +- .../api/scala/functions/ReduceFunction.scala | 20 +- .../api/scala/operators/CoGroupOperator.scala | 15 +- .../api/scala/operators/ReduceOperator.scala | 22 +- .../KMeansIterativeNepheleITCase.java | 3 +- .../DependencyConnectedComponentsITCase.java | 15 +- ...ntsWithParametrizableAggregatorITCase.java | 17 +- ...tsWithParametrizableConvergenceITCase.java | 13 +- .../ConnectedComponentsNepheleITCase.java | 5 +- .../CustomCompensatableDotProductCoGroup.java | 9 +- .../CustomRankCombiner.java | 12 +- .../javaApiOperators/GroupReduceITCase.java | 35 ++- .../graph/WorksetConnectedComponents.java | 2 +- 28 files changed, 550 insertions(+), 149 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/record/CoGroupWrappingFunctionTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java index 0d7b28f6923d8..edfbf89601da3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GenericCoGrouper.java @@ -16,12 +16,10 @@ * limitations under the License. */ - package org.apache.flink.api.common.functions; import org.apache.flink.util.Collector; - public interface GenericCoGrouper extends Function { /** @@ -34,5 +32,4 @@ public interface GenericCoGrouper extends Function { * @param out A collector that collects all output pairs. */ void coGroup(Iterable records1, Iterable records2, Collector out) throws Exception; - } diff --git a/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java b/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java new file mode 100644 index 0000000000000..a15c31cb1f07b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.util; + +public class TraversableOnceException extends RuntimeException { + + private static final long serialVersionUID = 7636881584773577290L; + + public TraversableOnceException() { + super("The Iterable can be iterated over only once. Only the first call to 'iterator()' will succeed."); + } +} diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java index 27b7e9ff31d28..2f9f911c98bab 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java @@ -19,6 +19,7 @@ package org.apache.flink.example.java.graph; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.operators.Order; @@ -153,19 +154,21 @@ private static class TriadBuilder extends GroupReduceFunction { private final Triad outTriad = new Triad(); @Override - public void reduce(Iterable edges, Collector out) throws Exception { + public void reduce(Iterable edgesIter, Collector out) throws Exception { + + final Iterator edges = edgesIter.iterator(); // clear vertex list vertices.clear(); // read first edge - Edge firstEdge = edges.iterator().next(); + Edge firstEdge = edges.next(); outTriad.setFirstVertex(firstEdge.getFirstVertex()); vertices.add(firstEdge.getSecondVertex()); // build and emit triads - for (Edge e : edges) { - Integer higherVertexId = e.getSecondVertex(); + while (edges.hasNext()) { + Integer higherVertexId = edges.next().getSecondVertex(); // combine vertex with all previously read vertices for (Integer lowerVertexId : vertices) { diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java index 39c92eddae681..8ae3daf90d554 100644 --- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java +++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java @@ -19,6 +19,7 @@ package org.apache.flink.example.java.graph; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.operators.Order; @@ -165,18 +166,19 @@ private static class DegreeCounter extends GroupReduceFunction edges, Collector out) { + public void reduce(Iterable edgesIter, Collector out) { + Iterator edges = edgesIter.iterator(); otherVertices.clear(); // get first edge - Edge edge = edges.iterator().next(); + Edge edge = edges.next(); Integer groupVertex = edge.getFirstVertex(); this.otherVertices.add(edge.getSecondVertex()); // get all other edges (assumes edges are sorted by second vertex) - for (Edge e : edges) { - edge = e; + while (edges.hasNext()) { + edge = edges.next(); Integer otherVertex = edge.getSecondVertex(); // collect unique vertices if(!otherVertices.contains(otherVertex) && otherVertex != groupVertex) { @@ -273,19 +275,20 @@ private static class TriadBuilder extends GroupReduceFunction { private final Triad outTriad = new Triad(); @Override - public void reduce(Iterable edges, Collector out) throws Exception { + public void reduce(Iterable edgesIter, Collector out) throws Exception { + final Iterator edges = edgesIter.iterator(); // clear vertex list vertices.clear(); // read first edge - Edge firstEdge = edges.iterator().next(); + Edge firstEdge = edges.next(); outTriad.setFirstVertex(firstEdge.getFirstVertex()); vertices.add(firstEdge.getSecondVertex()); // build and emit triads - for (Edge e : edges) { - Integer higherVertexId = e.getSecondVertex(); + while (edges.hasNext()) { + Integer higherVertexId = edges.next().getSecondVertex(); // combine vertex with all previously read vertices for(Integer lowerVertexId : vertices) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java index 47074282aaca6..c09f3a85c69fc 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java @@ -21,6 +21,7 @@ import java.util.Iterator; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.TraversableOnceException; /** * An iterator that reads 2-tuples (key value pairs) and returns only the values (second field). @@ -32,9 +33,11 @@ public class TupleUnwrappingIterator implements Iterator, Iterable, private K lastKey; private Iterator> iterator; + private boolean iteratorAvailable; public void set(Iterator> iterator) { this.iterator = iterator; + this.iteratorAvailable = true; } public K getLastKey() { @@ -60,6 +63,11 @@ public void remove() { @Override public Iterator iterator() { - return this; + if (iteratorAvailable) { + iteratorAvailable = false; + return this; + } else { + throw new TraversableOnceException(); + } } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java index 13289ec61dd79..bde190bfca4ba 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/record/operators/ReduceOperator.java @@ -37,6 +37,7 @@ import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.RecordOperator; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; +import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.api.java.operators.translation.WrappingFunction; @@ -61,6 +62,8 @@ public class ReduceOperator extends GroupReduceOperatorBase>[] keyTypes; + private final UserCodeWrapper originalFunction; + // -------------------------------------------------------------------------------------------- /** @@ -69,8 +72,11 @@ public class ReduceOperator extends GroupReduceOperatorBase>(wrapper)); + UserCodeWrapper original = new UserCodeObjectWrapper(udf); + UserCodeWrapper> wrapped = + new UserCodeObjectWrapper>(new WrappingReduceFunction(udf)); + + return new Builder(original, wrapped); } /** @@ -81,8 +87,11 @@ public static Builder builder(ReduceFunction udf) { * @param keyColumn The position of the key. */ public static Builder builder(ReduceFunction udf, Class> keyClass, int keyColumn) { - WrappingReduceFunction wrapper = new WrappingReduceFunction(udf); - return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn); + UserCodeWrapper original = new UserCodeObjectWrapper(udf); + UserCodeWrapper> wrapped = + new UserCodeObjectWrapper>(new WrappingReduceFunction(udf)); + + return new Builder(original, wrapped, keyClass, keyColumn); } /** @@ -91,8 +100,11 @@ public static Builder builder(ReduceFunction udf, Class> keyCla * @param udf The {@link ReduceFunction} implementation for this Reduce contract. */ public static Builder builder(Class udf) { - WrappingReduceFunction wrapper = new WrappingClassReduceFunction(udf); - return new Builder(new UserCodeObjectWrapper>(wrapper)); + UserCodeWrapper original = new UserCodeClassWrapper(udf); + UserCodeWrapper> wrapped = + new UserCodeObjectWrapper>(new WrappingClassReduceFunction(udf)); + + return new Builder(original, wrapped); } /** @@ -103,8 +115,11 @@ public static Builder builder(Class udf) { * @param keyColumn The position of the key. */ public static Builder builder(Class udf, Class> keyClass, int keyColumn) { - WrappingReduceFunction wrapper = new WrappingClassReduceFunction(udf); - return new Builder(new UserCodeObjectWrapper>(wrapper), keyClass, keyColumn); + UserCodeWrapper original = new UserCodeClassWrapper(udf); + UserCodeWrapper> wrapped = + new UserCodeObjectWrapper>(new WrappingClassReduceFunction(udf)); + + return new Builder(original, wrapped, keyClass, keyColumn); } /** @@ -112,8 +127,10 @@ public static Builder builder(Class udf, Class(function))); + setSemanticProperties(FunctionAnnotation.readSingleConstantAnnotations(originalFunction)); } // -------------------------------------------------------------------------------------------- @@ -138,7 +154,7 @@ public Class>[] getKeyClasses() { @Override public boolean isCombinable() { - return super.isCombinable() || getUserCodeWrapper().getUserCodeAnnotation(Combinable.class) != null; + return super.isCombinable() || originalFunction.getUserCodeAnnotation(Combinable.class) != null; } /** @@ -190,7 +206,8 @@ public boolean isCombinable() { public static class Builder { /* The required parameters */ - private final UserCodeWrapper> udf; + private final UserCodeWrapper originalUdf; + private final UserCodeWrapper> udfWrapper; private final List>> keyClasses; private final List keyColumns; @@ -203,10 +220,11 @@ public static class Builder { /** * Creates a Builder with the provided {@link ReduceFunction} implementation. * - * @param udf The {@link ReduceFunction} implementation for this Reduce contract. + * @param wrappedUdf The {@link ReduceFunction} implementation for this Reduce contract. */ - private Builder(UserCodeWrapper> udf) { - this.udf = udf; + private Builder(UserCodeWrapper originalUdf, UserCodeWrapper> wrappedUdf) { + this.originalUdf = originalUdf; + this.udfWrapper = wrappedUdf; this.keyClasses = new ArrayList>>(); this.keyColumns = new ArrayList(); this.inputs = new ArrayList>(); @@ -216,12 +234,16 @@ private Builder(UserCodeWrapper> udf) { /** * Creates a Builder with the provided {@link ReduceFunction} implementation. * - * @param udf The {@link ReduceFunction} implementation for this Reduce contract. + * @param wrappedUdf The {@link ReduceFunction} implementation for this Reduce contract. * @param keyClass The class of the key data type. * @param keyColumn The position of the key. */ - private Builder(UserCodeWrapper> udf, Class> keyClass, int keyColumn) { - this.udf = udf; + private Builder(UserCodeWrapper originalUdf, + UserCodeWrapper> wrappedUdf, + Class> keyClass, int keyColumn) + { + this.originalUdf = originalUdf; + this.udfWrapper = wrappedUdf; this.keyClasses = new ArrayList>>(); this.keyClasses.add(keyClass); this.keyColumns = new ArrayList(); @@ -337,7 +359,7 @@ public Builder name(String name) { */ public ReduceOperator build() { if (name == null) { - name = udf.getUserCodeClass().getName(); + name = udfWrapper.getUserCodeClass().getName(); } return new ReduceOperator(this); } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/record/CoGroupWrappingFunctionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/record/CoGroupWrappingFunctionTest.java new file mode 100644 index 0000000000000..a7ccf6ea407bd --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/record/CoGroupWrappingFunctionTest.java @@ -0,0 +1,223 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.record; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.lang3.SerializationUtils; +import org.apache.flink.api.common.functions.AbstractFunction; +import org.apache.flink.api.common.functions.GenericCoGrouper; +import org.apache.flink.api.common.operators.DualInputSemanticProperties; +import org.apache.flink.api.common.operators.util.FieldSet; +import org.apache.flink.api.common.operators.util.UserCodeWrapper; +import org.apache.flink.api.java.record.functions.CoGroupFunction; +import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; +import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; +import org.apache.flink.api.java.record.operators.CoGroupOperator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.Record; +import org.apache.flink.util.Collector; +import org.junit.Test; + +@SuppressWarnings("serial") +public class CoGroupWrappingFunctionTest { + + @SuppressWarnings("unchecked") + @Test + public void testWrappedCoGroupObject() { + try { + AtomicInteger methodCounter = new AtomicInteger(); + + CoGroupOperator coGroupOp = CoGroupOperator.builder(new TestCoGroupFunction(methodCounter), LongValue.class, 1, 2).build(); + + AbstractFunction cogrouper = (AbstractFunction) coGroupOp.getUserCodeWrapper().getUserCodeObject(); + + // test the method invocations + cogrouper.close(); + cogrouper.open(new Configuration()); + assertEquals(2, methodCounter.get()); + + // prepare the coGroup + final List target = new ArrayList(); + Collector collector = new Collector() { + @Override + public void collect(Record record) { + target.add(record); + } + @Override + public void close() {} + }; + + List source1 = new ArrayList(); + source1.add(new Record(new IntValue(42))); + source1.add(new Record(new IntValue(13))); + + List source2 = new ArrayList(); + source2.add(new Record(new LongValue(11))); + source2.add(new Record(new LongValue(17))); + + // test coGroup + ((GenericCoGrouper) cogrouper).coGroup(source1, source2, collector); + assertEquals(4, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(2).getField(0, LongValue.class)); + assertEquals(new LongValue(17), target.get(3).getField(0, LongValue.class)); + target.clear(); + + // test the serialization + SerializationUtils.clone(cogrouper); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testWrappedCoGroupClass() { + try { + CoGroupOperator coGroupOp = CoGroupOperator.builder(TestCoGroupFunction.class, LongValue.class, 1, 2).build(); + + UserCodeWrapper> udf = coGroupOp.getUserCodeWrapper(); + UserCodeWrapper> copy = SerializationUtils.clone(udf); + GenericCoGrouper cogrouper = copy.getUserCodeObject(); + + // prepare the coGpuü + final List target = new ArrayList(); + Collector collector = new Collector() { + @Override + public void collect(Record record) { + target.add(record); + } + @Override + public void close() {} + }; + + List source1 = new ArrayList(); + source1.add(new Record(new IntValue(42))); + source1.add(new Record(new IntValue(13))); + + List source2 = new ArrayList(); + source2.add(new Record(new LongValue(11))); + source2.add(new Record(new LongValue(17))); + + // test coGroup + cogrouper.coGroup(source1, source2, collector); + assertEquals(4, target.size()); + assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); + assertEquals(new IntValue(13), target.get(1).getField(0, IntValue.class)); + assertEquals(new LongValue(11), target.get(2).getField(0, LongValue.class)); + assertEquals(new LongValue(17), target.get(3).getField(0, LongValue.class)); + target.clear(); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testExtractSemantics() { + try { + { + CoGroupOperator coGroupOp = CoGroupOperator.builder(new TestCoGroupFunction(), LongValue.class, 1, 2).build(); + + DualInputSemanticProperties props = coGroupOp.getSemanticProperties(); + FieldSet fw2 = props.getForwardedField1(2); + FieldSet fw4 = props.getForwardedField2(4); + + assertNotNull(fw2); + assertNotNull(fw4); + assertEquals(1, fw2.size()); + assertEquals(1, fw4.size()); + assertTrue(fw2.contains(2)); + assertTrue(fw4.contains(4)); + } + { + CoGroupOperator coGroupOp = CoGroupOperator.builder(TestCoGroupFunction.class, LongValue.class, 1, 2).build(); + + DualInputSemanticProperties props = coGroupOp.getSemanticProperties(); + FieldSet fw2 = props.getForwardedField1(2); + FieldSet fw4 = props.getForwardedField2(4); + + assertNotNull(fw2); + assertNotNull(fw4); + assertEquals(1, fw2.size()); + assertEquals(1, fw4.size()); + assertTrue(fw2.contains(2)); + assertTrue(fw4.contains(4)); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + + @ConstantFieldsFirst(2) + @ConstantFieldsSecond(4) + public static class TestCoGroupFunction extends CoGroupFunction { + + private final AtomicInteger methodCounter; + + private TestCoGroupFunction(AtomicInteger methodCounter) { + this.methodCounter= methodCounter; + } + + public TestCoGroupFunction() { + methodCounter = new AtomicInteger(); + } + + + @Override + public void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception { + while (records1.hasNext()) { + out.collect(records1.next()); + } + while (records2.hasNext()) { + out.collect(records2.next()); + } + } + + @Override + public void close() throws Exception { + methodCounter.incrementAndGet(); + super.close(); + } + + @Override + public void open(Configuration parameters) throws Exception { + methodCounter.incrementAndGet(); + super.open(parameters); + } + }; +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java index 78331cef5ea1d..2b28e226f1718 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/record/ReduceWrappingFunctionTest.java @@ -18,8 +18,7 @@ package org.apache.flink.api.java.record; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; import java.util.ArrayList; import java.util.Iterator; @@ -30,9 +29,13 @@ import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; +import org.apache.flink.api.common.operators.SingleInputSemanticProperties; +import org.apache.flink.api.common.operators.util.FieldSet; import org.apache.flink.api.common.operators.util.UserCodeWrapper; +import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.record.functions.ReduceFunction; import org.apache.flink.api.java.record.operators.ReduceOperator; +import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.IntValue; import org.apache.flink.types.LongValue; @@ -126,7 +129,7 @@ public void close() {} source.add(new Record(new IntValue(13), new LongValue(17))); // test reduce - ((GenericGroupReduce) reducer).reduce(source, collector); + reducer.reduce(source, collector); assertEquals(2, target.size()); assertEquals(new IntValue(42), target.get(0).getField(0, IntValue.class)); assertEquals(new LongValue(11), target.get(0).getField(1, LongValue.class)); @@ -149,8 +152,66 @@ public void close() {} } } + @Test + public void testExtractSemantics() { + try { + { + ReduceOperator reduceOp = ReduceOperator.builder(new TestReduceFunction()).build(); + + SingleInputSemanticProperties props = reduceOp.getSemanticProperties(); + FieldSet fw2 = props.getForwardedField(2); + FieldSet fw4 = props.getForwardedField(4); + + assertNotNull(fw2); + assertNotNull(fw4); + assertEquals(1, fw2.size()); + assertEquals(1, fw4.size()); + assertTrue(fw2.contains(2)); + assertTrue(fw4.contains(4)); + } + { + ReduceOperator reduceOp = ReduceOperator.builder(TestReduceFunction.class).build(); + + SingleInputSemanticProperties props = reduceOp.getSemanticProperties(); + FieldSet fw2 = props.getForwardedField(2); + FieldSet fw4 = props.getForwardedField(4); + + assertNotNull(fw2); + assertNotNull(fw4); + assertEquals(1, fw2.size()); + assertEquals(1, fw4.size()); + assertTrue(fw2.contains(2)); + assertTrue(fw4.contains(4)); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testCombinable() { + try { + { + ReduceOperator reduceOp = ReduceOperator.builder(new TestReduceFunction()).build(); + assertTrue(reduceOp.isCombinable()); + } + { + ReduceOperator reduceOp = ReduceOperator.builder(TestReduceFunction.class).build(); + assertTrue(reduceOp.isCombinable()); + } + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + // -------------------------------------------------------------------------------------------- + @Combinable + @ConstantFields({2, 4}) public static class TestReduceFunction extends ReduceFunction { private final AtomicInteger methodCounter; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 45fe05a0e203d..3fe808bef448a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -689,7 +689,6 @@ protected S initStub(Class stubSuperClass) throws Exception { S stub = config.getStubWrapper(this.userCodeClassLoader).getUserCodeObject(stubSuperClass, this.userCodeClassLoader); // check if the class is a subclass, if the check is required if (stubSuperClass != null && !stubSuperClass.isAssignableFrom(stub.getClass())) { - Thread.dumpStack(); throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + stubSuperClass.getName() + "' as is required."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 76326d423270d..a950630b3a67a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -47,6 +47,7 @@ import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TraversableOnceException; /** @@ -511,6 +512,8 @@ private static final class CombineValueIterator implements Iterator, Itera private int last; // the position of the last value to be returned private int position; // the position of the next value to be returned + + private boolean iteratorAvailable; /** * Creates an iterator over the values in a BufferSortable. @@ -534,6 +537,7 @@ public CombineValueIterator(InMemorySorter buffer, E instance) { public void set(int first, int last) { this.last = last; this.position = first; + this.iteratorAvailable = true; } @Override @@ -566,7 +570,12 @@ public void remove() { @Override public Iterator iterator() { - return this; + if (iteratorAvailable) { + iteratorAvailable = false; + return this; + } else { + throw new TraversableOnceException(); + } } }; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java index f583593b18cf1..be43cc8bbb521 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TraversableOnceException; /** * The KeyValueIterator returns a key and all values that belong to the key (share the same key). @@ -98,6 +99,8 @@ public boolean nextKey() throws IOException return false; } } + + this.valuesIterator.iteratorAvailable = true; // Whole value-iterator was read and a new key is available. if (this.lookAheadHasNext) { @@ -151,19 +154,20 @@ public ValuesIterator getValues() { // -------------------------------------------------------------------------------------------- - public final class ValuesIterator implements Iterator, Iterable - { + public final class ValuesIterator implements Iterator, Iterable { + private final TypeSerializer serializer = KeyGroupedIterator.this.serializer; private final TypeComparator comparator = KeyGroupedIterator.this.comparator; private E staging = this.serializer.createInstance(); private boolean currentIsUnconsumed = false; + private boolean iteratorAvailable = true; + private ValuesIterator() {} @Override - public boolean hasNext() - { + public boolean hasNext() { if (KeyGroupedIterator.this.current == null || KeyGroupedIterator.this.lookAheadHasNext) { return false; } @@ -224,7 +228,13 @@ public void remove() { @Override public Iterator iterator() { - return this; + if (iteratorAvailable) { + iteratorAvailable = false; + return this; + } + else { + throw new TraversableOnceException(); + } } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java index 0558e47ea1f2e..14e4ae8bde2e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TraversableOnceException; /** * This class wraps a {@link MutableObjectIterator} into a regular {@link Iterator}. @@ -39,6 +40,8 @@ public class MutableToRegularIteratorWrapper implements Iterator, Iterable private T current, next; private boolean currentIsAvailable; + + private boolean iteratorAvailable = true; public MutableToRegularIteratorWrapper(MutableObjectIterator source, TypeSerializer serializer) { this.source = source; @@ -88,6 +91,12 @@ public void remove() { @Override public Iterator iterator() { - return this; + if (iteratorAvailable) { + iteratorAvailable = false; + return this; + } + else { + throw new TraversableOnceException(); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java index f13adab51a459..d607a0b6bd26a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java @@ -154,7 +154,7 @@ public static final class ConcatSumReducer extends GroupReduceFunction> values, Collector> out) { - Tuple2 current = values.iterator().next(); + Tuple2 current = new Tuple2("", 0); for (Tuple2 next : values) { next.f0 = current.f0 + next.f0; @@ -170,7 +170,7 @@ public static final class ConcatSumMutableReducer extends GroupReduceFunction> values, Collector> out) { - Tuple2 current = values.iterator().next(); + Tuple2 current = new Tuple2(new StringValue(""), new IntValue(0)); for (Tuple2 next : values) { next.f0.append(current.f0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java index 92c0edcfc9a14..dff695a972c88 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java @@ -151,7 +151,7 @@ public static final class ConcatSumReducer extends GroupReduceFunction> values, Collector> out) { - Tuple2 current = values.iterator().next(); + Tuple2 current = new Tuple2("", 0); for (Tuple2 next : values) { next.f0 = current.f0 + next.f0; @@ -167,7 +167,7 @@ public static final class ConcatSumMutableReducer extends GroupReduceFunction> values, Collector> out) { - Tuple2 current = values.iterator().next(); + Tuple2 current = new Tuple2(new StringValue(""), new IntValue(0)); for (Tuple2 next : values) { next.f0.append(current.f0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java index e9010d23128d4..27fa540548be7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java @@ -31,6 +31,7 @@ import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TraversableOnceException; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -38,17 +39,15 @@ /** * Test for the key grouped iterator, which advances in windows containing the same key and provides a sub-iterator * over the records with the same key. - * */ -public class KeyGroupedIteratorTest -{ +public class KeyGroupedIteratorTest { + private MutableObjectIterator sourceIter; // the iterator that provides the input private KeyGroupedIterator psi; // the grouping iterator, progressing in key steps @Before - public void setup() - { + public void setup() { final ArrayList source = new ArrayList(); // add elements to the source @@ -91,8 +90,7 @@ public Record next(Record reuse) throws IOException { } @Test - public void testNextKeyOnly() throws Exception - { + public void testNextKeyOnly() throws Exception { try { Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(1)))); @@ -131,6 +129,8 @@ public void testFullIterationThroughAllValues() throws IOException try { // Key 1, Value A Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(1)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 1, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -139,6 +139,8 @@ public void testFullIterationThroughAllValues() throws IOException // Key 2, Value B Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(2)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 2, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -147,6 +149,8 @@ public void testFullIterationThroughAllValues() throws IOException // Key 3, Values C, D Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(3)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 3, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -173,6 +177,8 @@ public void testFullIterationThroughAllValues() throws IOException // Key 4, Values E, F, G Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(4)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 4, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -193,6 +199,8 @@ public void testFullIterationThroughAllValues() throws IOException // Key 5, Values H, I, J, K, L Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(5)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 5, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -231,6 +239,7 @@ public void testFullIterationThroughAllValues() throws IOException Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + Assert.assertNull(this.psi.getValues()); } catch (Exception e) { e.printStackTrace(); Assert.fail("The test encountered an unexpected exception."); @@ -244,12 +253,18 @@ public void testMixedProgress() throws Exception // Progression only via nextKey() and hasNext() - Key 1, Value A Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); // Progression only through nextKey() - Key 2, Value B Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); // Progression first though haNext() and next(), then through hasNext() - Key 3, Values C, D Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(3)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 3, this.psi.getCurrent().getField(0, IntValue.class).getValue()); @@ -260,6 +275,8 @@ public void testMixedProgress() throws Exception // Progression first via next() only, then hasNext() only Key 4, Values E, F, G Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertEquals("KeyGroupedIterator returned a wrong value.", new StringValue("E"), this.psi.getValues().next().getField(1, StringValue.class)); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); @@ -270,6 +287,8 @@ public void testMixedProgress() throws Exception Assert.assertTrue("KeyGroupedIterator returned a wrong key.", this.psi.getComparatorWithCurrentReference().equalToReference(new Record(new IntValue(5)))); Assert.assertEquals("KeyGroupedIterator returned a wrong key.", 5, this.psi.getCurrent().getField(0, IntValue.class).getValue()); Assert.assertEquals("KeyGroupedIterator returned a wrong value.", new StringValue("I"), this.psi.getValues().next().getField(1, StringValue.class)); + Assert.assertTrue(hasIterator(this.psi.getValues())); + Assert.assertFalse(hasIterator(this.psi.getValues())); Assert.assertTrue("KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); // end @@ -355,4 +374,14 @@ public StringValue getString() { return string; } } + + public boolean hasIterator(Iterable iterable) { + try { + iterable.iterator(); + return true; + } + catch (TraversableOnceException e) { + return false; + } + } } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala index 82d71a5b5c403..6c7e93b285dc2 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.functions -import java.lang.{Iterable => JIterable} +import java.util.{Iterator => JIterator} import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT} import org.apache.flink.api.scala.analysis.UDF2 @@ -53,9 +53,9 @@ abstract class CoGroupFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends } abstract class CoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Out] { - override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) - val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) + override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords) + val firstRightRecord = rightIterator.initialize(rightRecords) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) @@ -72,11 +72,11 @@ abstract class CoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGr } abstract class FlatCoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Iterator[Out]] { - override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) + override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords) outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo) - val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) + val firstRightRecord = rightIterator.initialize(rightRecords) outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) val output = apply(leftIterator, rightIterator) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala index 84765a5788795..de7b7d1e41d26 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.scala.functions import scala.Iterator -import java.lang.{Iterable => JIterable} +import java.util.{Iterator => JIterator} import org.apache.flink.api.scala.analysis.{UDTSerializer, FieldSelector, UDT} import org.apache.flink.api.scala.analysis.UDF1 @@ -47,12 +47,12 @@ abstract class ReduceFunctionBase[In: UDT, Out: UDT] extends JReduceFunction wit abstract class ReduceFunction[In: UDT] extends ReduceFunctionBase[In, In] with Function2[In, In, In] { - override def combine(records: JIterable[Record], out: Collector[Record]) = { + override def combine(records: JIterator[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(records: JIterable[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records.iterator()) + override def reduce(records: JIterator[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = reduceIterator.reduce(apply) @@ -63,8 +63,8 @@ abstract class ReduceFunction[In: UDT] extends ReduceFunctionBase[In, In] with F } abstract class GroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] { - override def reduce(records: JIterable[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records.iterator()) + override def reduce(records: JIterator[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = apply(reduceIterator) @@ -75,8 +75,8 @@ abstract class GroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase } abstract class CombinableGroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] { - override def combine(records: JIterable[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records.iterator()) + override def combine(records: JIterator[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = combine(reduceIterator) @@ -85,8 +85,8 @@ abstract class CombinableGroupReduceFunction[In: UDT, Out: UDT] extends ReduceFu out.collect(reduceRecord) } - override def reduce(records: JIterable[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records.iterator()) + override def reduce(records: JIterator[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = reduce(reduceIterator) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala index f6164966b3a3a..6a71bb76eb71e 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala @@ -22,7 +22,6 @@ import language.experimental.macros import scala.reflect.macros.Context import java.util.{ Iterator => JIterator } -import java.lang.{ Iterable => JIterable } import org.apache.flink.types.Record import org.apache.flink.util.Collector @@ -54,7 +53,7 @@ class CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn](val leftKeySelection: Lis def flatMap[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CoGroupMacros.flatMap[LeftIn, RightIn, Out] } -class NoKeyCoGroupBuilder(s: JCoGroupFunction) extends CoGroupOperator.Builder(new UserCodeObjectWrapper(s)) +class NoKeyCoGroupBuilder(s: JCoGroupFunction) extends CoGroupOperator.Builder(new UserCodeObjectWrapper(new CoGroupOperator.WrappingCoGroupFunction(s))) object CoGroupMacros { @@ -109,10 +108,10 @@ object CoGroupMacros { new CoGroupFunctionBase[LeftIn, RightIn, Out] { - override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { + override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) - val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) + val firstLeftRecord = leftIterator.initialize(leftRecords) + val firstRightRecord = rightIterator.initialize(rightRecords) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) @@ -182,9 +181,9 @@ object CoGroupMacros { new CoGroupFunctionBase[LeftIn, RightIn, Out] { - override def coGroup(leftRecords: JIterable[Record], rightRecords: JIterable[Record], out: Collector[Record]) = { - val firstLeftRecord = leftIterator.initialize(leftRecords.iterator()) - val firstRightRecord = rightIterator.initialize(rightRecords.iterator()) + override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = { + val firstLeftRecord = leftIterator.initialize(leftRecords) + val firstRightRecord = rightIterator.initialize(rightRecords) if (firstRightRecord != null) { outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala index 1fa62bc1cc762..f25b5a34b832a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala @@ -16,14 +16,12 @@ * limitations under the License. */ - package org.apache.flink.api.scala.operators import language.experimental.macros import scala.language.reflectiveCalls import scala.reflect.macros.Context -import java.lang.{ Iterable => JIterable } import java.util.{ Iterator => JIterator } import org.apache.flink.api.scala.DataSet @@ -127,12 +125,11 @@ object ReduceMacros { implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice new ReduceFunctionBase[In, In] { - override def combine(records: JIterable[Record], out: Collector[Record]) = { + override def combine(records: JIterator[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(recordsIterable: JIterable[Record], out: Collector[Record]) = { - val records: JIterator[Record] = recordsIterable.iterator() + override def reduce(records: JIterator[Record], out: Collector[Record]) = { if (records.hasNext) { val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) @@ -193,8 +190,8 @@ object ReduceMacros { implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice new ReduceFunctionBase[In, Out] { - override def reduce(recordsIterable: JIterable[Record], out: Collector[Record]) = { - val records: JIterator[Record] = recordsIterable.iterator() + override def reduce(recordsIterable: JIterator[Record], out: Collector[Record]) = { + val records: JIterator[Record] = recordsIterable if (records.hasNext) { val firstRecord = reduceIterator.initialize(records) @@ -252,12 +249,12 @@ object ReduceMacros { implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice new ReduceFunctionBase[In, In] { - override def combine(records: JIterable[Record], out: Collector[Record]) = { + override def combine(records: JIterator[Record], out: Collector[Record]) = { reduce(records, out) } - override def reduce(records: JIterable[Record], out: Collector[Record]) = { - val firstRecord = reduceIterator.initialize(records.iterator()) + override def reduce(records: JIterator[Record], out: Collector[Record]) = { + val firstRecord = reduceIterator.initialize(records) reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo) val output = fun.splice.apply(reduceIterator) @@ -323,9 +320,8 @@ object ReduceMacros { this.countPosition = udf.getOutputLength - 1; } - override def reduce(recordsIterable: JIterable[Record], result: Collector[Record]) : Unit = { + override def reduce(records: JIterator[Record], result: Collector[Record]) : Unit = { - val records: JIterator[Record] = recordsIterable.iterator() var record : Record = null var counter: Int = 0 while (records.hasNext()) { @@ -339,7 +335,7 @@ object ReduceMacros { result.collect(record) } - override def combine(records: JIterable[Record], result: Collector[Record]) : Unit = { + override def combine(records: JIterator[Record], result: Collector[Record]) : Unit = { reduce(records, result) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java index 4b3a4bf30625a..2217679161312 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.record.io.CsvInputFormat; +import org.apache.flink.api.java.record.operators.ReduceOperator.WrappingReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparatorFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; import org.apache.flink.configuration.Configuration; @@ -252,7 +253,7 @@ private static JobTaskVertex createReducer(JobGraph jobGraph, int numSubTasks, T tailConfig.setOutputSerializer(outputSerializer); // the udf - tailConfig.setStubWrapper(new UserCodeObjectWrapper(new RecomputeClusterCenter())); + tailConfig.setStubWrapper(new UserCodeObjectWrapper(new WrappingReduceFunction(new RecomputeClusterCenter()))); return tail; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java index 0db1b81291008..ae1ad51099d99 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java @@ -207,19 +207,18 @@ public static final class MinimumReduce extends GroupReduceFunction @Override public void reduce(Iterable> values, Collector> out) { + Long vertexId = 0L; + Long minimumCompId = Long.MAX_VALUE; - final Tuple2 first = values.iterator().next(); - final Long vertexId = first.f0; - Long minimumCompId = first.f1; - - for ( Tuple2 value : values ) { + for (Tuple2 value: values) { + vertexId = value.f0; Long candidateCompId = value.f1; - if ( candidateCompId < minimumCompId ) { + if (candidateCompId < minimumCompId) { minimumCompId = candidateCompId; } } - resultVertex.setField(vertexId, 0); - resultVertex.setField(minimumCompId, 1); + resultVertex.f0 = vertexId; + resultVertex.f1 = minimumCompId; out.collect(resultVertex); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java index 4146475ee9e50..01b193822df09 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java @@ -156,27 +156,26 @@ public Tuple2 join(Tuple2 vertexWithCompId, } } - public static final class MinimumReduce extends GroupReduceFunction - , Tuple2> { + public static final class MinimumReduce extends GroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; - final Tuple2 resultVertex = new Tuple2(); + + private final Tuple2 resultVertex = new Tuple2(); @Override public void reduce(Iterable> values, Collector> out) { - - final Tuple2 first = values.iterator().next(); - final Long vertexId = first.f0; - Long minimumCompId = first.f1; + Long vertexId = 0L; + Long minimumCompId = Long.MAX_VALUE; for (Tuple2 value: values) { + vertexId = value.f0; Long candidateCompId = value.f1; if (candidateCompId < minimumCompId) { minimumCompId = candidateCompId; } } - resultVertex.setField(vertexId, 0); - resultVertex.setField(minimumCompId, 1); + resultVertex.f0 = vertexId; + resultVertex.f1 = minimumCompId; out.collect(resultVertex); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java index f8396618c07dd..0714558bd01e0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java @@ -159,19 +159,18 @@ public static final class MinimumReduce extends GroupReduceFunction @Override public void reduce(Iterable> values, Collector> out) { + Long vertexId = 0L; + Long minimumCompId = Long.MAX_VALUE; - final Tuple2 first = values.iterator().next(); - final Long vertexId = first.f0; - Long minimumCompId = first.f1; - - for (Tuple2 value : values) { + for (Tuple2 value: values) { + vertexId = value.f0; Long candidateCompId = value.f1; if (candidateCompId < minimumCompId) { minimumCompId = candidateCompId; } } - resultVertex.setField(vertexId, 0); - resultVertex.setField(minimumCompId, 1); + resultVertex.f0 = vertexId; + resultVertex.f1 = minimumCompId; out.collect(resultVertex); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java index 3060b68458aa8..4fd22a30742d6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.aggregators.LongSumAggregator; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; +import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -31,6 +32,8 @@ import org.apache.flink.api.java.record.io.CsvInputFormat; import org.apache.flink.api.java.record.io.CsvOutputFormat; import org.apache.flink.api.java.record.io.FileOutputFormat; +import org.apache.flink.api.java.record.operators.ReduceOperator.WrappingReduceFunction; +import org.apache.flink.api.java.record.operators.ReduceOperator.WrappingClassReduceFunction; import org.apache.flink.api.java.typeutils.runtime.record.RecordComparatorFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory; import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory; @@ -317,7 +320,7 @@ private static JobTaskVertex createIterationIntermediate(JobGraph jobGraph, int intermediateConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); intermediateConfig.setDriverComparator(comparator, 0); intermediateConfig.setStubWrapper( - new UserCodeClassWrapper(MinimumComponentIDReduce.class)); + new UserCodeObjectWrapper(new WrappingClassReduceFunction(MinimumComponentIDReduce.class))); } return intermediate; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index e4dbcc871dc4c..51167fc945123 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -18,6 +18,7 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank; +import java.util.Iterator; import java.util.Set; import org.apache.flink.api.common.functions.AbstractFunction; @@ -81,15 +82,17 @@ public void open(Configuration parameters) throws Exception { } @Override - public void coGroup(Iterable currentPageRankIterator, Iterable partialRanks, + public void coGroup(Iterable currentPageRankIterable, Iterable partialRanks, Collector collector) { - if (!currentPageRankIterator.iterator().hasNext()) { + final Iterator currentPageRankIterator = currentPageRankIterable.iterator(); + + if (!currentPageRankIterator.hasNext()) { long missingVertex = partialRanks.iterator().next().getVertexID(); throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!"); } - VertexWithRankAndDangling currentPageRank = currentPageRankIterator.iterator().next(); + VertexWithRankAndDangling currentPageRank = currentPageRankIterator.next(); long edges = 0; double summedRank = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java index b9e2391c6985f..755b50d3d3f3f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java @@ -18,6 +18,8 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank; +import java.util.Iterator; + import org.apache.flink.api.common.functions.AbstractFunction; import org.apache.flink.api.common.functions.GenericCombine; import org.apache.flink.api.common.functions.GenericGroupReduce; @@ -37,13 +39,15 @@ public void reduce(Iterable records, Collector o } @Override - public void combine(Iterable records, Collector out) throws Exception { - VertexWithRank next = records.iterator().next(); + public void combine(Iterable recordsIterable, Collector out) throws Exception { + final Iterator records = recordsIterable.iterator(); + + VertexWithRank next = records.next(); this.accumulator.setVertexID(next.getVertexID()); double rank = next.getRank(); - for (VertexWithRank r : records) { - rank += r.getRank(); + while (records.hasNext()) { + rank += records.next().getRank(); } this.accumulator.setRank(rank); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index d34f205a9d6eb..a86159b6c6c69 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -21,6 +21,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; +import java.util.Iterator; import java.util.LinkedList; import org.apache.flink.api.common.operators.Order; @@ -438,20 +439,21 @@ public static class Tuple3SortedGroupReduce extends GroupReduceFunction> values, Collector> out) { - - Tuple3 t = values.iterator().next(); - - int sum = t.f0; - long key = t.f1; - String concat = t.f2; + int sum = 0; + long key = 0; + StringBuilder concat = new StringBuilder(); for (Tuple3 next : values) { - sum += next.f0; - concat += "-" + next.f2; + key = next.f1; + concat.append(next.f2).append("-"); } - out.collect(new Tuple3(sum, key, concat)); + if (concat.length() > 0) { + concat.setLength(concat.length() - 1); + } + + out.collect(new Tuple3(sum, key, concat.toString())); } } @@ -483,17 +485,18 @@ public static class CustomTypeGroupReduce extends GroupReduceFunction values, Collector out) { + final Iterator iter = values.iterator(); CustomType o = new CustomType(); - CustomType c = values.iterator().next(); + CustomType c = iter.next(); o.myString = "Hello!"; o.myInt = c.myInt; o.myLong = c.myLong; - for ( CustomType next : values) { + while (iter.hasNext()) { + CustomType next = iter.next(); o.myLong += next.myLong; - } out.collect(o); @@ -545,13 +548,7 @@ public static class AllAddingCustomTypeGroupReduce extends GroupReduceFunction values, Collector out) { - CustomType o = new CustomType(); - CustomType c = values.iterator().next(); - - o.myString = "Hello!"; - o.myInt = c.myInt; - o.myLong = c.myLong; - + CustomType o = new CustomType(0, 0, "Hello!"); for (CustomType next : values) { o.myInt += next.myInt; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java index 52f14c6123df1..07fd658bb3c04 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java @@ -79,7 +79,7 @@ public void join(Record vertexWithComponent, Record edge, Collector out) } /** - * Minimum aggregation over (Vertex-ID, Component-ID) pairs, selecting the pair with the smallest Comonent-ID. + * Minimum aggregation over (Vertex-ID, Component-ID) pairs, selecting the pair with the smallest Component-ID. */ @Combinable @ConstantFields(0)