From 2a3905eb7e7465df08f707ad6f439cb0426a6aab Mon Sep 17 00:00:00 2001 From: mjsax Date: Thu, 25 Jun 2015 01:49:23 +0200 Subject: [PATCH] migrated test from execute() to collect() -> for package 'org.apache.flink.test.javaApiOperators' deaktivated instable test (see comment section https://issues.apache.org/jira/browse/FLINK-2275) --- .../connectors/kafka/KafkaITCase.java | 2 + .../javaApiOperators/AggregateITCase.java | 59 +-- .../test/javaApiOperators/CoGroupITCase.java | 8 +- .../test/javaApiOperators/CrossITCase.java | 140 +++-- .../test/javaApiOperators/DataSinkITCase.java | 38 +- .../javaApiOperators/DataSourceITCase.java | 22 +- .../test/javaApiOperators/DistinctITCase.java | 100 ++-- .../test/javaApiOperators/FilterITCase.java | 83 ++- .../test/javaApiOperators/FirstNITCase.java | 57 +- .../test/javaApiOperators/FlatMapITCase.java | 82 ++- .../javaApiOperators/GroupCombineITCase.java | 204 +++---- .../javaApiOperators/GroupReduceITCase.java | 497 +++++++++--------- .../test/javaApiOperators/JoinITCase.java | 320 +++++------ .../test/javaApiOperators/MapITCase.java | 95 ++-- .../javaApiOperators/PartitionITCase.java | 75 +-- .../test/javaApiOperators/ProjectITCase.java | 25 +- .../test/javaApiOperators/ReduceITCase.java | 141 +++-- .../ReplicatingDataSourceITCase.java | 35 +- .../javaApiOperators/SortPartitionITCase.java | 96 ++-- .../javaApiOperators/SumMinMaxITCase.java | 43 +- .../test/javaApiOperators/TypeHintITCase.java | 65 +-- .../test/javaApiOperators/UnionITCase.java | 46 +- .../util/CollectionDataSets.java | 14 + 23 files changed, 1001 insertions(+), 1246 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java index 2af56c1503126..4b763b2b8bccf 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java @@ -75,6 +75,7 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -224,6 +225,7 @@ public void testOffsetManipulation() { * */ @Test + @Ignore public void testPersistentSourceWithOffsetUpdates() throws Exception { LOG.info("Starting testPersistentSourceWithOffsetUpdates()"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java index ea7fc5a3c05e6..d02f2288f7f34 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java @@ -18,17 +18,15 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -38,31 +36,15 @@ public class AggregateITCase extends MultipleProgramsTestBase { - public AggregateITCase(TestExecutionMode mode){ + public AggregateITCase(TestExecutionMode mode) { super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testFullAggregate() throws Exception { /* - * Full Aggregate - */ + * Full Aggregate + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -70,36 +52,38 @@ public void testFullAggregate() throws Exception { DataSet> aggregateDs = ds .aggregate(Aggregations.SUM, 0) .and(Aggregations.MAX, 1) - .project(0, 1); + .project(0, 1); + + List> result = aggregateDs.collect(); - aggregateDs.writeAsCsv(resultPath); - env.execute(); + String expected = "231,6\n"; - expected = "231,6\n"; + compareResultAsTuples(result, expected); } @Test public void testGroupedAggregate() throws Exception { /* - * Grouped Aggregate - */ + * Grouped Aggregate + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> aggregateDs = ds.groupBy(1) .aggregate(Aggregations.SUM, 0) - .project(1, 0); + .project(1, 0); - aggregateDs.writeAsCsv(resultPath); - env.execute(); + List> result = aggregateDs.collect(); - expected = "1,1\n" + + String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; + + compareResultAsTuples(result, expected); } @Test @@ -114,11 +98,12 @@ public void testNestedAggregate() throws Exception { DataSet> aggregateDs = ds.groupBy(1) .aggregate(Aggregations.MIN, 0) .aggregate(Aggregations.MIN, 0) - .project(0); + .project(0); + + List> result = aggregateDs.collect(); - aggregateDs.writeAsCsv(resultPath); - env.execute(); + String expected = "1\n"; - expected = "1\n"; + compareResultAsTuples(result, expected); } } 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 7faa6cc599d7e..7bc8480d3a29b 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 @@ -78,8 +78,8 @@ public void testCoGroupTuplesWithKeyFieldSelector() throws Exception { @Test public void testCoGroupOnTwoCustomTypeInputsWithKeyExtractors() throws Exception { /* - * CoGroup on two custom type inputs with key extractors - */ + * CoGroup on two custom type inputs with key extractors + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -515,7 +515,9 @@ public void testCoGroupWithAtomicType2() throws Exception { compareResultAsText(result, expected); } - + + + // -------------------------------------------------------------------------------------------- // UDF classes // -------------------------------------------------------------------------------------------- diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java index 74868a0f28a9b..63d1ec72815c6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.javaApiOperators; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.functions.RichCrossFunction; @@ -30,11 +31,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -47,22 +44,6 @@ public CrossITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception { /* @@ -75,10 +56,9 @@ public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.cross(ds2).with(new Tuple5Cross()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "0,HalloHallo\n" + + String expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1,Hallo WeltHallo\n" + @@ -87,6 +67,8 @@ public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception { "2,Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4,Hallo Welt wieHallo Welt wie\n"; + + compareResultAsTuples(result, expected); } @Test @@ -101,10 +83,9 @@ public void testCorrectnessOfCrossIfUDFReturnsLeftInputObject() throws Exception DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.cross(ds2).with(new Tuple3ReturnLeft()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "1,1,Hi\n" + "1,1,Hi\n" + "2,2,Hello\n" + @@ -113,6 +94,8 @@ public void testCorrectnessOfCrossIfUDFReturnsLeftInputObject() throws Exception "3,2,Hello world\n" + "3,2,Hello world\n" + "3,2,Hello world\n"; + + compareResultAsTuples(result, expected); } @Test @@ -127,10 +110,10 @@ public void testCorrectnessOfCrossIfUDFReturnsRightInputObject() throws Exceptio DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.cross(ds2).with(new Tuple5ReturnRight()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs + .collect(); - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + @@ -140,6 +123,7 @@ public void testCorrectnessOfCrossIfUDFReturnsRightInputObject() throws Exceptio "2,3,2,Hallo Welt wie,1\n" + "2,3,2,Hallo Welt wie,1\n"; + compareResultAsTuples(result, expected); } @Test @@ -156,10 +140,9 @@ public void testCorrectnessOfCrossWithBroadcastSet() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.cross(ds2).with(new Tuple5CrossBC()).withBroadcastSet(intDs, "ints"); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "2,0,55\n" + + String expected = "2,0,55\n" + "3,0,55\n" + "3,0,55\n" + "3,0,55\n" + @@ -168,6 +151,8 @@ public void testCorrectnessOfCrossWithBroadcastSet() throws Exception { "3,0,55\n" + "4,2,55\n" + "4,4,55\n"; + + compareResultAsTuples(result, expected); } @Test @@ -182,10 +167,9 @@ public void testCorrectnessOfCrossWithHuge() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.crossWithHuge(ds2).with(new Tuple5Cross()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "0,HalloHallo\n" + + String expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1,Hallo WeltHallo\n" + @@ -194,6 +178,8 @@ public void testCorrectnessOfCrossWithHuge() throws Exception { "2,Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4,Hallo Welt wieHallo Welt wie\n"; + + compareResultAsTuples(result, expected); } @Test @@ -208,10 +194,9 @@ public void testCorrectnessOfCrossWithTiny() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> crossDs = ds.crossWithTiny(ds2).with(new Tuple5Cross()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "0,HalloHallo\n" + + String expected = "0,HalloHallo\n" + "1,HalloHallo Welt\n" + "2,HalloHallo Welt wie\n" + "1,Hallo WeltHallo\n" + @@ -220,6 +205,8 @@ public void testCorrectnessOfCrossWithTiny() throws Exception { "2,Hallo Welt wieHallo\n" + "3,Hallo Welt wieHallo Welt\n" + "4,Hallo Welt wieHallo Welt wie\n"; + + compareResultAsTuples(result, expected); } @Test @@ -236,12 +223,11 @@ public void testProjectCrossOnATupleInput1() throws Exception{ .projectFirst(2, 1) .projectSecond(3) .projectFirst(0) - .projectSecond(4,1); + .projectSecond(4,1); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "Hi,1,Hallo,1,1,1\n" + + String expected = "Hi,1,Hallo,1,1,1\n" + "Hi,1,Hallo Welt,1,2,2\n" + "Hi,1,Hallo Welt wie,1,1,3\n" + "Hello,2,Hallo,2,1,1\n" + @@ -250,6 +236,8 @@ public void testProjectCrossOnATupleInput1() throws Exception{ "Hello world,2,Hallo,3,1,1\n" + "Hello world,2,Hallo Welt,3,2,2\n" + "Hello world,2,Hallo Welt wie,3,1,3\n"; + + compareResultAsTuples(result, expected); } @Test @@ -266,12 +254,11 @@ public void testProjectCrossOnATupleInput2() throws Exception { .projectSecond(3) .projectFirst(2, 1) .projectSecond(4,1) - .projectFirst(0); + .projectFirst(0); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "Hallo,Hi,1,1,1,1\n" + + String expected = "Hallo,Hi,1,1,1,1\n" + "Hallo Welt,Hi,1,2,2,1\n" + "Hallo Welt wie,Hi,1,1,3,1\n" + "Hallo,Hello,2,1,1,2\n" + @@ -281,6 +268,7 @@ public void testProjectCrossOnATupleInput2() throws Exception { "Hallo Welt,Hello world,2,2,2,3\n" + "Hallo Welt wie,Hello world,2,1,3,3\n"; + compareResultAsTuples(result, expected); } @Test @@ -295,10 +283,10 @@ public void testCorrectnessOfDefaultCross() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet, Tuple5>> crossDs = ds.cross(ds2); - crossDs.writeAsCsv(resultPath); - env.execute(); + List, Tuple5>> result = crossDs.collect(); - expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" + + String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" + + "(1,1,Hi),(1,1,0,Hallo,1)\n" + "(1,1,Hi),(2,3,2,Hallo Welt wie,1)\n" + "(2,2,Hello),(2,2,1,Hallo Welt,2)\n" + @@ -307,6 +295,8 @@ public void testCorrectnessOfDefaultCross() throws Exception { "(3,2,Hello world),(2,2,1,Hallo Welt,2)\n" + "(3,2,Hello world),(1,1,0,Hallo,1)\n" + "(3,2,Hello world),(2,3,2,Hallo Welt wie,1)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -321,10 +311,9 @@ public void testCorrectnessOfCrossOnTwoCustomTypeInputs() throws Exception { DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); DataSet crossDs = ds.cross(ds2).with(new CustomTypeCross()); - crossDs.writeAsText(resultPath); - env.execute(); + List result = crossDs.collect(); - expected = "1,0,HiHi\n" + String expected = "1,0,HiHi\n" + "2,1,HiHello\n" + "2,2,HiHello world\n" + "2,1,HelloHi\n" @@ -333,6 +322,8 @@ public void testCorrectnessOfCrossOnTwoCustomTypeInputs() throws Exception { + "2,2,Hello worldHi\n" + "4,3,Hello worldHello\n" + "4,4,Hello worldHello world"; + + compareResultAsText(result, expected); } @Test @@ -347,10 +338,9 @@ public void testCorrectnessOfCrossATupleInputAndACustomTypeInput() throws Except DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); DataSet> crossDs = ds.cross(ds2).with(new MixedCross()); - crossDs.writeAsCsv(resultPath); - env.execute(); + List> result = crossDs.collect(); - expected = "2,0,HalloHi\n" + + String expected = "2,0,HalloHi\n" + "3,0,HalloHello\n" + "3,0,HalloHello world\n" + "3,0,Hallo WeltHi\n" + @@ -359,24 +349,26 @@ public void testCorrectnessOfCrossATupleInputAndACustomTypeInput() throws Except "3,0,Hallo Welt wieHi\n" + "4,2,Hallo Welt wieHello\n" + "4,4,Hallo Welt wieHello world\n"; + + compareResultAsTuples(result, expected); } - + public static class Tuple5Cross implements CrossFunction, Tuple5, Tuple2> { private static final long serialVersionUID = 1L; - + @Override public Tuple2 cross( Tuple5 first, Tuple5 second) throws Exception { - + return new Tuple2(first.f2+second.f2, first.f3+second.f3); } } - + public static class CustomTypeCross implements CrossFunction { private static final long serialVersionUID = 1L; @@ -384,12 +376,12 @@ public static class CustomTypeCross implements CrossFunction, CustomType, Tuple3> { private static final long serialVersionUID = 1L; @@ -401,12 +393,12 @@ public Tuple3 cross( return new Tuple3(first.f0 + second.myInt, first.f2 * second.myLong, first.f3 + second.myString); } - + } - - + + public static class Tuple3ReturnLeft implements CrossFunction, Tuple5, Tuple3> { - + private static final long serialVersionUID = 1L; @Override @@ -417,9 +409,9 @@ public Tuple3 cross( return first; } } - + public static class Tuple5ReturnRight implements CrossFunction, Tuple5, Tuple5> { - + private static final long serialVersionUID = 1L; @Override @@ -427,29 +419,29 @@ public Tuple5 cross( Tuple3 first, Tuple5 second) throws Exception { - + return second; } } - + public static class Tuple5CrossBC extends RichCrossFunction, Tuple5, Tuple3> { private static final long serialVersionUID = 1L; - + private int broadcast = 42; - + @Override public void open(Configuration config) { - + Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for(Integer i : ints) { sum += i; } broadcast = sum; - + } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java index 5dc38677d1d0b..b49bd33581aeb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java @@ -50,7 +50,6 @@ public DataSinkITCase(TestExecutionMode mode) { } private String resultPath; - private String expected; @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -62,7 +61,6 @@ public void before() throws Exception{ @Test public void testIntSortingParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = CollectionDataSets.getIntegerDataSet(env); @@ -70,14 +68,13 @@ public void testIntSortingParallelism1() throws Exception { env.execute(); - expected = "5\n5\n5\n5\n5\n4\n4\n4\n4\n3\n3\n3\n2\n2\n1\n"; + String expected = "5\n5\n5\n5\n5\n4\n4\n4\n4\n3\n3\n3\n2\n2\n1\n"; compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); } @Test public void testStringSortingParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = CollectionDataSets.getStringDataSet(env); @@ -85,7 +82,7 @@ public void testStringSortingParallelism1() throws Exception { env.execute(); - expected = "Hello\n" + + String expected = "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "Hi\n" + @@ -99,7 +96,6 @@ public void testStringSortingParallelism1() throws Exception { @Test public void testTupleSortingSingleAscParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); @@ -107,7 +103,7 @@ public void testTupleSortingSingleAscParallelism1() throws Exception { env.execute(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" + @@ -130,12 +126,10 @@ public void testTupleSortingSingleAscParallelism1() throws Exception { "21,6,Comment#15\n"; compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } @Test public void testTupleSortingSingleDescParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); @@ -143,7 +137,7 @@ public void testTupleSortingSingleDescParallelism1() throws Exception { env.execute(); - expected = "21,6,Comment#15\n" + + String expected = "21,6,Comment#15\n" + "20,6,Comment#14\n" + "19,6,Comment#13\n" + "18,6,Comment#12\n" + @@ -164,13 +158,12 @@ public void testTupleSortingSingleDescParallelism1() throws Exception { "3,2,Hello world\n" + "2,2,Hello\n" + "1,1,Hi\n"; - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); + compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); } @Test public void testTupleSortingDualParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); @@ -178,7 +171,7 @@ public void testTupleSortingDualParallelism1() throws Exception { env.execute(); - expected = "16,6,Comment#10\n" + + String expected = "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + @@ -199,13 +192,12 @@ public void testTupleSortingDualParallelism1() throws Exception { "2,2,Hello\n" + "3,2,Hello world\n" + "1,1,Hi\n"; - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); + compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); } @Test public void testTupleSortingNestedParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet, String, Integer>> ds = @@ -217,7 +209,7 @@ public void testTupleSortingNestedParallelism1() throws Exception { env.execute(); - expected = + String expected = "((2,1),a,3)\n" + "((2,2),b,4)\n" + "((1,2),a,1)\n" + @@ -231,7 +223,6 @@ public void testTupleSortingNestedParallelism1() throws Exception { @Test public void testTupleSortingNestedParallelism1_2() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet, String, Integer>> ds = @@ -243,7 +234,7 @@ public void testTupleSortingNestedParallelism1_2() throws Exception { env.execute(); - expected = + String expected = "((2,1),a,3)\n" + "((1,3),a,2)\n" + "((1,2),a,1)\n" + @@ -257,7 +248,6 @@ public void testTupleSortingNestedParallelism1_2() throws Exception { @Test public void testPojoSortingSingleParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); @@ -265,7 +255,7 @@ public void testPojoSortingSingleParallelism1() throws Exception { env.execute(); - expected = "1 First (10,100,1000,One) 10100\n" + + String expected = "1 First (10,100,1000,One) 10100\n" + "2 First_ (10,105,1000,One) 10200\n" + "3 First (11,102,3000,One) 10200\n" + "4 First_ (11,106,1000,One) 10300\n" + @@ -275,12 +265,10 @@ public void testPojoSortingSingleParallelism1() throws Exception { "8 Third_ (30,300,1000,Three) 10100\n"; compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } @Test public void testPojoSortingDualParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); @@ -291,7 +279,7 @@ public void testPojoSortingDualParallelism1() throws Exception { env.execute(); - expected = + String expected = "5 First (11,102,2000,One) 10100\n" + "3 First (11,102,3000,One) 10200\n" + "1 First (10,100,1000,One) 10100\n" + @@ -307,7 +295,6 @@ public void testPojoSortingDualParallelism1() throws Exception { @Test public void testPojoSortingNestedParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); @@ -319,7 +306,7 @@ public void testPojoSortingNestedParallelism1() throws Exception { env.execute(); - expected = + String expected = "2 First_ (10,105,1000,One) 10200\n" + "1 First (10,100,1000,One) 10100\n" + "4 First_ (11,106,1000,One) 10300\n" + @@ -334,7 +321,6 @@ public void testPojoSortingNestedParallelism1() throws Exception { @Test public void testSortingParallelism4() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet ds = env.generateSequence(0, 1000); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java index 9755caa1e9e6e..aa40754fb59ef 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -30,44 +31,35 @@ /** * Tests for the DataSource */ - public class DataSourceITCase extends JavaProgramTestBase { - private String resultPath; private String inputPath; - private String expectedResult; - @Override protected void preSubmit() throws Exception { inputPath = createTempFile("input", "ab\n" + "cd\n" + "ef\n"); - resultPath = getTempDirPath("result"); } @Override protected void testProgram() throws Exception { /* - * Test passing a configuration object to an input format - */ + * Test passing a configuration object to an input format + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Configuration ifConf = new Configuration(); ifConf.setString("prepend", "test"); DataSet ds = env.createInput(new TestInputFormat(new Path(inputPath))).withParameters(ifConf); - ds.writeAsText(resultPath); - env.execute(); + List result = ds.collect(); - expectedResult= "ab\n" + String expectedResult = "ab\n" + "cd\n" + "ef\n"; - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expectedResult, resultPath); + + compareResultAsText(result, expectedResult); } private static class TestInputFormat extends TextInputFormat { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java index 4568ab6708aeb..02dbb761123da 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -32,11 +33,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -48,22 +45,6 @@ public DistinctITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelector() throws Exception { /* @@ -75,17 +56,17 @@ public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelector() throws Excep DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); DataSet> distinctDs = ds.union(ds).distinct(0, 1, 2); - distinctDs.writeAsCsv(resultPath); - env.execute(); + List> result = distinctDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllFieldsSelected() - throws Exception{ + public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllFieldsSelected() throws Exception{ /* * check correctness of distinct on tuples with key field selector with not all fields selected */ @@ -93,13 +74,14 @@ public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllField final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> distinctDs = ds.union(ds).distinct(0).project(0); + DataSet> distinctDs = ds.union(ds).distinct(0).project(0); - distinctDs.writeAsCsv(resultPath); - env.execute(); + List> result = distinctDs.collect(); - expected = "1\n" + + String expected = "1\n" + "2\n"; + + compareResultAsTuples(result, expected); } @Test @@ -111,15 +93,13 @@ public void testCorrectnessOfDistinctOnTuplesWithKeyExtractorFunction() throws E final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> reduceDs = ds.union(ds) - .distinct(new KeySelector1()).project(0); + DataSet> reduceDs = ds.union(ds).distinct(new KeySelector1()).project(0); + List> result = reduceDs.collect(); - reduceDs.writeAsCsv(resultPath); - env.execute(); + String expected = "1\n" + "2\n"; - expected = "1\n" + - "2\n"; + compareResultAsTuples(result, expected); } public static class KeySelector1 implements KeySelector, Integer> { @@ -143,15 +123,16 @@ public void testCorrectnessOfDistinctOnCustomTypeWithTypeExtractor() throws Exce .distinct(new KeySelector3()) .map(new Mapper3()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1\n" + + String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; + + compareResultAsTuples(result, expected); } public static class Mapper3 extends RichMapFunction> { @@ -180,17 +161,17 @@ public void testCorrectnessOfDistinctOnTuples() throws Exception{ DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); DataSet> distinctDs = ds.union(ds).distinct(); - distinctDs.writeAsCsv(resultPath); - env.execute(); + List> result = distinctDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor() throws - Exception{ + public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor() throws Exception{ /* * check correctness of distinct on custom type with tuple-returning type extractor */ @@ -200,12 +181,11 @@ public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor DataSet> ds = CollectionDataSets.get5TupleDataSet(env); DataSet> reduceDs = ds .distinct(new KeySelector2()) - .project(0,4); + .project(0,4); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1\n" + + String expected = "1,1\n" + "2,1\n" + "2,2\n" + "3,2\n" + @@ -215,6 +195,8 @@ public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor "5,1\n" + "5,2\n" + "5,3\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector2 implements KeySelector, Tuple2> { @@ -234,14 +216,14 @@ public void testCorrectnessOfDistinctOnTuplesWithFieldExpressions() throws Excep final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> reduceDs = ds.union(ds) - .distinct("f0").project(0); + DataSet> reduceDs = ds.union(ds).distinct("f0").project(0); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1\n" + + String expected = "1\n" + "2\n"; + + compareResultAsTuples(result, expected); } @Test @@ -255,10 +237,11 @@ public void testCorrectnessOfDistinctOnPojos() throws Exception { DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); DataSet reduceDs = ds.distinct("nestedPojo.longNumber").map(new Mapper2()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "10000\n20000\n30000\n"; - expected = "10000\n20000\n30000\n"; + compareResultAsText(result, expected); } public static class Mapper2 implements MapFunction { @@ -278,10 +261,11 @@ public void testDistinctOnFullPojo() throws Exception { DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); DataSet reduceDs = ds.distinct().map(new Mapper1()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "10000\n20000\n30000\n"; - expected = "10000\n20000\n30000\n"; + compareResultAsText(result, expected); } public static class Mapper1 implements MapFunction { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java index c46bc469d6982..993b137f51989 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.javaApiOperators; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.RichFilterFunction; @@ -27,11 +28,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -43,22 +40,6 @@ public FilterITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testAllRejectingFilter() throws Exception { /* @@ -71,10 +52,11 @@ public void testAllRejectingFilter() throws Exception { DataSet> filterDs = ds. filter(new Filter1()); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "\n"; + String expected = "\n"; + + compareResultAsTuples(result, expected); } public static class Filter1 implements FilterFunction> { @@ -97,10 +79,9 @@ public void testAllPassingFilter() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. filter(new Filter2()); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" + @@ -121,6 +102,8 @@ public void testAllPassingFilter() throws Exception { "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class Filter2 implements FilterFunction> { @@ -143,12 +126,14 @@ public void testFilterOnStringTupleField() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. filter(new Filter3()); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "3,2,Hello world\n" + + String expected = "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n"; + compareResultAsTuples(result, expected); + } public static class Filter3 implements FilterFunction> { @@ -163,18 +148,17 @@ public boolean filter(Tuple3 value) throws Exception { @Test public void testFilterOnIntegerTupleField() throws Exception { /* - * Test filter on Integer tuple field. - */ + * Test filter on Integer tuple field. + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. filter(new Filter4()); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "2,2,Hello\n" + + String expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4,Comment#2\n" + @@ -184,6 +168,8 @@ public void testFilterOnIntegerTupleField() throws Exception { "16,6,Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n"; + + compareResultAsTuples(result, expected); } public static class Filter4 implements FilterFunction> { @@ -206,13 +192,14 @@ public void testFilterBasicType() throws Exception { DataSet ds = CollectionDataSets.getStringDataSet(env); DataSet filterDs = ds. filter(new Filter5()); - filterDs.writeAsText(resultPath); - env.execute(); + List result = filterDs.collect(); - expected = "Hi\n" + + String expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n"; + + compareResultAsText(result, expected); } public static class Filter5 implements FilterFunction { @@ -235,12 +222,14 @@ public void testFilterOnCustomType() throws Exception { DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); DataSet filterDs = ds. filter(new Filter6()); - filterDs.writeAsText(resultPath); - env.execute(); + List result = filterDs.collect(); - expected = "3,3,Hello world, how are you?\n" + + String expected = "3,3,Hello world, how are you?\n" + + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n"; + + compareResultAsText(result, expected); } public static class Filter6 implements FilterFunction { @@ -265,13 +254,14 @@ public void testRichFilterOnStringTupleField() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. filter(new RichFilter1()).withBroadcastSet(ints, "ints"); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"; + + compareResultAsTuples(result, expected); } public static class RichFilter1 extends RichFilterFunction> { @@ -306,14 +296,15 @@ public void testFilterWithBroadcastVariables() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> filterDs = ds. filter(new RichFilter2()).withBroadcastSet(intDs, "ints"); - filterDs.writeAsCsv(resultPath); - env.execute(); + List> result = filterDs.collect(); - expected = "11,5,Comment#5\n" + + String expected = "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5,Comment#9\n"; + + compareResultAsTuples(result, expected); } public static class RichFilter2 extends RichFilterFunction> { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java index 15d98dd1ddb71..3eb870dd6f7ac 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java @@ -18,6 +18,8 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.DataSet; @@ -29,11 +31,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -43,22 +41,6 @@ public FirstNITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testFirstNOnUngroupedDS() throws Exception { /* @@ -70,10 +52,11 @@ public void testFirstNOnUngroupedDS() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> seven = ds.first(7).map(new OneMapper()).sum(0); - seven.writeAsText(resultPath); - env.execute(); + List> result = seven.collect(); - expected = "(7)\n"; + String expected = "(7)\n"; + + compareResultAsText(result, expected); } @Test @@ -88,10 +71,11 @@ public void testFirstNOnGroupedDS() throws Exception { DataSet> first = ds.groupBy(1).first(4) .map(new OneMapper2()).groupBy(0).sum(1); - first.writeAsText(resultPath); - env.execute(); + List> result = first.collect(); + + String expected = "(1,1)\n(2,2)\n(3,3)\n(4,4)\n(5,4)\n(6,4)\n"; - expected = "(1,1)\n(2,2)\n(3,3)\n(4,4)\n(5,4)\n(6,4)\n"; + compareResultAsText(result, expected); } @Test @@ -104,17 +88,18 @@ public void testFirstNOnGroupedAndSortedDS() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> first = ds.groupBy(1).sortGroup(0, Order.DESCENDING).first(3) - .project(1,0); + .project(1,0); - first.writeAsText(resultPath); - env.execute(); + List> result = first.collect(); - expected = "(1,1)\n" + String expected = "(1,1)\n" + "(2,3)\n(2,2)\n" + "(3,6)\n(3,5)\n(3,4)\n" + "(4,10)\n(4,9)\n(4,8)\n" + "(5,15)\n(5,14)\n(5,13)\n" + "(6,21)\n(6,20)\n(6,19)\n"; + + compareResultAsText(result, expected); } /** @@ -137,13 +122,13 @@ public Double getKey(String value) throws Exception { } }, Order.DESCENDING).first(1); - b.writeAsText(resultPath); - ee.execute(); + List result = b.collect(); - expected = "a\nb"; + String expected = "a\nb"; + compareResultAsText(result, expected); } - + public static class OneMapper implements MapFunction, Tuple1> { private static final long serialVersionUID = 1L; private final Tuple1 one = new Tuple1(1); @@ -152,7 +137,7 @@ public Tuple1 map(Tuple3 value) { return one; } } - + public static class OneMapper2 implements MapFunction, Tuple2> { private static final long serialVersionUID = 1L; private final Tuple2 one = new Tuple2(0l,1); @@ -162,5 +147,5 @@ public Tuple2 map(Tuple3 value) { return one; } } - + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java index 37cf1fc27dd7c..4962da8b68787 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.javaApiOperators; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; @@ -28,11 +29,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -44,22 +41,6 @@ public FlatMapITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testNonPassingFlatMap() throws Exception { /* @@ -72,10 +53,11 @@ public void testNonPassingFlatMap() throws Exception { DataSet nonPassingFlatMapDs = ds. flatMap(new FlatMapper1()); - nonPassingFlatMapDs.writeAsText(resultPath); - env.execute(); + List result = nonPassingFlatMapDs.collect(); - expected = "\n"; + String expected = "\n"; + + compareResultAsText(result, expected); } public static class FlatMapper1 implements FlatMapFunction { @@ -101,10 +83,9 @@ public void testDataDuplicatingFlatMap() throws Exception { DataSet duplicatingFlatMapDs = ds. flatMap(new FlatMapper2()); - duplicatingFlatMapDs.writeAsText(resultPath); - env.execute(); + List result = duplicatingFlatMapDs.collect(); - expected = "Hi\n" + "HI\n" + + String expected = "Hi\n" + "HI\n" + "Hello\n" + "HELLO\n" + "Hello world\n" + "HELLO WORLD\n" + "Hello world, how are you?\n" + "HELLO WORLD, HOW ARE YOU?\n" + @@ -112,6 +93,8 @@ public void testDataDuplicatingFlatMap() throws Exception { "Luke Skywalker\n" + "LUKE SKYWALKER\n" + "Random comment\n" + "RANDOM COMMENT\n" + "LOL\n" + "LOL\n"; + + compareResultAsText(result, expected); } public static class FlatMapper2 implements FlatMapFunction { @@ -136,10 +119,9 @@ public void testFlatMapWithVaryingNumberOfEmittedTuples() throws Exception { DataSet> varyingTuplesMapDs = ds. flatMap(new FlatMapper3()); - varyingTuplesMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = varyingTuplesMapDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "5,3,I am fine.\n" + @@ -153,6 +135,8 @@ public void testFlatMapWithVaryingNumberOfEmittedTuples() throws Exception { "17,6,Comment#11\n" + "17,6,Comment#11\n" + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "20,6,Comment#14\n"; + + compareResultAsTuples(result, expected); } public static class FlatMapper3 implements FlatMapFunction, Tuple3> { @@ -180,10 +164,9 @@ public void testTypeConversionFlatMapperCustomToTuple() throws Exception { DataSet> typeConversionFlatMapDs = ds. flatMap(new FlatMapper4()); - typeConversionFlatMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = typeConversionFlatMapDs.collect(); - expected = "1,0,Hi\n" + + String expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, how are you?\n" + @@ -204,6 +187,8 @@ public void testTypeConversionFlatMapperCustomToTuple() throws Exception { "6,18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class FlatMapper4 implements FlatMapFunction> { @@ -212,8 +197,7 @@ public static class FlatMapper4 implements FlatMapFunction(); @Override - public void flatMap(CustomType value, Collector> out) - throws Exception { + public void flatMap(CustomType value, Collector> out) throws Exception { outTuple.setField(value.myInt, 0); outTuple.setField(value.myLong, 1); outTuple.setField(value.myString, 2); @@ -233,10 +217,10 @@ public void testTypeConversionFlatMapperTupleToBasic() throws Exception { DataSet typeConversionFlatMapDs = ds. flatMap(new FlatMapper5()); - typeConversionFlatMapDs.writeAsText(resultPath); - env.execute(); + List result = typeConversionFlatMapDs.collect(); - expected = "Hi\n" + "Hello\n" + "Hello world\n" + + String expected = "Hi\n" + "Hello\n" + "Hello world\n" + + "Hello world, how are you?\n" + "I am fine.\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + @@ -247,21 +231,21 @@ public void testTypeConversionFlatMapperTupleToBasic() throws Exception { "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + "Comment#14\n" + "Comment#15\n"; + + compareResultAsText(result, expected); } public static class FlatMapper5 implements FlatMapFunction,String> { private static final long serialVersionUID = 1L; @Override - public void flatMap(Tuple3 value, - Collector out) throws Exception { + public void flatMap(Tuple3 value, Collector out) throws Exception { out.collect(value.f2); } } @Test - public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() throws - Exception { + public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() throws Exception { /* * Test flatmapper if UDF returns input object * multiple times and changes it in between @@ -273,10 +257,9 @@ public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() DataSet> inputObjFlatMapDs = ds. flatMap(new FlatMapper6()); - inputObjFlatMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = inputObjFlatMapDs.collect(); - expected = "0,1,Hi\n" + + String expected = "0,1,Hi\n" + "0,2,Hello\n" + "1,2,Hello\n" + "0,2,Hello world\n" + "1,2,Hello world\n" + "2,2,Hello world\n" + "0,3,I am fine.\n" + @@ -292,6 +275,8 @@ public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() "0,6,Comment#12\n" + "1,6,Comment#12\n" + "0,6,Comment#13\n" + "1,6,Comment#13\n" + "2,6,Comment#13\n" + "0,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class FlatMapper6 implements FlatMapFunction, Tuple3> { @@ -321,10 +306,9 @@ public void testFlatMapWithBroadcastSet() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> bcFlatMapDs = ds. flatMap(new RichFlatMapper1()).withBroadcastSet(ints, "ints"); - bcFlatMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = bcFlatMapDs.collect(); - expected = "55,1,Hi\n" + + String expected = "55,1,Hi\n" + "55,2,Hello\n" + "55,2,Hello world\n" + "55,3,Hello world, how are you?\n" + @@ -345,6 +329,8 @@ public void testFlatMapWithBroadcastSet() throws Exception { "55,6,Comment#13\n" + "55,6,Comment#14\n" + "55,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class RichFlatMapper1 extends RichFlatMapFunction, Tuple3> { @@ -370,5 +356,5 @@ public void flatMap(Tuple3 value, out.collect(outTuple); } } - + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java index 3e9fde750398a..7e6de0448ec95 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java @@ -1,20 +1,20 @@ /* -* 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. -*/ + * 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.test.javaApiOperators; @@ -32,38 +32,29 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Arrays; - +import java.util.List; @SuppressWarnings("serial") @RunWith(Parameterized.class) /** -* The GroupCombine operator is not easy to test because it is essentially just a combiner. The result can be -* the result of a normal groupReduce at any stage its execution. The basic idea is to preserve the grouping key -* in the partial result, so that we can do a reduceGroup afterwards to finalize the results for verification. -* In addition, we can use hashPartition to partition the data and check if no shuffling (just combining) has -* been performed. -*/ + * The GroupCombine operator is not easy to test because it is essentially just a combiner. The result can be + * the result of a normal groupReduce at any stage its execution. The basic idea is to preserve the grouping key + * in the partial result, so that we can do a reduceGroup afterwards to finalize the results for verification. + * In addition, we can use hashPartition to partition the data and check if no shuffling (just combining) has + * been performed. + */ public class GroupCombineITCase extends MultipleProgramsTestBase { public GroupCombineITCase(TestExecutionMode mode) { super(mode); } - private String resultPath; - - private String expected; - private static String identityResult = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + @@ -86,21 +77,6 @@ public GroupCombineITCase(TestExecutionMode mode) { "20,6,Comment#14\n" + "21,6,Comment#15\n"; - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception { - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception { - if (expected != null) { - compareResultsByLinesInMemory(expected, resultPath); - } - } - @Test public void testAllGroupCombineIdentity() throws Exception { @@ -108,19 +84,15 @@ public void testAllGroupCombineIdentity() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds // combine .combineGroup(new IdentityFunction()) // fully reduce .reduceGroup(new IdentityFunction()); + List> result = reduceDs.collect(); - reduceDs.writeAsCsv(resultPath); - - env.execute(); - - expected = identityResult; + compareResultAsTuples(result, identityResult); } @Test @@ -136,11 +108,9 @@ public void testIdentity() throws Exception { // fully reduce .reduceGroup(new IdentityFunction()); - reduceDs.writeAsCsv(resultPath); - - env.execute(); + List> result = reduceDs.collect(); - expected = identityResult; + compareResultAsTuples(result, identityResult); } @Test @@ -157,12 +127,9 @@ public void testIdentityWithGroupBy() throws Exception { // fully reduce .reduceGroup(new IdentityFunction()); + List> result = reduceDs.collect(); - reduceDs.writeAsCsv(resultPath); - - env.execute(); - - expected = identityResult; + compareResultAsTuples(result, identityResult); } @Test @@ -182,11 +149,9 @@ public void testIdentityWithGroupByAndSort() throws Exception { // fully reduce .reduceGroup(new IdentityFunction()); - reduceDs.writeAsCsv(resultPath); + List> result = reduceDs.collect(); - env.execute(); - - expected = identityResult; + compareResultAsTuples(result, identityResult); } @Test @@ -201,7 +166,7 @@ public void testPartialReduceWithIdenticalInputOutputType() throws Exception { // wrap values as Kv pairs with the grouping key as key .map(new Tuple3KvWrapper()); - dsWrapped + List> result = dsWrapped .groupBy(0) // reduce partially .combineGroup(new Tuple3toTuple3GroupReduce()) @@ -214,19 +179,16 @@ public void testPartialReduceWithIdenticalInputOutputType() throws Exception { public Tuple3 map(Tuple2> value) throws Exception { return value.f1; } - }) - .writeAsCsv(resultPath); - + }).collect(); - - env.execute(); - - expected = "1,1,combined\n" + + String expected = "1,1,combined\n" + "5,4,combined\n" + "15,9,combined\n" + "34,16,combined\n" + "65,25,combined\n" + "111,36,combined\n"; + + compareResultAsTuples(result, expected); } @Test @@ -241,33 +203,29 @@ public void testPartialReduceWithDifferentInputOutputType() throws Exception { // wrap values as Kv pairs with the grouping key as key .map(new Tuple3KvWrapper()); - dsWrapped + List> result = dsWrapped .groupBy(0) - // reduce partially + // reduce partially .combineGroup(new Tuple3toTuple2GroupReduce()) .groupBy(0) - // reduce fully to check result + // reduce fully to check result .reduceGroup(new Tuple2toTuple2GroupReduce()) - //unwrap + //unwrap .map(new MapFunction>, Tuple2>() { @Override public Tuple2 map(Tuple2> value) throws Exception { return value.f1; } - }) - .writeAsCsv(resultPath); + }).collect(); - - - env.execute(); - - expected = "1,3\n" + + String expected = "1,3\n" + "5,20\n" + "15,58\n" + "34,52\n" + "65,70\n" + "111,96\n"; + compareResultAsTuples(result, expected); } @Test @@ -284,7 +242,9 @@ public void testCheckPartitionShuffleGroupBy() throws Exception { // partition and group data UnsortedGrouping> partitionedDS = ds.partitionByHash(0).groupBy(1); - partitionedDS.combineGroup(new GroupCombineFunction, Tuple2>() { + List> result = partitionedDS + .combineGroup( + new GroupCombineFunction, Tuple2>() { @Override public void combine(Iterable> values, Collector> out) throws Exception { int count = 0; @@ -295,29 +255,17 @@ public void combine(Iterable> values, Collector list = new ArrayList(); - readAllResultLines(list, resultPath); - - String[] result = list.toArray(new String[list.size()]); - Arrays.sort(result); - - String[] expected = notExpected.split("\n"); - Arrays.sort(expected); + String[] resultAsStringArray = new String[result.size()]; + for (int i = 0; i < resultAsStringArray.length; ++i) { + resultAsStringArray[i] = result.get(i).toString(); + } + Arrays.sort(resultAsStringArray); - Assert.assertEquals("The two arrays were identical.", false, Arrays.equals(expected, result)); + Assert.assertEquals("The two arrays were identical.", false, Arrays.equals(localExpected, resultAsStringArray)); } @Test @@ -334,28 +282,29 @@ public void testCheckPartitionShuffleDOP1() throws Exception { // partition and group data UnsortedGrouping> partitionedDS = ds.partitionByHash(0).groupBy(1); - partitionedDS.combineGroup(new GroupCombineFunction, Tuple2>() { - @Override - public void combine(Iterable> values, Collector> out) throws Exception { - int count = 0; - long key = 0; - for (Tuple3 value : values) { - key = value.f1; - count++; - } - out.collect(new Tuple2(key, count)); - } - }).writeAsCsv(resultPath); - - env.execute(); + List> result = partitionedDS + .combineGroup( + new GroupCombineFunction, Tuple2>() { + @Override + public void combine(Iterable> values, Collector> out) throws Exception { + int count = 0; + long key = 0; + for (Tuple3 value : values) { + key = value.f1; + count++; + } + out.collect(new Tuple2(key, count)); + } + }).collect(); - expected = "6,6\n" + + String expected = "6,6\n" + "5,5\n" + "4,4\n" + "3,3\n" + "2,2\n" + "1,1\n"; + compareResultAsTuples(result, expected); } @Test @@ -373,15 +322,15 @@ public Tuple1 map(String value) throws Exception { // all methods on DataSet ds.combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); + .output(new DiscardingOutputFormat>()); // all methods on UnsortedGrouping ds.groupBy(0).combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); + .output(new DiscardingOutputFormat>()); // all methods on SortedGrouping ds.groupBy(0).sortGroup(0, Order.ASCENDING).combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); + .output(new DiscardingOutputFormat>()); env.execute(); } @@ -407,7 +356,7 @@ public void combine(Iterable> values, Collector, Tuple3>, - GroupReduceFunction, Tuple3> { + GroupReduceFunction, Tuple3> { @Override public void combine(Iterable> values, Collector> out) throws Exception { @@ -427,6 +376,7 @@ public void reduce(Iterable> values, Collector, Tuple3, Tuple3> { + @Override public void combine(Iterable>> values, Collector>> out) throws Exception { int i = 0; long l = 0; @@ -478,6 +428,7 @@ public void reduce(Iterable>> values, Collect public static class Tuple2toTuple2GroupReduce implements KvGroupReduce, Tuple2, Tuple2> { + @Override public void combine(Iterable>> values, Collector>> out) throws Exception { int i = 0; long l = 0; @@ -516,5 +467,4 @@ public interface CombineAndReduceGroup extends GroupCombineFuncti public interface KvGroupReduce extends CombineAndReduceGroup, Tuple2, Tuple2> { } - -} \ No newline at end of file +} 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 4061195fe3894..2d008fb1911eb 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 @@ -41,18 +41,16 @@ import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; import org.joda.time.DateTime; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; + import scala.math.BigInt; import java.util.Collection; import java.util.Iterator; +import java.util.List; @SuppressWarnings("serial") @RunWith(Parameterized.class) @@ -62,24 +60,6 @@ public GroupReduceITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - if(expected != null) { - compareResultsByLinesInMemory(expected, resultPath); - } - } - @Test public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector() throws Exception{ /* @@ -92,20 +72,20 @@ public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector() throws Ex DataSet> reduceDs = ds. groupBy(1).reduceGroup(new Tuple3GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1\n" + + String expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() throws - Exception { + public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() throws Exception { /* * check correctness of groupReduce on tuples with multiple key field selector */ @@ -116,10 +96,9 @@ public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() DataSet> reduceDs = ds. groupBy(4, 0).reduceGroup(new Tuple5GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,0,P-),1\n" + + String expected = "1,1,0,P-),1\n" + "2,3,0,P-),1\n" + "2,2,0,P-),2\n" + "3,9,0,P-),2\n" + @@ -129,11 +108,12 @@ public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() "5,11,0,P-),1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting() throws - Exception { + public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting() throws Exception { /* * check correctness of groupReduce on tuples with key field selector and group sorting */ @@ -145,15 +125,17 @@ public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSort DataSet> reduceDs = ds. groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup(new Tuple3SortedGroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + + "5,2,Hello-Hello world\n" + "15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" + "34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" + "65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" + "111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n"; + + compareResultAsTuples(result, expected); } @Test @@ -168,15 +150,16 @@ public void testCorrectnessOfGroupReduceOnTuplesWithKeyExtractor() throws Except DataSet> reduceDs = ds. groupBy(new KeySelector1()).reduceGroup(new Tuple3GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1\n" + + String expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector1 implements KeySelector, Long> { @@ -200,15 +183,16 @@ public void testCorrectnessOfGroupReduceOnCustomTypeWithTypeExtractor() throws E DataSet reduceDs = ds. groupBy(new KeySelector2()).reduceGroup(new CustomTypeGroupReduce()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "1,0,Hello!\n" + + String expected = "1,0,Hello!\n" + "2,3,Hello!\n" + "3,12,Hello!\n" + "4,30,Hello!\n" + "5,60,Hello!\n" + "6,105,Hello!\n"; + + compareResultAsText(result, expected); } public static class KeySelector2 implements KeySelector { @@ -231,10 +215,11 @@ public void testCorrectnessOfAllGroupReduceForTuples() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds.reduceGroup(new AllAddingTuple3GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); + + String expected = "231,91,Hello World\n"; - expected = "231,91,Hello World\n"; + compareResultAsTuples(result, expected); } @Test @@ -248,10 +233,11 @@ public void testCorrectnessOfAllGroupReduceForCustomTypes() throws Exception { DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); DataSet reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "91,210,Hello!"; - expected = "91,210,Hello!"; + compareResultAsText(result, expected); } @Test @@ -268,21 +254,20 @@ public void testCorrectnessOfGroupReduceWithBroadcastSet() throws Exception { DataSet> reduceDs = ds. groupBy(1).reduceGroup(new BCTuple3GroupReduce()).withBroadcastSet(intDs, "ints"); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,55\n" + + String expected = "1,1,55\n" + "5,2,55\n" + "15,3,55\n" + "34,4,55\n" + "65,5,55\n" + "111,6,55\n"; + + compareResultAsTuples(result, expected); } @Test - public void - testCorrectnessOfGroupReduceIfUDFReturnsInputObjectsMultipleTimesWhileChangingThem() throws - Exception{ + public void testCorrectnessOfGroupReduceIfUDFReturnsInputObjectsMultipleTimesWhileChangingThem() throws Exception{ /* * check correctness of groupReduce if UDF returns input objects multiple times and changes it in between */ @@ -293,20 +278,20 @@ public void testCorrectnessOfGroupReduceWithBroadcastSet() throws Exception { DataSet> reduceDs = ds. groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "11,1,Hi!\n" + + String expected = "11,1,Hi!\n" + "21,1,Hi again!\n" + "12,2,Hi!\n" + "22,2,Hi again!\n" + "13,2,Hi!\n" + "23,2,Hi again!\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() - throws Exception { + public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() throws Exception { /* * check correctness of groupReduce on custom type with key extractor and combine */ @@ -318,15 +303,16 @@ public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() DataSet reduceDs = ds. groupBy(new KeySelector3()).reduceGroup(new CustomTypeGroupReduceWithCombine()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "1,0,test1\n" + + String expected = "1,0,test1\n" + "2,3,test2\n" + "3,12,test3\n" + "4,30,test4\n" + "5,60,test5\n" + "6,105,test6\n"; + + compareResultAsText(result, expected); } public static class KeySelector3 implements KeySelector { @@ -340,7 +326,6 @@ public Integer getKey(CustomType in) { @Test public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception { - /* * check correctness of groupReduce on tuples with combine */ @@ -353,20 +338,20 @@ public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception { DataSet> reduceDs = ds. groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,test1\n" + + String expected = "1,test1\n" + "5,test2\n" + "15,test3\n" + "34,test4\n" + "65,test5\n" + "111,test6\n"; + + compareResultAsTuples(result, expected); } @Test public void testCorrectnessOfAllGroupReduceForTuplesWithCombine() throws Exception { - /* * check correctness of all-groupreduce for tuples with combine */ @@ -382,11 +367,12 @@ public void testCorrectnessOfAllGroupReduceForTuplesWithCombine() throws Excepti DataSet> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine()) .withParameters(cfg); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "322," + + String expected = "322," + "testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n"; + + compareResultAsTuples(result, expected); } @Test @@ -401,20 +387,21 @@ public void testCorrectnessOfGroupreduceWithDescendingGroupSort() throws Excepti DataSet> reduceDs = ds. groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + + "5,2,Hello world-Hello\n" + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + compareResultAsTuples(result, expected); } @Test - public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() throws - Exception { + public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() throws Exception { /* * check correctness of groupReduce on tuples with tuple-returning key selector */ @@ -425,10 +412,9 @@ public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() DataSet> reduceDs = ds. groupBy(new KeySelector4()).reduceGroup(new Tuple5GroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,0,P-),1\n" + + String expected = "1,1,0,P-),1\n" + "2,3,0,P-),1\n" + "2,2,0,P-),2\n" + "3,9,0,P-),2\n" + @@ -438,6 +424,8 @@ public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() "5,11,0,P-),1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector4 implements KeySelector, Tuple2> { @@ -450,11 +438,10 @@ public Tuple2 getKey(Tuple5 t) } @Test - public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting() throws - Exception { + public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting() throws Exception { /* * check that input of combiner is also sorted for combinable groupReduce with group sorting - */ + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); @@ -463,15 +450,16 @@ public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting( DataSet> reduceDs = ds. groupBy(1).sortGroup(0, Order.ASCENDING).reduceGroup(new OrderCheckingCombinableReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "7,4,Comment#1\n" + "11,5,Comment#5\n" + "16,6,Comment#10\n"; + + compareResultAsTuples(result, expected); } @Test @@ -486,10 +474,11 @@ public void testDeepNesting() throws Exception { DataSet> reduceDs = ds.groupBy("nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal") .reduceGroup(new GroupReducer1()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); + + String expected = "aa,1\nbb,2\ncc,3\n"; - expected = "aa,1\nbb,2\ncc,3\n"; + compareResultAsTuples(result, expected); } public static class GroupReducer1 implements GroupReduceFunction> { @@ -498,7 +487,7 @@ public static class GroupReducer1 implements GroupReduceFunction values, Collector> out) - throws Exception { + throws Exception { int c = 0; String n = null; for(CrazyNested v : values) { c++; // haha @@ -519,10 +508,11 @@ public void testPojoExtendingFromTupleWithCustomFields() throws Exception { DataSet reduceDs = ds.groupBy("special", "f2") .reduceGroup(new GroupReducer2()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "3\n2\n"; + String expected = "3\n2\n"; + + compareResultAsText(result, expected); } public static class GroupReducer2 implements GroupReduceFunction { @@ -544,14 +534,15 @@ public void testPojoContainigWritableAndTuples() throws Exception { DataSet reduceDs = ds.groupBy("hadoopFan", "theTuple.*") // full tuple selection .reduceGroup(new GroupReducer3()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "1\n5\n"; - expected = "1\n5\n"; + compareResultAsText(result, expected); } public static class GroupReducer3 implements GroupReduceFunction { - + @Override public void reduce(Iterable values, Collector out) { out.collect(countElements(values)); @@ -570,10 +561,11 @@ public void testTupleContainingPojosAndRegularFields() throws Exception { DataSet reduceDs = ds.groupBy("f0", "f1.*") // nested full tuple selection .reduceGroup(new GroupReducer4()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "3\n1\n"; - expected = "3\n1\n"; + compareResultAsText(result, expected); } public static class GroupReducer4 implements GroupReduceFunction, Integer> { @@ -597,15 +589,17 @@ public void testStringBasedDefinitionOnGroupSort() throws Exception { DataSet> reduceDs = ds. groupBy(1).sortGroup("f2", Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + + "5,2,Hello world-Hello\n" + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + compareResultAsTuples(result, expected); } @Test @@ -618,12 +612,13 @@ public void testIntBasedDefinitionOnGroupSortForFullNestedTuple() throws Excepti DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); DataSet reduceDs = ds.groupBy("f1").sortGroup(0, Order.DESCENDING).reduceGroup(new NestedTupleReducer()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "a--(2,1)-(1,3)-(1,2)-\n" + + String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n"+ "c--(4,9)-(3,6)-(3,3)-\n"; + + compareResultAsText(result, expected); } @Test @@ -640,12 +635,13 @@ public void testIntBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exce .sortGroup("f0.f0", Order.ASCENDING) .sortGroup("f0.f1", Order.ASCENDING) .reduceGroup(new NestedTupleReducer()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "a--(1,2)-(1,3)-(2,1)-\n" + + String expected = "a--(1,2)-(1,3)-(2,1)-\n" + "b--(2,2)-\n"+ "c--(3,3)-(3,6)-(4,9)-\n"; + + compareResultAsText(result, expected); } @Test @@ -659,12 +655,13 @@ public void testStringBasedDefinitionOnGroupSortForPartialNestedTuple() throws E DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); // f0.f0 is first integer DataSet reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).reduceGroup(new NestedTupleReducer()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "a--(2,1)-(1,3)-(1,2)-\n" + + String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n"+ "c--(4,9)-(3,3)-(3,6)-\n"; + + compareResultAsText(result, expected); } @Test @@ -678,12 +675,13 @@ public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeys() throws Exce DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); // f0.f0 is first integer DataSet reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).sortGroup("f0.f1", Order.DESCENDING).reduceGroup(new NestedTupleReducer()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "a--(2,1)-(1,3)-(1,2)-\n" + + String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n"+ "c--(4,9)-(3,6)-(3,3)-\n"; + + compareResultAsText(result, expected); } @Test @@ -698,11 +696,13 @@ public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos() th // f0.f0 is first integer DataSet reduceDs = ds.groupBy("hadoopFan").sortGroup("theTuple.f0", Order.DESCENDING).sortGroup("theTuple.f1", Order.DESCENDING) .reduceGroup(new GroupReducer5()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "1---(10,100)-\n" + + String expected = "1---(10,100)-\n" + + "2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-\n"; + + compareResultAsText(result, expected); } @Test @@ -720,16 +720,17 @@ public void testTupleKeySelectorGroupSort() throws Exception { .sortGroup(new StringFieldExtractor>(2), Order.DESCENDING) .reduceGroup(new Tuple3SortedGroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - // return expected result - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + + "5,2,Hello world-Hello\n" + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + compareResultAsTuples(result, expected); } public static class TwoTuplePojoExtractor implements KeySelector> { @@ -764,16 +765,17 @@ public void testPojoKeySelectorGroupSort() throws Exception { .sortGroup(new StringPojoExtractor(), Order.DESCENDING) .reduceGroup(new CustomTypeSortedGroupReduce()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - // return expected result - expected = "1,0,Hi\n" + + String expected = "1,0,Hi\n" + + "2,3,Hello world-Hello\n" + "3,12,Luke Skywalker-I am fine.-Hello world, how are you?\n" + "4,30,Comment#4-Comment#3-Comment#2-Comment#1\n" + "5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + compareResultAsText(result, expected); } public static class LongFieldExtractor implements KeySelector { @@ -839,19 +841,18 @@ public void testTupleKeySelectorSortWithCombine() throws Exception { .sortGroup(new StringFieldExtractor>(2), Order.DESCENDING) .reduceGroup(new Tuple3SortedGroupReduceWithCombine()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - // return expected result - if (super.mode == TestExecutionMode.COLLECTION) { - expected = null; - } else { - expected = "1,Hi\n" + + if (super.mode != TestExecutionMode.COLLECTION) { + String expected = "1,Hi\n" + + "5,Hello world-Hello\n" + "15,Luke Skywalker-I am fine.-Hello world, how are you?\n" + "34,Comment#4-Comment#3-Comment#2-Comment#1\n" + "65,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + "111,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + compareResultAsTuples(result, expected); } } @@ -879,15 +880,16 @@ public void testTupleKeySelectorSortCombineOnTuple() throws Exception { .sortGroup(new FiveToTwoTupleExtractor(), Order.DESCENDING) .reduceGroup(new Tuple5SortedGroupReduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - // return expected result - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + + "2,5,0,Hallo Welt-Hallo Welt wie,1\n" + "3,15,0,BCD-ABC-Hallo Welt wie gehts?,2\n" + "4,34,0,FGH-CDE-EFG-DEF,1\n" + "5,65,0,IJK-HIJ-KLM-JKL-GHI,1\n"; + + compareResultAsTuples(result, expected); } public static class GroupReducer5 implements GroupReduceFunction { @@ -923,10 +925,11 @@ public void testGroupingWithPojoContainingMultiplePojos() throws Exception { // f0.f0 is first integer DataSet reduceDs = ds.groupBy("p2.a2") .reduceGroup(new GroupReducer6()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "b\nccc\nee\n"; - expected = "b\nccc\nee\n"; + compareResultAsText(result, expected); } public static class GroupReducer6 implements GroupReduceFunction { @@ -954,10 +957,11 @@ public void testJavaCollectionsWithinPojos() throws Exception { // f0.f0 is first integer DataSet reduceDs = ds.groupBy("key") .reduceGroup(new GroupReducer7()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "callFor key 0 we got: pojo.a=apojo.a=bFor key 0 we got: pojo.a=a2pojo.a=b2\n"; + String expected = "callFor key 0 we got: pojo.a=apojo.a=bFor key 0 we got: pojo.a=a2pojo.a=b2\n"; + + compareResultAsText(result, expected); } public static class GroupReducer7 implements GroupReduceFunction { @@ -990,20 +994,22 @@ public void testGroupByGenericType() throws Exception { // f0.f0 is first integer DataSet reduceDs = ds.groupBy("bigInt") .reduceGroup(new GroupReducer8()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); ExecutionConfig ec = env.getConfig(); // check if automatic type registration with Kryo worked Assert.assertTrue(ec.getRegisteredKryoTypes().contains(BigInt.class)); Assert.assertTrue(ec.getRegisteredKryoTypes().contains(java.sql.Date.class)); + String expected = null; - expected = "call\n" + + String localExpected = "[call\n" + "For key 92233720368547758070 we got:\n" + "PojoWithCollection{pojos.size()=2, key=0, sqlDate=2033-05-18, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=10, mixed=[{someKey=1}, /this/is/wrong, uhlala]}\n" + "For key 92233720368547758070 we got:\n" + - "PojoWithCollection{pojos.size()=2, key=0, sqlDate=1976-05-03, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=31104000, mixed=null}\n"; + "PojoWithCollection{pojos.size()=2, key=0, sqlDate=1976-05-03, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=31104000, mixed=null}]"; + + Assert.assertEquals(localExpected, result.toString()); } @Test @@ -1034,34 +1040,33 @@ public void reduce(Iterable> values } } }) - // add forward field information - .withForwardedFields("0") - // group again and reduce - .groupBy(0).reduceGroup( - new GroupReduceFunction, Tuple2>() { - @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - int k = 0; - long s = 0; - for (Tuple5 v : values) { - k = v.f0; - s += v.f1; + // add forward field information + .withForwardedFields("0") + // group again and reduce + .groupBy(0).reduceGroup( + new GroupReduceFunction, Tuple2>() { + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + int k = 0; + long s = 0; + for (Tuple5 v : values) { + k = v.f0; + s += v.f1; + } + out.collect(new Tuple2(k, s)); + } } - out.collect(new Tuple2(k, s)); - } - } - ); + ); - reduceDs.writeAsCsv(resultPath); + List> result = reduceDs.collect(); - env.execute(); - - expected = "1,1\n" + + String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n"; + compareResultAsTuples(result, expected); } @Test @@ -1075,13 +1080,14 @@ public void reduce(Iterable values, Collector out) throws Exce } }); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "1\n" + + "2\n" + + "3\n" + + "4"; - expected = "1\n" + - "2\n" + - "3\n" + - "4"; + compareResultAsText(result, expected); } /** @@ -1095,10 +1101,11 @@ public void testJodatimeDateTimeWithKryo() throws Exception { DataSet> ds = env.fromElements(new Tuple2(1, DateTime.now())); DataSet> reduceDs = ds.groupBy("f1").sum(0).project(0); - reduceDs.writeAsText(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "(1)\n"; + String expected = "1\n"; + + compareResultAsTuples(result, expected); } public static class GroupReducer8 implements GroupReduceFunction { @@ -1120,7 +1127,7 @@ public static class NestedTupleReducer implements GroupReduceFunction, String>> values, Collector out) - throws Exception { + throws Exception { boolean once = false; StringBuilder concat = new StringBuilder(); for(Tuple2, String> value : values) { @@ -1134,26 +1141,26 @@ public void reduce( out.collect(concat.toString()); } } - + public static class Tuple3GroupReduce implements GroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; @Override public void reduce(Iterable> values, Collector> out) { - + int i = 0; long l = 0l; - + for (Tuple3 t : values) { i += t.f0; l = t.f1; } - + out.collect(new Tuple2(i, l)); - + } } - + public static class Tuple3SortedGroupReduce implements GroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; @@ -1163,21 +1170,21 @@ public void reduce(Iterable> values, Collector next : values) { sum += next.f0; key = next.f1; concat.append(next.f2).append("-"); } - + if (concat.length() > 0) { concat.setLength(concat.length() - 1); } - + out.collect(new Tuple3(sum, key, concat.toString())); } } - + public static class Tuple5GroupReduce implements GroupReduceFunction, Tuple5> { private static final long serialVersionUID = 1L; @@ -1189,13 +1196,13 @@ public void reduce( int i = 0; long l = 0l; long l2 = 0l; - + for ( Tuple5 t : values ) { i = t.f0; l += t.f1; l2 = t.f4; } - + out.collect(new Tuple5(i, l, 0, "P-)", l2)); } } @@ -1205,8 +1212,8 @@ public static class Tuple5SortedGroupReduce implements GroupReduceFunction> values, - Collector> out) + Iterable> values, + Collector> out) { int i = 0; long l = 0l; @@ -1226,29 +1233,29 @@ public void reduce( out.collect(new Tuple5(i, l, 0, concat.toString(), l2)); } } - + public static class CustomTypeGroupReduce implements GroupReduceFunction { private static final long serialVersionUID = 1L; - + @Override public void reduce(Iterable values, Collector out) { final Iterator iter = values.iterator(); - + CustomType o = new CustomType(); CustomType c = iter.next(); - + o.myString = "Hello!"; o.myInt = c.myInt; o.myLong = c.myLong; - + while (iter.hasNext()) { CustomType next = iter.next(); o.myLong += next.myLong; } - + out.collect(o); - + } } @@ -1271,7 +1278,7 @@ public void reduce(Iterable values, Collector out) { CustomType next = iter.next(); concat.append("-").append(next.myString); o.myLong += next.myLong; - + } o.myString = concat.toString(); @@ -1287,7 +1294,7 @@ public static class InputReturningTuple3GroupReduce implements GroupReduceFuncti public void reduce(Iterable> values, Collector> out) { for ( Tuple3 t : values ) { - + if(t.f0 < 4) { t.f2 = "Hi!"; t.f0 += 10; @@ -1299,74 +1306,74 @@ public void reduce(Iterable> values, Collector, Tuple3> { private static final long serialVersionUID = 1L; - + @Override public void reduce(Iterable> values, Collector> out) { int i = 0; long l = 0l; - + for ( Tuple3 t : values ) { i += t.f0; l += t.f1; } - + out.collect(new Tuple3(i, l, "Hello World")); } } - + public static class AllAddingCustomTypeGroupReduce implements GroupReduceFunction { private static final long serialVersionUID = 1L; - + @Override public void reduce(Iterable values, Collector out) { CustomType o = new CustomType(0, 0, "Hello!"); - + for (CustomType next : values) { o.myInt += next.myInt; o.myLong += next.myLong; } - + out.collect(o); } } - + public static class BCTuple3GroupReduce extends RichGroupReduceFunction,Tuple3> { private static final long serialVersionUID = 1L; private String f2Replace = ""; - + @Override public void open(Configuration config) { - + Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for(Integer i : ints) { sum += i; } f2Replace = sum+""; - + } @Override public void reduce(Iterable> values, Collector> out) { - + int i = 0; long l = 0l; - + for ( Tuple3 t : values ) { i += t.f0; l = t.f1; } - + out.collect(new Tuple3(i, l, this.f2Replace)); - + } } - + @RichGroupReduceFunction.Combinable public static class Tuple3GroupReduceWithCombine extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -1437,75 +1444,75 @@ public void reduce(Iterable> values, Collector(i, s)); } } - + @RichGroupReduceFunction.Combinable public static class Tuple3AllGroupReduceWithCombine extends RichGroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; - + @Override public void combine(Iterable> values, Collector> out) { - + Tuple3 o = new Tuple3(0, 0l, ""); - + for ( Tuple3 t : values ) { o.f0 += t.f0; o.f1 += t.f1; o.f2 += "test"; } - + out.collect(o); } @Override public void reduce(Iterable> values, Collector> out) { - + int i = 0; String s = ""; - + for ( Tuple3 t : values ) { i += t.f0 + t.f1; s += t.f2; } - + out.collect(new Tuple2(i, s)); - + } } - + @RichGroupReduceFunction.Combinable public static class CustomTypeGroupReduceWithCombine extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; - + @Override public void combine(Iterable values, Collector out) throws Exception { - + CustomType o = new CustomType(); - + for ( CustomType c : values ) { o.myInt = c.myInt; o.myLong += c.myLong; o.myString = "test"+c.myInt; } - + out.collect(o); } @Override public void reduce(Iterable values, Collector out) { - + CustomType o = new CustomType(0, 0, ""); - + for ( CustomType c : values) { o.myInt = c.myInt; o.myLong += c.myLong; o.myString = c.myString; } - + out.collect(o); - + } } - + @RichGroupReduceFunction.Combinable public static class OrderCheckingCombinableReduce extends RichGroupReduceFunction, Tuple3> { private static final long serialVersionUID = 1L; @@ -1514,28 +1521,28 @@ public static class OrderCheckingCombinableReduce extends RichGroupReduceFunctio public void reduce(Iterable> values, Collector> out) throws Exception { Iterator> it = values.iterator(); Tuple3 t = it.next(); - + int i = t.f0; out.collect(t); - + while(it.hasNext()) { t = it.next(); if(i > t.f0 || t.f2.equals("INVALID-ORDER!")) { t.f2 = "INVALID-ORDER!"; out.collect(t); } - } + } } - + @Override - public void combine(Iterable> values, Collector> out) { - + public void combine(Iterable> values, Collector> out) { + Iterator> it = values.iterator(); Tuple3 t = it.next(); - + int i = t.f0; out.collect(t); - + while(it.hasNext()) { t = it.next(); if(i > t.f0) { @@ -1545,15 +1552,15 @@ public void combine(Iterable> values, Collector extends RichMapFunction { @Override public T map(T value) { return value; } } - + private static int countElements(Iterable iterable) { int c = 0; for (@SuppressWarnings("unused") Object o : iterable) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java index fe436a3fc2480..61e07fe8f6ef7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.javaApiOperators; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; @@ -35,11 +36,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -53,22 +50,6 @@ public JoinITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testUDFJoinOnTuplesWithKeyFieldPositions() throws Exception { /* @@ -81,16 +62,17 @@ public void testUDFJoinOnTuplesWithKeyFieldPositions() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(1) - .with(new T3T5FlatJoin()); + .where(1) + .equalTo(1) + .with(new T3T5FlatJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo\n" + + String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; + + compareResultAsTuples(result, expected); } @Test @@ -105,19 +87,20 @@ public void testeUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(0,1) - .equalTo(0,4) - .with(new T3T5FlatJoin()); + .where(0,1) + .equalTo(0,4) + .with(new T3T5FlatJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo\n" + + String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"; + + compareResultAsTuples(result, expected); } @Test @@ -132,16 +115,17 @@ public void testDefaultJoinOnTuples() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet,Tuple5>> joinDs = ds1.join(ds2) - .where(0) - .equalTo(2); + .where(0) + .equalTo(2); - joinDs.writeAsCsv(resultPath); - env.execute(); + List, Tuple5>> result = joinDs.collect(); - expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" + + String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" + "(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" + "(3,2,Hello world),(3,4,3,Hallo Welt wie gehts?,2)\n"; + compareResultAsTuples(result, expected); + } @Test @@ -159,12 +143,13 @@ public void testJoinWithHuge() throws Exception { .equalTo(1) .with(new T3T5FlatJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo\n" + + String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; + + compareResultAsTuples(result, expected); } @Test @@ -179,16 +164,17 @@ public void testJoinWithTiny() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.joinWithTiny(ds2) - .where(1) - .equalTo(1) - .with(new T3T5FlatJoin()); + .where(1) + .equalTo(1) + .with(new T3T5FlatJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo\n" + + String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; + + compareResultAsTuples(result, expected); } @Test @@ -203,16 +189,17 @@ public void testJoinThatReturnsTheLeftInputObject() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(1) - .with(new LeftReturningJoin()); + .where(1) + .equalTo(1) + .with(new LeftReturningJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; + + compareResultAsTuples(result, expected); } @Test @@ -227,16 +214,17 @@ public void testJoinThatReturnsTheRightInputObject() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(1) - .with(new RightReturningJoin()); + .where(1) + .equalTo(1) + .with(new RightReturningJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n"; + + compareResultAsTuples(result, expected); } @Test @@ -253,18 +241,19 @@ public void testJoinWithBroadcastSet() throws Exception { DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(4) - .with(new T3T5BCJoin()) - .withBroadcastSet(intDs, "ints"); + .where(1) + .equalTo(4) + .with(new T3T5BCJoin()) + .withBroadcastSet(intDs, "ints"); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo,55\n" + + String expected = "Hi,Hallo,55\n" + "Hi,Hallo Welt wie,55\n" + "Hello,Hallo Welt,55\n" + "Hello world,Hallo Welt,55\n"; + + compareResultAsTuples(result, expected); } @Test @@ -280,17 +269,18 @@ public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyField DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(new KeySelector1()) - .equalTo(0) - .with(new CustT3Join()); + .where(new KeySelector1()) + .equalTo(0) + .with(new CustT3Join()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hi\n" + + String expected = "Hi,Hi\n" + "Hello,Hello\n" + "Hello world,Hello\n"; + compareResultAsTuples(result, expected); + } public static class KeySelector1 implements KeySelector { @@ -312,19 +302,20 @@ public void testProjectOnATuple1Input() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(1) - .projectFirst(2,1) - .projectSecond(3) - .projectFirst(0) - .projectSecond(4,1); + .where(1) + .equalTo(1) + .projectFirst(2,1) + .projectSecond(3) + .projectFirst(0) + .projectSecond(4,1); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,1,Hallo,1,1,1\n" + + String expected = "Hi,1,Hallo,1,1,1\n" + "Hello,2,Hallo Welt,2,2,2\n" + "Hello world,2,Hallo Welt,3,2,2\n"; + + compareResultAsTuples(result, expected); } @Test @@ -339,19 +330,20 @@ public void testProjectJoinOnATuple2Input() throws Exception { DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1) - .equalTo(1) - .projectSecond(3) - .projectFirst(2,1) - .projectSecond(4,1) - .projectFirst(0); + .where(1) + .equalTo(1) + .projectSecond(3) + .projectFirst(2,1) + .projectSecond(4,1) + .projectFirst(0); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hallo,Hi,1,1,1,1\n" + + String expected = "Hallo,Hi,1,1,1,1\n" + "Hallo Welt,Hello,2,2,2,2\n" + "Hallo Welt,Hello world,2,2,2,3\n"; + + compareResultAsTuples(result, expected); } @Test @@ -367,15 +359,16 @@ public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyE DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(1).equalTo(new KeySelector2()) - .with(new T3CustJoin()); + .where(1).equalTo(new KeySelector2()) + .with(new T3CustJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hello\n" + + String expected = "Hi,Hello\n" + "Hello,Hello world\n" + "Hello world,Hello world\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector2 implements KeySelector { @@ -398,21 +391,18 @@ public void testDefaultJoinOnTwoCustomTypeInputsWithKeyExtractors() throws Excep DataSet> joinDs = ds1.join(ds2) - .where( - new KeySelector5() - ) - .equalTo( - new KeySelector6() - ); + .where(new KeySelector5()) + .equalTo(new KeySelector6()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "1,0,Hi,1,0,Hi\n" + + String expected = "1,0,Hi,1,0,Hi\n" + "2,1,Hello,2,1,Hello\n" + "2,1,Hello,2,2,Hello world\n" + "2,2,Hello world,2,1,Hello\n" + "2,2,Hello world,2,2,Hello world\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector5 implements KeySelector { @@ -441,19 +431,20 @@ public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); DataSet> joinDs = ds1.join(ds2) - .where(new KeySelector3()) - .equalTo(new KeySelector4()) - .with(new T3T5FlatJoin()); + .where(new KeySelector3()) + .equalTo(new KeySelector4()) + .with(new T3T5FlatJoin()); - joinDs.writeAsCsv(resultPath); - env.execute(); + List> result = joinDs.collect(); - expected = "Hi,Hallo\n" + + String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector3 implements KeySelector, Tuple2> { @@ -486,12 +477,13 @@ public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception DataSet >> joinDs = ds1.join(ds2).where("nestedPojo.longNumber").equalTo("f6"); - joinDs.writeAsCsv(resultPath); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + + String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -506,12 +498,13 @@ public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exceptio DataSet >> joinDs = ds1.join(ds2).where("nestedPojo.longNumber").equalTo(6); // <--- difference! - joinDs.writeAsCsv(resultPath); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + + String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -526,13 +519,14 @@ public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exceptio DataSet >> joinDs = ds1.join(ds2).where("nestedPojo.longNumber", "number", "str").equalTo("f6","f0","f1"); - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + + String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -547,13 +541,14 @@ public void testNestedIntoTuple() throws Exception { DataSet >> joinDs = ds1.join(ds2).where("nestedPojo.longNumber", "number","nestedTupleWithCustom.f0").equalTo("f6","f0","f2"); - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + + String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -568,13 +563,14 @@ public void testNestedIntoTupleIntoPojo() throws Exception { DataSet >> joinDs = ds1.join(ds2).where("nestedTupleWithCustom.f0","nestedTupleWithCustom.f1.myInt","nestedTupleWithCustom.f1.myLong").equalTo("f2","f3","f4"); - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + + String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -589,14 +585,15 @@ public void testNonPojoToVerifyFullTupleKeys() throws Exception { DataSet, String>, Tuple2, String> >> joinDs = ds1.join(ds2).where(0).equalTo("f0.f0", "f0.f1"); // key is now Tuple2 - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List, String>, Tuple2, String>>> result = joinDs.collect(); - expected = "((1,1),one),((1,1),one)\n" + + String expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3),three)\n"; + compareResultAsTuples(result, expected); + } @Test @@ -611,13 +608,14 @@ public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception { DataSet, String>, Tuple2, String> >> joinDs = ds1.join(ds2).where("f0.f0").equalTo("f0.f0"); // key is now Integer from Tuple2 - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List, String>, Tuple2, String>>> result = joinDs.collect(); - expected = "((1,1),one),((1,1),one)\n" + + String expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3),three)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -632,18 +630,19 @@ public void testFullPojoWithFullTuple() throws Exception { DataSet >> joinDs = ds1.join(ds2).where("*").equalTo("*"); - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List>> result = joinDs.collect(); - expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n"+ + String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" + "2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n"+ "3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n"; + + compareResultAsTuples(result, expected); } @Test public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreaterThanZero() - throws Exception { + throws Exception { /* * Non-POJO test to verify "nested" tuple-element selection with the first key field greater than 0. */ @@ -654,13 +653,14 @@ public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreat DataSet, Tuple3>, Tuple2, Tuple3>>> joinDs = ds2.join(ds2).where("f1.f0").equalTo("f0.f0"); - joinDs.writeAsCsv(resultPath); env.setParallelism(1); - env.execute(); + List, Tuple3>, Tuple2, Tuple3>>> result = joinDs.collect(); - expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" + + String expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" + "((2,2,Hello),(2,2,Hello)),((2,2,Hello),(2,2,Hello))\n" + "((3,2,Hello world),(3,2,Hello world)),((3,2,Hello world),(3,2,Hello world))\n"; + + compareResultAsTuples(result, expected); } @Test @@ -672,11 +672,12 @@ public void testJoinWithAtomicType1() throws Exception { DataSet, Integer>> joinDs = ds1.join(ds2).where(0).equalTo("*"); - joinDs.writeAsCsv(resultPath); - env.execute(); + List, Integer>> result = joinDs.collect(); - expected = "(1,1,Hi),1\n" + - "(2,2,Hello),2"; + String expected = "(1,1,Hi),1\n" + + "(2,2,Hello),2"; + + compareResultAsTuples(result, expected); } public void testJoinWithAtomicType2() throws Exception { @@ -687,11 +688,12 @@ public void testJoinWithAtomicType2() throws Exception { DataSet>> joinDs = ds1.join(ds2).where("*").equalTo(0); - joinDs.writeAsCsv(resultPath); - env.execute(); + List>> result = joinDs.collect(); + + String expected = "1,(1,1,Hi)\n" + + "2,(2,2,Hello)"; - expected = "1,(1,1,Hi)\n" + - "2,(2,2,Hello)"; + compareResultAsTuples(result, expected); } public static class T3T5FlatJoin implements FlatJoinFunction, Tuple5, Tuple2> { @@ -705,41 +707,41 @@ public void join(Tuple3 first, } } - + public static class LeftReturningJoin implements JoinFunction, Tuple5, Tuple3> { @Override public Tuple3 join(Tuple3 first, - Tuple5 second) { - + Tuple5 second) { + return first; } } - + public static class RightReturningJoin implements JoinFunction, Tuple5, Tuple5> { @Override public Tuple5 join(Tuple3 first, - Tuple5 second) { - + Tuple5 second) { + return second; } } - + public static class T3T5BCJoin extends RichFlatJoinFunction, Tuple5, Tuple3> { private int broadcast; - + @Override public void open(Configuration config) { - + Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for(Integer i : ints) { sum += i; } broadcast = sum; - + } /* @@ -750,24 +752,24 @@ public Tuple3 join( return new Tuple3(first.f2, second.f3, broadcast); } - */ + */ @Override public void join(Tuple3 first, Tuple5 second, Collector> out) throws Exception { out.collect(new Tuple3 (first.f2, second.f3, broadcast)); } } - + public static class T3CustJoin implements JoinFunction, CustomType, Tuple2> { @Override public Tuple2 join(Tuple3 first, - CustomType second) { + CustomType second) { return new Tuple2(first.f2, second.myString); } } - + public static class CustT3Join implements JoinFunction, Tuple2> { @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java index 083f0a32abae6..e5a494b2f0272 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java @@ -19,21 +19,17 @@ package org.apache.flink.test.javaApiOperators; import java.util.Collection; +import java.util.List; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; import org.junit.Assert; - import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -46,22 +42,6 @@ public MapITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testIdentityMapWithBasicType() throws Exception { /* @@ -74,10 +54,9 @@ public void testIdentityMapWithBasicType() throws Exception { DataSet identityMapDs = ds. map(new Mapper1()); - identityMapDs.writeAsText(resultPath); - env.execute(); + List result = identityMapDs.collect(); - expected = "Hi\n" + + String expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + @@ -85,6 +64,8 @@ public void testIdentityMapWithBasicType() throws Exception { "Luke Skywalker\n" + "Random comment\n" + "LOL\n"; + + compareResultAsText(result, expected); } public static class Mapper1 implements MapFunction { @@ -108,10 +89,9 @@ public void testIdentityMapWithTuple() throws Exception { DataSet> identityMapDs = ds. map(new Mapper2()); - identityMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = identityMapDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" + @@ -132,6 +112,8 @@ public void testIdentityMapWithTuple() throws Exception { "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class Mapper2 implements MapFunction, Tuple3> { @@ -139,7 +121,7 @@ public static class Mapper2 implements MapFunction @Override public Tuple3 map(Tuple3 value) - throws Exception { + throws Exception { return value; } } @@ -156,10 +138,9 @@ public void testTypeConversionMapperCustomToTuple() throws Exception { DataSet> typeConversionMapDs = ds. map(new Mapper3()); - typeConversionMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = typeConversionMapDs.collect(); - expected = "1,0,Hi\n" + + String expected = "1,0,Hi\n" + "2,1,Hello\n" + "2,2,Hello world\n" + "3,3,Hello world, how are you?\n" + @@ -180,6 +161,8 @@ public void testTypeConversionMapperCustomToTuple() throws Exception { "6,18,Comment#13\n" + "6,19,Comment#14\n" + "6,20,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class Mapper3 implements MapFunction> { @@ -207,10 +190,9 @@ public void testTypeConversionMapperTupleToBasic() throws Exception { DataSet typeConversionMapDs = ds. map(new Mapper4()); - typeConversionMapDs.writeAsText(resultPath); - env.execute(); + List result = typeConversionMapDs.collect(); - expected = "Hi\n" + "Hello\n" + "Hello world\n" + + String expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + "I am fine.\n" + "Luke Skywalker\n" + "Comment#1\n" + "Comment#2\n" + @@ -221,6 +203,8 @@ public void testTypeConversionMapperTupleToBasic() throws Exception { "Comment#11\n" + "Comment#12\n" + "Comment#13\n" + "Comment#14\n" + "Comment#15\n"; + + compareResultAsText(result, expected); } public static class Mapper4 implements MapFunction, String> { @@ -234,7 +218,7 @@ public String map(Tuple3 value) throws Exception { @Test public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() throws - Exception { + Exception { /* * Test mapper on tuple - Increment Integer field, reorder second and third fields */ @@ -245,10 +229,9 @@ public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() DataSet> tupleMapDs = ds. map(new Mapper5()); - tupleMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = tupleMapDs.collect(); - expected = "2,Hi,1\n" + + String expected = "2,Hi,1\n" + "3,Hello,2\n" + "4,Hello world,2\n" + "5,Hello world, how are you?,3\n" + @@ -269,6 +252,8 @@ public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() "20,Comment#13,6\n" + "21,Comment#14,6\n" + "22,Comment#15,6\n"; + + compareResultAsTuples(result, expected); } public static class Mapper5 implements MapFunction, Tuple3> { @@ -277,7 +262,7 @@ public static class Mapper5 implements MapFunction @Override public Tuple3 map(Tuple3 value) - throws Exception { + throws Exception { Integer incr = Integer.valueOf(value.f0.intValue() + 1); out.setFields(incr, value.f2, value.f1); return out; @@ -296,10 +281,9 @@ public void testMapperOnCustomLowercaseString() throws Exception { DataSet customMapDs = ds. map(new Mapper6()); - customMapDs.writeAsText(resultPath); - env.execute(); + List result = customMapDs.collect(); - expected = "1,0,hi\n" + + String expected = "1,0,hi\n" + "2,1,hello\n" + "2,2,hello world\n" + "3,3,hello world, how are you?\n" + @@ -320,6 +304,8 @@ public void testMapperOnCustomLowercaseString() throws Exception { "6,18,comment#13\n" + "6,19,comment#14\n" + "6,20,comment#15\n"; + + compareResultAsText(result, expected); } public static class Mapper6 implements MapFunction { @@ -347,10 +333,9 @@ public void test() throws Exception { DataSet> inputObjMapDs = ds. map(new Mapper7()); - inputObjMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = inputObjMapDs.collect(); - expected = "2,1,Hi\n" + + String expected = "2,1,Hi\n" + "3,2,Hello\n" + "4,2,Hello world\n" + "5,3,Hello world, how are you?\n" + @@ -371,6 +356,8 @@ public void test() throws Exception { "20,6,Comment#13\n" + "21,6,Comment#14\n" + "22,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class Mapper7 implements MapFunction, Tuple3> { @@ -378,7 +365,7 @@ public static class Mapper7 implements MapFunction @Override public Tuple3 map(Tuple3 value) - throws Exception { + throws Exception { Integer incr = Integer.valueOf(value.f0.intValue() + 1); value.setField(incr, 0); return value; @@ -398,10 +385,9 @@ public void testMapWithBroadcastSet() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> bcMapDs = ds. map(new RichMapper1()).withBroadcastSet(ints, "ints"); - bcMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = bcMapDs.collect(); - expected = "55,1,Hi\n" + + String expected = "55,1,Hi\n" + "55,2,Hello\n" + "55,2,Hello world\n" + "55,3,Hello world, how are you?\n" + @@ -422,6 +408,8 @@ public void testMapWithBroadcastSet() throws Exception { "55,6,Comment#13\n" + "55,6,Comment#14\n" + "55,6,Comment#15\n"; + + compareResultAsTuples(result, expected); } public static class RichMapper1 extends RichMapFunction, @@ -442,7 +430,7 @@ public void open(Configuration config) { @Override public Tuple3 map(Tuple3 value) - throws Exception { + throws Exception { out.setFields(f2Replace, value.f1, value.f2); return out; } @@ -464,12 +452,13 @@ public void testPassingConfigurationObject() throws Exception { conf.setInteger(testKey, testValue); DataSet> bcMapDs = ds. map(new RichMapper2()).withParameters(conf); - bcMapDs.writeAsCsv(resultPath); - env.execute(); + List> result = bcMapDs.collect(); - expected = "1,1,Hi\n" + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world"; + + compareResultAsTuples(result, expected); } public static class RichMapper2 extends RichMapFunction, diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java index 3637680e0ce67..c231eac037930 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.javaApiOperators; import java.util.HashSet; +import java.util.List; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -34,11 +35,7 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -49,22 +46,6 @@ public PartitionITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testHashPartitionByKeyField() throws Exception { /* @@ -77,15 +58,16 @@ public void testHashPartitionByKeyField() throws Exception { DataSet uniqLongs = ds .partitionByHash(1) .mapPartition(new UniqueLongMapper()); - uniqLongs.writeAsText(resultPath); - env.execute(); + List result = uniqLongs.collect(); - expected = "1\n" + + String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; + + compareResultAsText(result, expected); } @Test @@ -100,15 +82,16 @@ public void testHashPartitionByKeySelector() throws Exception { DataSet uniqLongs = ds .partitionByHash(new KeySelector1()) .mapPartition(new UniqueLongMapper()); - uniqLongs.writeAsText(resultPath); - env.execute(); + List result = uniqLongs.collect(); - expected = "1\n" + + String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; + + compareResultAsText(result, expected); } public static class KeySelector1 implements KeySelector, Long> { @@ -134,26 +117,25 @@ public void testForcedRebalancing() throws Exception { DataSet> uniqLongs = ds // introduce some partition skew by filtering .filter(new Filter1()) - // rebalance + // rebalance .rebalance() - // count values in each partition + // count values in each partition .map(new PartitionIndexMapper()) .groupBy(0) .reduce(new Reducer1()) - // round counts to mitigate runtime scheduling effects (lazy split assignment) + // round counts to mitigate runtime scheduling effects (lazy split assignment) .map(new Mapper1()); - uniqLongs.writeAsText(resultPath); + List> result = uniqLongs.collect(); - env.execute(); - - StringBuilder result = new StringBuilder(); + StringBuilder expected = new StringBuilder(); int numPerPartition = 2220 / env.getParallelism() / 10; for (int i = 0; i < env.getParallelism(); i++) { - result.append('(').append(i).append(',').append(numPerPartition).append(")\n"); + expected.append('(').append(i).append(',') + .append(numPerPartition).append(")\n"); } - expected = result.toString(); + compareResultAsText(result, expected.toString()); } public static class Filter1 implements FilterFunction { @@ -172,13 +154,14 @@ public boolean filter(Long value) throws Exception { public static class Reducer1 implements ReduceFunction> { private static final long serialVersionUID = 1L; + @Override public Tuple2 reduce(Tuple2 v1, Tuple2 v2) { return new Tuple2(v1.f0, v1.f1+v2.f1); } } public static class Mapper1 implements MapFunction, Tuple2>{ + Integer>>{ private static final long serialVersionUID = 1L; @Override @@ -202,16 +185,16 @@ public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exceptio DataSet uniqLongs = ds .partitionByHash(1).setParallelism(4) .mapPartition(new UniqueLongMapper()); - uniqLongs.writeAsText(resultPath); + List result = uniqLongs.collect(); - env.execute(); - - expected = "1\n" + + String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; + + compareResultAsText(result, expected); } @Test @@ -227,13 +210,13 @@ public void testHashPartitionWithKeyExpression() throws Exception { DataSet uniqLongs = ds .partitionByHash("nestedPojo.longNumber").setParallelism(4) .mapPartition(new UniqueNestedPojoLongMapper()); - uniqLongs.writeAsText(resultPath); + List result = uniqLongs.collect(); - env.execute(); - - expected = "10000\n" + + String expected = "10000\n" + "20000\n" + "30000\n"; + + compareResultAsText(result, expected); } public static class UniqueLongMapper implements MapPartitionFunction, Long> { @@ -250,7 +233,7 @@ public void mapPartition(Iterable> records, Collec } } } - + public static class UniqueNestedPojoLongMapper implements MapPartitionFunction { private static final long serialVersionUID = 1L; @@ -265,7 +248,7 @@ public void mapPartition(Iterable records, Collector out) throws Exc } } } - + public static class PartitionIndexMapper extends RichMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java index aa75836a67d3e..1054c6258ab96 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java @@ -18,6 +18,8 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; @@ -27,14 +29,6 @@ public class ProjectITCase extends JavaProgramTestBase { - private String resultPath; - private String expectedResult; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - @Override protected void testProgram() throws Exception { /* @@ -45,11 +39,10 @@ protected void testProgram() throws Exception { DataSet> ds = CollectionDataSets.get5TupleDataSet(env); DataSet> projDs = ds. - project(3,4,2); - projDs.writeAsCsv(resultPath); + project(3,4,2); + List> result = projDs.collect(); - env.execute(); - expectedResult = "Hallo,1,0\n" + + String expectedResult = "Hallo,1,0\n" + "Hallo Welt,2,1\n" + "Hallo Welt wie,1,2\n" + "Hallo Welt wie gehts?,2,3\n" + @@ -64,10 +57,8 @@ protected void testProgram() throws Exception { "IJK,3,12\n" + "JKL,2,13\n" + "KLM,2,14\n"; + + compareResultAsTuples(result, expectedResult); } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expectedResult, resultPath); - } + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java index 02324641356ef..4259b63aeb1c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Date; +import java.util.List; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -37,12 +38,8 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.PojoWithDateAndEnum; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -54,22 +51,6 @@ public ReduceITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testReduceOnTuplesWithKeyFieldSelector() throws Exception { /* @@ -82,15 +63,16 @@ public void testReduceOnTuplesWithKeyFieldSelector() throws Exception { DataSet> reduceDs = ds. groupBy(1).reduce(new Tuple3Reduce("B-)")); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n"; + + compareResultAsTuples(result, expected); } @Test @@ -105,10 +87,10 @@ public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception{ DataSet> reduceDs = ds. groupBy(4,0).reduce(new Tuple5Reduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs + .collect(); - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0,P-),2\n" + @@ -118,6 +100,8 @@ public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception{ "5,11,10,GHI,1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n"; + + compareResultAsTuples(result, expected); } @Test @@ -132,15 +116,16 @@ public void testReduceOnTuplesWithKeyExtractor() throws Exception { DataSet> reduceDs = ds. groupBy(new KeySelector1()).reduce(new Tuple3Reduce("B-)")); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "5,2,B-)\n" + "15,3,B-)\n" + "34,4,B-)\n" + "65,5,B-)\n" + "111,6,B-)\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector1 implements KeySelector, Long> { @@ -163,15 +148,16 @@ public void testReduceOnCustomTypeWithKeyExtractor() throws Exception { DataSet reduceDs = ds. groupBy(new KeySelector2()).reduce(new CustomTypeReduce()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); - expected = "1,0,Hi\n" + + String expected = "1,0,Hi\n" + "2,3,Hello!\n" + "3,12,Hello!\n" + "4,30,Hello!\n" + "5,60,Hello!\n" + "6,105,Hello!\n"; + + compareResultAsText(result, expected); } public static class KeySelector2 implements KeySelector { @@ -194,10 +180,11 @@ public void testAllReduceForTuple() throws Exception { DataSet> reduceDs = ds. reduce(new AllAddingTuple3Reduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "231,91,Hello World\n"; + String expected = "231,91,Hello World\n"; + + compareResultAsTuples(result, expected); } @Test @@ -212,10 +199,11 @@ public void testAllReduceForCustomTypes() throws Exception { DataSet reduceDs = ds. reduce(new AllAddingCustomTypeReduce()); - reduceDs.writeAsText(resultPath); - env.execute(); + List result = reduceDs.collect(); + + String expected = "91,210,Hello!"; - expected = "91,210,Hello!"; + compareResultAsText(result, expected); } @Test @@ -232,15 +220,16 @@ public void testReduceWithBroadcastSet() throws Exception { DataSet> reduceDs = ds. groupBy(1).reduce(new BCTuple3Reduce()).withBroadcastSet(intDs, "ints"); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs.collect(); - expected = "1,1,Hi\n" + + String expected = "1,1,Hi\n" + "5,2,55\n" + "15,3,55\n" + "34,4,55\n" + "65,5,55\n" + "111,6,55\n"; + + compareResultAsTuples(result, expected); } @Test @@ -255,10 +244,10 @@ public void testReduceATupleReturningKeySelector() throws Exception { DataSet> reduceDs = ds . groupBy(new KeySelector3()).reduce(new Tuple5Reduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs + .collect(); - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0,P-),2\n" + @@ -268,6 +257,8 @@ public void testReduceATupleReturningKeySelector() throws Exception { "5,11,10,GHI,1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n"; + + compareResultAsTuples(result, expected); } public static class KeySelector3 implements KeySelector, Tuple2> { @@ -291,10 +282,10 @@ public void testReduceOnTupleWithMultipleKeyExpressions() throws Exception { DataSet> reduceDs = ds. groupBy("f4","f0").reduce(new Tuple5Reduce()); - reduceDs.writeAsCsv(resultPath); - env.execute(); + List> result = reduceDs + .collect(); - expected = "1,1,0,Hallo,1\n" + + String expected = "1,1,0,Hallo,1\n" + "2,3,2,Hallo Welt wie,1\n" + "2,2,1,Hallo Welt,2\n" + "3,9,0,P-),2\n" + @@ -304,6 +295,8 @@ public void testReduceOnTupleWithMultipleKeyExpressions() throws Exception { "5,11,10,GHI,1\n" + "5,29,0,P-),2\n" + "5,25,0,P-),3\n"; + + compareResultAsTuples(result, expected); } @Test @@ -317,9 +310,11 @@ public void testSupportForDataAndEnumSerialization() throws Exception { DataSet res = ds.groupBy("group").reduceGroup(new GroupReducer1()); - res.writeAsText(resultPath); - env.execute(); - expected = "ok\nok"; + List result = res.collect(); + + String expected = "ok\nok"; + + compareResultAsText(result, expected); } public static class Mapper1 implements MapFunction { @@ -369,20 +364,20 @@ public void reduce(Iterable values, out.collect("ok"); } } - + public static class Tuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); private final String f2Replace; - - public Tuple3Reduce() { + + public Tuple3Reduce() { this.f2Replace = null; } - - public Tuple3Reduce(String f2Replace) { + + public Tuple3Reduce(String f2Replace) { this.f2Replace = f2Replace; } - + @Override public Tuple3 reduce( @@ -397,41 +392,41 @@ public Tuple3 reduce( return out; } } - + public static class Tuple5Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple5 out = new Tuple5(); - + @Override public Tuple5 reduce( Tuple5 in1, Tuple5 in2) - throws Exception { - + throws Exception { + out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4); return out; } } - + public static class CustomTypeReduce implements ReduceFunction { private static final long serialVersionUID = 1L; private final CustomType out = new CustomType(); - + @Override public CustomType reduce(CustomType in1, CustomType in2) throws Exception { - + out.myInt = in1.myInt; out.myLong = in1.myLong + in2.myLong; out.myString = "Hello!"; return out; } } - + public static class AllAddingTuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); - + @Override public Tuple3 reduce( Tuple3 in1, @@ -441,37 +436,37 @@ public Tuple3 reduce( return out; } } - + public static class AllAddingCustomTypeReduce implements ReduceFunction { private static final long serialVersionUID = 1L; private final CustomType out = new CustomType(); - + @Override public CustomType reduce(CustomType in1, CustomType in2) throws Exception { - + out.myInt = in1.myInt + in2.myInt; out.myLong = in1.myLong + in2.myLong; out.myString = "Hello!"; return out; } } - + public static class BCTuple3Reduce extends RichReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); private String f2Replace = ""; - + @Override public void open(Configuration config) { - + Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for(Integer i : ints) { sum += i; } f2Replace = sum+""; - + } @Override @@ -483,5 +478,5 @@ public Tuple3 reduce( return out; } } - + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java index c7ca37dad8890..8cc54aac2b326 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java @@ -19,6 +19,8 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.io.ReplicatingInputFormat; @@ -32,11 +34,7 @@ import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.NumberSequenceIterator; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -51,23 +49,6 @@ public ReplicatingDataSourceITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - - private String expectedResult; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expectedResult, resultPath); // 500500 = 0+1+2+3+...+999+1000 - } - @Test public void testReplicatedSourceToJoin() throws Exception { /* @@ -85,11 +66,11 @@ public void testReplicatedSourceToJoin() throws Exception { .projectFirst(0) .sum(0); - pairs.writeAsText(resultPath); - env.execute(); + List result = pairs.collect(); - expectedResult = "(500500)"; + String expectedResult = "(500500)"; + compareResultAsText(result, expectedResult); } @Test @@ -120,11 +101,11 @@ public Tuple1 map(Tuple2, Tuple1> value) throws Excepti }) .sum(0); - pairs.writeAsText(resultPath); - env.execute(); + List> result = pairs.collect(); - expectedResult = "(500500)"; + String expectedResult = "(500500)"; + compareResultAsText(result, expectedResult); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java index d961f3ae4a978..2b7226b8f5ddf 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java @@ -31,16 +31,13 @@ import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.util.Collector; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.Serializable; import java.util.Iterator; +import java.util.List; @RunWith(Parameterized.class) public class SortPartitionITCase extends MultipleProgramsTestBase { @@ -49,22 +46,6 @@ public SortPartitionITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testSortPartitionByKeyField() throws Exception { /* @@ -75,16 +56,15 @@ public void testSortPartitionByKeyField() throws Exception { env.setParallelism(4); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(4) // parallelize input .sortPartition(1, Order.DESCENDING) .mapPartition(new OrderCheckMapper>(new Tuple3Checker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } @Test @@ -97,19 +77,19 @@ public void testSortPartitionByTwoKeyFields() throws Exception { env.setParallelism(2); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(2) // parallelize input .sortPartition(4, Order.ASCENDING) .sortPartition(2, Order.DESCENDING) .mapPartition(new OrderCheckMapper>(new Tuple5Checker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } + @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testSortPartitionByFieldExpression() throws Exception { /* @@ -120,16 +100,15 @@ public void testSortPartitionByFieldExpression() throws Exception { env.setParallelism(4); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(4) // parallelize input .sortPartition("f1", Order.DESCENDING) .mapPartition(new OrderCheckMapper>(new Tuple3Checker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } @Test @@ -142,17 +121,16 @@ public void testSortPartitionByTwoFieldExpressions() throws Exception { env.setParallelism(2); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(2) // parallelize input .sortPartition("f4", Order.ASCENDING) .sortPartition("f2", Order.DESCENDING) .mapPartition(new OrderCheckMapper>(new Tuple5Checker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } @Test @@ -165,17 +143,16 @@ public void testSortPartitionByNestedFieldExpression() throws Exception { env.setParallelism(3); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(3) // parallelize input .sortPartition("f0.f1", Order.ASCENDING) .sortPartition("f1", Order.DESCENDING) .mapPartition(new OrderCheckMapper, String>>(new NestedTupleChecker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } @Test @@ -188,17 +165,16 @@ public void testSortPartitionPojoByNestedFieldExpression() throws Exception { env.setParallelism(3); DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); - ds + List result = ds .map(new IdMapper()).setParallelism(1) // parallelize input .sortPartition("nestedTupleWithCustom.f1.myString", Order.ASCENDING) .sortPartition("number", Order.DESCENDING) .mapPartition(new OrderCheckMapper(new PojoChecker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } @Test @@ -211,15 +187,14 @@ public void testSortPartitionParallelismChange() throws Exception { env.setParallelism(3); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds + List result = ds .sortPartition(1, Order.DESCENDING).setParallelism(3) // change parallelism .mapPartition(new OrderCheckMapper>(new Tuple3Checker())) - .distinct() - .writeAsText(resultPath); + .distinct().collect(); - env.execute(); + String expected = "(true)\n"; - expected = "(true)\n"; + compareResultAsText(result, expected); } public static interface OrderChecker extends Serializable { @@ -237,7 +212,7 @@ public boolean inOrder(Tuple3 t1, Tuple3> { @Override public boolean inOrder(Tuple5 t1, - Tuple5 t2) { + Tuple5 t2) { return t1.f4 < t2.f4 || t1.f4 == t2.f4 && t1.f2 >= t2.f2; } } @@ -245,19 +220,18 @@ public boolean inOrder(Tuple5 t1, public static class NestedTupleChecker implements OrderChecker, String>> { @Override public boolean inOrder(Tuple2, String> t1, - Tuple2, String> t2) { + Tuple2, String> t2) { return t1.f0.f1 < t2.f0.f1 || t1.f0.f1 == t2.f0.f1 && t1.f1.compareTo(t2.f1) >= 0; - } + } } public static class PojoChecker implements OrderChecker { @Override - public boolean inOrder(POJO t1, - POJO t2) { + public boolean inOrder(POJO t1, POJO t2) { return t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) < 0 || t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) == 0 && - t1.number >= t2.number; + t1.number >= t2.number; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java index e6367c361c578..e5bdc196f4f68 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java @@ -18,6 +18,8 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; @@ -25,11 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -40,22 +38,6 @@ public SumMinMaxITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testSumMaxAndProject() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -66,10 +48,11 @@ public void testSumMaxAndProject() throws Exception { .andMax(1) .project(0, 1); - sumDs.writeAsCsv(resultPath); - env.execute(); + List> result = sumDs.collect(); + + String expected = "231,6\n"; - expected = "231,6\n"; + compareResultAsTuples(result, expected); } @Test @@ -85,15 +68,16 @@ public void testGroupedAggregate() throws Exception { .sum(0) .project(1, 0); - aggregateDs.writeAsCsv(resultPath); - env.execute(); + List> result = aggregateDs.collect(); - expected = "1,1\n" + + String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; + + compareResultAsTuples(result, expected); } @Test @@ -110,9 +94,10 @@ public void testNestedAggregate() throws Exception { .min(0) .project(0); - aggregateDs.writeAsCsv(resultPath); - env.execute(); + List> result = aggregateDs.collect(); + + String expected = "1\n"; - expected = "1\n"; + compareResultAsTuples(result, expected); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java index 350227a965e1e..a2c10bcf07139 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.Collection; import java.util.LinkedList; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -44,25 +45,14 @@ public class TypeHintITCase extends JavaProgramTestBase { private static int NUM_PROGRAMS = 3; private int curProgId = config.getInteger("ProgramId", -1); - private String resultPath; - private String expectedResult; public TypeHintITCase(Configuration config) { super(config); } - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } @Override protected void testProgram() throws Exception { - expectedResult = TypeHintProgs.runProgram(curProgId, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expectedResult, resultPath); + TypeHintProgs.runProgram(curProgId); } @Parameters @@ -81,7 +71,7 @@ public static Collection getConfigurations() throws FileNotFoundExcept private static class TypeHintProgs { - public static String runProgram(int progId, String resultPath) throws Exception { + public static void runProgram(int progId) throws Exception { switch(progId) { // Test identity map with missing types and string type hint case 1: { @@ -91,13 +81,14 @@ public static String runProgram(int progId, String resultPath) throws Exception DataSet> identityMapDs = ds .map(new Mapper, Tuple3>()) .returns("Tuple3"); - identityMapDs.writeAsText(resultPath); - env.execute(); + List> result = identityMapDs.collect(); - // return expected result - return "(2,2,Hello)\n" + - "(3,2,Hello world)\n" + - "(1,1,Hi)\n"; + String expectedResult = "(2,2,Hello)\n" + + "(3,2,Hello world)\n" + + "(1,1,Hi)\n"; + + compareResultAsText(result, expectedResult); + break; } // Test identity map with missing types and type information type hint case 2: { @@ -108,32 +99,34 @@ public static String runProgram(int progId, String resultPath) throws Exception // all following generics get erased during compilation .map(new Mapper, Tuple3>()) .returns(new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)); - identityMapDs.writeAsText(resultPath); - env.execute(); + List> result = identityMapDs + .collect(); + + String expectedResult = "(2,2,Hello)\n" + + "(3,2,Hello world)\n" + + "(1,1,Hi)\n"; - // return expected result - return "(2,2,Hello)\n" + - "(3,2,Hello world)\n" + - "(1,1,Hi)\n"; + compareResultAsText(result, expectedResult); + break; } // Test flat map with class type hint case 3: { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - @SuppressWarnings({ "rawtypes", "unchecked" }) DataSet identityMapDs = ds. flatMap(new FlatMapper, Integer>()) - .returns((Class) Integer.class); - identityMapDs.writeAsText(resultPath); - env.execute(); - - // return expected result - return "2\n" + - "3\n" + - "1\n"; + .returns(Integer.class); + List result = identityMapDs.collect(); + + String expectedResult = "2\n" + + "3\n" + + "1\n"; + + compareResultAsText(result, expectedResult); + break; } - default: + default: throw new IllegalArgumentException("Invalid program id"); } } @@ -150,7 +143,7 @@ public V map(T value) throws Exception { return (V) value; } } - + public static class FlatMapper implements FlatMapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java index 2e7ae9c85823f..7ab2764c3c380 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java @@ -18,15 +18,13 @@ package org.apache.flink.test.javaApiOperators; +import java.util.List; + import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.flink.api.java.DataSet; @@ -61,22 +59,6 @@ public UnionITCase(TestExecutionMode mode){ super(mode); } - private String resultPath; - private String expected; - - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception{ - resultPath = tempFolder.newFile().toURI().toString(); - } - - @After - public void after() throws Exception{ - compareResultsByLinesInMemory(expected, resultPath); - } - @Test public void testUnion2IdenticalDataSets() throws Exception { /* @@ -87,10 +69,11 @@ public void testUnion2IdenticalDataSets() throws Exception { DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env)); - unionDs.writeAsCsv(resultPath); - env.execute(); + List> result = unionDs.collect(); + + String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING; - expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING; + compareResultAsTuples(result, expected); } @Test @@ -107,11 +90,13 @@ public void testUnion5IdenticalDataSets() throws Exception { .union(CollectionDataSets.get3TupleDataSet(env)) .union(CollectionDataSets.get3TupleDataSet(env)); - unionDs.writeAsCsv(resultPath); - env.execute(); + List> result = unionDs.collect(); - expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING + + String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING + + FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING; + + compareResultAsTuples(result, expected); } @Test @@ -128,10 +113,11 @@ public void testUnionWithEmptyDataSet() throws Exception { DataSet> unionDs = CollectionDataSets.get3TupleDataSet(env) .union(empty); - unionDs.writeAsCsv(resultPath); - env.execute(); + List> result = unionDs.collect(); - expected = FULL_TUPLE_3_STRING; + String expected = FULL_TUPLE_3_STRING; + + compareResultAsTuples(result, expected); } public static class RichFilter1 extends RichFilterFunction> { @@ -142,5 +128,5 @@ public boolean filter(Tuple3 value) throws Exception { return false; } } - + } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java index a68fd8248d16f..1faf4c12fec55 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java @@ -24,6 +24,7 @@ import java.math.BigInteger; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.GregorianCalendar; import java.util.HashMap; @@ -313,6 +314,19 @@ public String toString() { } } + public static class CustomTypeComparator implements Comparator { + @Override + public int compare(CustomType o1, CustomType o2) { + int diff = o1.myInt - o2.myInt; + if (diff != 0) { + return diff; + } + diff = (int) (o1.myLong - o2.myLong); + return diff != 0 ? diff : o1.myString.compareTo(o2.myString); + } + + } + public static DataSet> getSmallTuplebasedDataSet(ExecutionEnvironment env) { List> data = new ArrayList>(); data.add(new Tuple7(1, "First", 10, 100, 1000L, "One", 10000L));