diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java index a766214c7fff9..aff78135e6672 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java @@ -54,7 +54,7 @@ import org.apache.flink.api.common.typeutils.GenericPairComparator; import org.apache.flink.api.java.tuple.Tuple2; -@SuppressWarnings({"serial", "deprecation"}) +@SuppressWarnings({"serial"}) public class NonReusingHashJoinIteratorITCase { private static final int MEMORY_SIZE = 16000000; // total memory diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java index a885e6b84db6c..814d0ed13724c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java @@ -64,7 +64,6 @@ * Test specialized hash join that keeps the build side data (in memory and on hard disk) * This is used for iterative tasks. */ -@SuppressWarnings("deprecation") public class NonReusingReOpenableHashTableITCase { private static final int PAGE_SIZE = 8 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java index 87707a45db4d2..af814c3b17332 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java @@ -62,7 +62,7 @@ import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.collectTupleData; import static org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.collectIntPairData; -@SuppressWarnings({"serial", "deprecation"}) +@SuppressWarnings({"serial"}) public class ReusingHashJoinIteratorITCase { private static final int MEMORY_SIZE = 16000000; // total memory diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java index 582077e0ce6ac..d2229477a6af0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java @@ -66,7 +66,6 @@ * Test specialized hash join that keeps the build side data (in memory and on hard disk) * This is used for iterative tasks. */ -@SuppressWarnings("deprecation") public class ReusingReOpenableHashTableITCase { private static final int PAGE_SIZE = 8 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java index cd40c82539fa4..571152b4dd6b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java @@ -173,7 +173,7 @@ else if (result instanceof CancellationFailure) { } } - private void awaitRunning(ActorGateway jobManager, JobID jobId, FiniteDuration timeout) + public static void awaitRunning(ActorGateway jobManager, JobID jobId, FiniteDuration timeout) throws Exception { checkNotNull(jobManager); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java index 6221c085faa52..6dc0a0bb3b1a7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java @@ -19,124 +19,59 @@ package org.apache.flink.test.accumulators; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.accumulators.IntCounter; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; +import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; import org.junit.Assert; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -/** - * To be finished !!! Didn't test with iterations yet;-( - */ -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class AccumulatorIterativeITCase extends RecordAPITestBase { - private static final String INPUT = "1\n" + "2\n" + "3\n"; - private static final String EXPECTED = "6\n"; - +public class AccumulatorIterativeITCase extends JavaProgramTestBase { private static final int NUM_ITERATIONS = 3; private static final int NUM_SUBTASKS = 1; - - protected String dataPath; - protected String resultPath; - - public AccumulatorIterativeITCase(Configuration config) { - super(config); - } - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", INPUT); - resultPath = getTempFilePath("result"); - } + private static final String ACC_NAME = "test"; @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - - Integer res = getJobExecutionResult().getAccumulatorResult("test"); - Assert.assertEquals(Integer.valueOf(NUM_ITERATIONS * 6), res); + protected boolean skipCollectionExecution() { + return true; } @Override - protected Plan getTestJob() { - Plan plan = getTestPlanPlan(config.getInteger("IterationAllReducer#NoSubtasks", 1), dataPath, resultPath); - return plan; - } - - @Parameters - public static Collection getConfigurations() { - Configuration config1 = new Configuration(); - config1.setInteger("IterationAllReducer#NoSubtasks", NUM_SUBTASKS); - return toParameterList(config1); - } - - static Plan getTestPlanPlan(int numSubTasks, String input, String output) { - - FileDataSource initialInput = new FileDataSource(TextInputFormat.class, input, "input"); + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(NUM_SUBTASKS); - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(NUM_ITERATIONS); - - ReduceOperator sumReduce = ReduceOperator.builder(new SumReducer()) - .input(iteration.getPartialSolution()) - .name("Compute sum (Reduce)") - .build(); + IterativeDataSet iteration = env.fromElements(1, 2, 3).iterate(NUM_ITERATIONS); - iteration.setNextPartialSolution(sumReduce); - - @SuppressWarnings("unchecked") - FileDataSink finalResult = new FileDataSink(new CsvOutputFormat("\n", " ", StringValue.class), output, iteration, "Output"); - - Plan plan = new Plan(finalResult, "Iteration with AllReducer (keyless Reducer)"); - plan.setDefaultParallelism(numSubTasks); - return plan; + iteration.closeWith(iteration.reduceGroup(new SumReducer())).output(new DiscardingOutputFormat()); + + Assert.assertEquals(Integer.valueOf(NUM_ITERATIONS * 6), (Integer)env.execute().getAccumulatorResult(ACC_NAME)); } - static final class SumReducer extends ReduceFunction implements Serializable { + static final class SumReducer extends RichGroupReduceFunction { private static final long serialVersionUID = 1L; private IntCounter testCounter = new IntCounter(); + + @Override + public void open(Configuration config) throws Exception { + getRuntimeContext().addAccumulator(ACC_NAME, this.testCounter); + } @Override - public void reduce(Iterator records, Collector out) { + public void reduce(Iterable values, Collector out) { // Compute the sum int sum = 0; - while (records.hasNext()) { - Record r = records.next(); - Integer value = Integer.parseInt(r.getField(0, StringValue.class).getValue()); + for (Integer value : values) { sum += value; testCounter.add(value); } - out.collect(new Record(new StringValue(Integer.toString(sum)))); - } - - @Override - public void close() throws Exception { - super.close(); - getRuntimeContext().addAccumulator("test", this.testCounter); + out.collect(sum); } } - } diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java index 4868aff29b31d..5f2950c8e77c8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java @@ -19,46 +19,24 @@ package org.apache.flink.test.broadcastvars; import java.util.Collection; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; +import java.util.List; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.FlatMapOperator; +import org.apache.flink.api.java.operators.JoinOperator; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; -@SuppressWarnings("deprecation") -public class BroadcastBranchingITCase extends RecordAPITestBase { - - private static final String SC1_ID_ABC = "1 61 6 29\n2 7 13 10\n3 8 13 27\n"; - - private static final String SC2_ID_X = "1 5\n2 3\n3 6"; - - private static final String SC3_ID_Y = "1 2\n2 3\n3 7"; - - private static final String RESULT = "2 112\n"; - - private String sc1Path; - private String sc2Path; - private String sc3Path; - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - sc1Path = createTempFile("broadcastBranchingInput/map_id_abc.txt", SC1_ID_ABC); - sc2Path = createTempFile("broadcastBranchingInput/map_id_x.txt", SC2_ID_X); - sc3Path = createTempFile("broadcastBranchingInput/map_id_y.txt", SC3_ID_Y); - resultPath = getTempDirPath("result"); - } +public class BroadcastBranchingITCase extends JavaProgramTestBase { + private static final String RESULT = "(2,112)\n"; // Sc1(id,a,b,c) -- // \ @@ -68,58 +46,50 @@ protected void preSubmit() throws Exception { // / // Sc3(id,y) -------- @Override - protected Plan getTestJob() { + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + // Sc1 generates M parameters a,b,c for second degree polynomials P(x) = ax^2 + bx + c identified by id - FileDataSource sc1 = new FileDataSource(new CsvInputFormat(), sc1Path); - CsvInputFormat.configureRecordFormat(sc1).fieldDelimiter(' ').field(StringValue.class, 0).field(IntValue.class, 1) - .field(IntValue.class, 2).field(IntValue.class, 3); + DataSet> sc1 = env + .fromElements(new Tuple4<>("1", 61, 6, 29), new Tuple4<>("2", 7, 13, 10), new Tuple4<>("3", 8, 13, 27)); // Sc2 generates N x values to be evaluated with the polynomial identified by id - FileDataSource sc2 = new FileDataSource(new CsvInputFormat(), sc2Path); - CsvInputFormat.configureRecordFormat(sc2).fieldDelimiter(' ').field(StringValue.class, 0).field(IntValue.class, 1); + DataSet> sc2 = env + .fromElements(new Tuple2<>("1", 5), new Tuple2<>("2", 3), new Tuple2<>("3", 6)); // Sc3 generates N y values to be evaluated with the polynomial identified by id - FileDataSource sc3 = new FileDataSource(new CsvInputFormat(), sc3Path); - CsvInputFormat.configureRecordFormat(sc3).fieldDelimiter(' ').field(StringValue.class, 0).field(IntValue.class, 1); + DataSet> sc3 = env + .fromElements(new Tuple2<>("1", 2), new Tuple2<>("2", 3), new Tuple2<>("3", 7)); // Jn1 matches x and y values on id and emits (id, x, y) triples - JoinOperator jn1 = JoinOperator.builder(Jn1.class, StringValue.class, 0, 0).input1(sc2).input2(sc3).build(); + JoinOperator, Tuple2, Tuple3> jn1 + = sc2.join(sc3).where(0).equalTo(0).with(new Jn1()); // Jn2 matches polynomial and arguments by id, computes p = min(P(x),P(y)) and emits (id, p) tuples - JoinOperator jn2 = JoinOperator.builder(Jn2.class, StringValue.class, 0, 0).input1(jn1).input2(sc1).build(); + JoinOperator, Tuple4, Tuple2> jn2 + = jn1.join(sc1).where(0).equalTo(0).with(new Jn2()); // Mp1 selects (id, x, y) triples where x = y and broadcasts z (=x=y) to Mp2 - MapOperator mp1 = MapOperator.builder(Mp1.class).input(jn1).build(); + FlatMapOperator, Tuple2> mp1 + = jn1.flatMap(new Mp1()); // Mp2 filters out all p values which can be divided by z - MapOperator mp2 = MapOperator.builder(Mp2.class).setBroadcastVariable("z", mp1).input(jn2).build(); - - FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - output.setInput(mp2); + List> result = jn2.flatMap(new Mp2()).withBroadcastSet(mp1, "z").collect(); - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); + JavaProgramTestBase.compareResultAsText(result, RESULT); } - public static class Jn1 extends JoinFunction { + public static class Jn1 implements JoinFunction, Tuple2, Tuple3> { private static final long serialVersionUID = 1L; @Override - public void join(Record sc2, Record sc3, Collector out) throws Exception { - Record r = new Record(3); - r.setField(0, sc2.getField(0, StringValue.class)); - r.setField(1, sc2.getField(1, IntValue.class)); - r.setField(2, sc3.getField(1, IntValue.class)); - out.collect(r); + public Tuple3 join(Tuple2 first, Tuple2 second) throws Exception { + return new Tuple3<>(first.f0, first.f1, second.f1); } } - public static class Jn2 extends JoinFunction { + public static class Jn2 implements JoinFunction, Tuple4, Tuple2> { private static final long serialVersionUID = 1L; private static int p(int x, int a, int b, int c) { @@ -127,35 +97,35 @@ private static int p(int x, int a, int b, int c) { } @Override - public void join(Record jn1, Record sc1, Collector out) throws Exception { - int x = jn1.getField(1, IntValue.class).getValue(); - int y = jn1.getField(2, IntValue.class).getValue(); - int a = sc1.getField(1, IntValue.class).getValue(); - int b = sc1.getField(2, IntValue.class).getValue(); - int c = sc1.getField(3, IntValue.class).getValue(); + public Tuple2 join(Tuple3 first, Tuple4 second) throws Exception { + int x = first.f1; + int y = first.f2; + int a = second.f1; + int b = second.f2; + int c = second.f3; int p_x = p(x, a, b, c); int p_y = p(y, a, b, c); int min = Math.min(p_x, p_y); - out.collect(new Record(jn1.getField(0, StringValue.class), new IntValue(min))); + return new Tuple2<>(first.f0, min); } } - public static class Mp1 extends MapFunction { + public static class Mp1 implements FlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @Override - public void map(Record jn1, Collector out) throws Exception { - if (jn1.getField(1, IntValue.class).getValue() == jn1.getField(2, IntValue.class).getValue()) { - out.collect(new Record(jn1.getField(0, StringValue.class), jn1.getField(1, IntValue.class))); + public void flatMap(Tuple3 value, Collector> out) throws Exception { + if (value.f1.compareTo(value.f2) == 0) { + out.collect(new Tuple2<>(value.f0, value.f1)); } } } - public static class Mp2 extends MapFunction { + public static class Mp2 extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; - private Collection zs; + private Collection> zs; @Override public void open(Configuration parameters) throws Exception { @@ -163,13 +133,13 @@ public void open(Configuration parameters) throws Exception { } @Override - public void map(Record jn2, Collector out) throws Exception { - int p = jn2.getField(1, IntValue.class).getValue(); + public void flatMap(Tuple2 value, Collector> out) throws Exception { + int p = value.f1; - for (Record z : zs) { - if (z.getField(0, StringValue.class).getValue().equals(jn2.getField(0, StringValue.class).getValue())) { - if (p % z.getField(1, IntValue.class).getValue() != 0) { - out.collect(jn2); + for (Tuple2 z : zs) { + if (z.f0.equals(value.f0)) { + if (p % z.f1 != 0) { + out.collect(value); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java index 1a96a1b66995e..14c396da18e8b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java @@ -19,12 +19,9 @@ package org.apache.flink.test.cancelling; -import java.util.concurrent.atomic.AtomicReference; - +import java.util.concurrent.TimeUnit; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobCancellationException; -import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; @@ -38,11 +35,16 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import static org.apache.flink.runtime.taskmanager.TaskCancelTest.awaitRunning; +import static org.apache.flink.runtime.taskmanager.TaskCancelTest.cancelJob; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; import org.apache.flink.util.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.junit.After; import org.junit.Before; +import scala.concurrent.duration.FiniteDuration; /** * @@ -80,8 +82,14 @@ public void startCluster() throws Exception { verifyJvmOptions(); Configuration config = new Configuration(); config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, true); - - this.executor = new ForkableFlinkMiniCluster(config); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4); + config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT()); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, 4096); + config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 2048); + + this.executor = new ForkableFlinkMiniCluster(config, false); + this.executor.start(); } @After @@ -104,43 +112,27 @@ public void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxTime try { // submit job final JobGraph jobGraph = getJobGraph(plan); - - final Thread currentThread = Thread.currentThread(); - final AtomicReference error = new AtomicReference(); - - boolean jobSuccessfullyCancelled = false; - - // trigger the cancelling asynchronous - new Thread() { - @Override - public void run() { - try { - Thread.sleep(msecsTillCanceling); - executor.getLeaderGateway(TestingUtils.TESTING_DURATION()) - .tell(new JobManagerMessages.CancelJob(jobGraph.getJobID())); - } - catch (Throwable t) { - error.set(t); - currentThread.interrupt(); - } - } - }.run(); - - try { - executor.submitJobAndWait(jobGraph, false); - } - catch (JobCancellationException exception) { - jobSuccessfullyCancelled = true; - } - catch (Exception e) { - throw new IllegalStateException("Job failed.", e); - } - - if (!jobSuccessfullyCancelled) { - throw new IllegalStateException("Job was not successfully cancelled."); - } - } - catch(Exception e) { + + executor.submitJobDetached(jobGraph); + + // Wait for the job to make some progress and then cancel + awaitRunning( + executor.getLeaderGateway(TestingUtils.TESTING_DURATION()), + jobGraph.getJobID(), + TestingUtils.TESTING_DURATION()); + + Thread.sleep(msecsTillCanceling); + + cancelJob( + executor.getLeaderGateway(TestingUtils.TESTING_DURATION()), + jobGraph.getJobID(), + new FiniteDuration(maxTimeTillCanceled, TimeUnit.MILLISECONDS)); + + // Wait for the job to be cancelled + JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.CANCELED, + executor.getLeaderGateway(TestingUtils.TESTING_DURATION()), + TestingUtils.TESTING_DURATION()); + } catch (Exception e) { LOG.error("Exception found in runAndCancelJob.", e); Assert.fail(StringUtils.stringifyException(e)); } @@ -154,7 +146,11 @@ private JobGraph getJobGraph(final Plan plan) throws Exception { return jgg.compileJobGraph(op); } - public void setTaskManagerNumSlots(int taskManagerNumSlots) { this.taskManagerNumSlots = taskManagerNumSlots; } + public void setTaskManagerNumSlots(int taskManagerNumSlots) { + this.taskManagerNumSlots = taskManagerNumSlots; + } - public int getTaskManagerNumSlots() { return this.taskManagerNumSlots; } + public int getTaskManagerNumSlots() { + return this.taskManagerNumSlots; + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java new file mode 100644 index 0000000000000..c0d8c7791291f --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java @@ -0,0 +1,177 @@ +/* + * 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.cancelling; + +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.RichJoinFunction; +import org.apache.flink.api.common.operators.base.JoinOperatorBase; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; +import org.apache.flink.test.util.InfiniteIntegerTupleInputFormat; +import org.apache.flink.test.util.UniformIntTupleGeneratorInputFormat; +import org.junit.Test; + +public class JoinCancelingITCase extends CancellingTestBase { + private static final int parallelism = 4; + + public JoinCancelingITCase() { + setTaskManagerNumSlots(parallelism); + } + + // --------------- Test Sort Matches that are canceled while still reading / sorting ----------------- + private void executeTask(JoinFunction, Tuple2, Tuple2> joiner, boolean slow) throws Exception { + executeTask(joiner, slow, parallelism); + } + + private void executeTask(JoinFunction, Tuple2, Tuple2> joiner, boolean slow, int parallelism) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet> input1 = env.createInput(new InfiniteIntegerTupleInputFormat(slow)); + DataSet> input2 = env.createInput(new InfiniteIntegerTupleInputFormat(slow)); + + input1.join(input2, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE) + .where(0) + .equalTo(0) + .with(joiner) + .output(new DiscardingOutputFormat>()); + + env.setParallelism(parallelism); + + runAndCancelJob(env.createProgramPlan(), 5 * 1000, 10 * 1000); + } + +// @Test + public void testCancelSortMatchWhileReadingSlowInputs() throws Exception { + executeTask(new SimpleMatcher(), true); + } + +// @Test + public void testCancelSortMatchWhileReadingFastInputs() throws Exception { + executeTask(new SimpleMatcher(), false); + } + +// @Test + public void testCancelSortMatchPriorToFirstRecordReading() throws Exception { + executeTask(new StuckInOpenMatcher(), false); + } + + private void executeTaskWithGenerator( + JoinFunction, Tuple2, Tuple2> joiner, + int keys, int vals, int msecsTillCanceling, int maxTimeTillCanceled) throws Exception { + UniformIntTupleGenerator g = new UniformIntTupleGenerator(keys, vals, false); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet> input1 = env.createInput(new UniformIntTupleGeneratorInputFormat(keys, vals)); + DataSet> input2 = env.createInput(new UniformIntTupleGeneratorInputFormat(keys, vals)); + + input1.join(input2, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE) + .where(0) + .equalTo(0) + .with(joiner) + .output(new DiscardingOutputFormat>()); + + env.setParallelism(parallelism); + + runAndCancelJob(env.createProgramPlan(), msecsTillCanceling, maxTimeTillCanceled); + } + +// @Test + public void testCancelSortMatchWhileDoingHeavySorting() throws Exception { + executeTaskWithGenerator(new SimpleMatcher(), 50000, 100, 30 * 1000, 30 * 1000); + } + + // --------------- Test Sort Matches that are canceled while in the Matching Phase ----------------- + +// @Test + public void testCancelSortMatchWhileJoining() throws Exception { + executeTaskWithGenerator(new DelayingMatcher(), 500, 3, 10 * 1000, 20 * 1000); + } + +// @Test + public void testCancelSortMatchWithLongCancellingResponse() throws Exception { + executeTaskWithGenerator(new LongCancelTimeMatcher(), 500, 3, 10 * 1000, 10 * 1000); + } + + // -------------------------------------- Test System corner cases --------------------------------- + +// @Test + public void testCancelSortMatchWithHighparallelism() throws Exception { + executeTask(new SimpleMatcher(), false, 64); + } + + // -------------------------------------------------------------------------------------------- + + public static final class SimpleMatcher implements JoinFunction, Tuple2, Tuple2> { + private static final long serialVersionUID = 1L; + + @Override + public Tuple2 join(Tuple2 first, Tuple2 second) throws Exception { + return new Tuple2<>(first.f0, second.f0); + } + } + + public static final class DelayingMatcher implements JoinFunction, Tuple2, Tuple2> { + private static final long serialVersionUID = 1L; + + private static final int WAIT_TIME_PER_RECORD = 10 * 1000; // 10 sec. + + @Override + public Tuple2 join(Tuple2 first, Tuple2 second) throws Exception { + Thread.sleep(WAIT_TIME_PER_RECORD); + return new Tuple2<>(first.f0, second.f0); + } + } + + public static final class LongCancelTimeMatcher implements JoinFunction, Tuple2, Tuple2> { + private static final long serialVersionUID = 1L; + + private static final int WAIT_TIME_PER_RECORD = 5 * 1000; // 5 sec. + + @Override + public Tuple2 join(Tuple2 first, Tuple2 second) throws Exception { + final long start = System.currentTimeMillis(); + long remaining = WAIT_TIME_PER_RECORD; + do { + try { + Thread.sleep(remaining); + } catch (InterruptedException iex) {} + } while ((remaining = WAIT_TIME_PER_RECORD - System.currentTimeMillis() + start) > 0); + return new Tuple2<>(first.f0, second.f0); + } + } + + public static final class StuckInOpenMatcher extends RichJoinFunction, Tuple2, Tuple2> { + private static final long serialVersionUID = 1L; + + @Override + public void open(Configuration parameters) throws Exception { + synchronized (this) { + wait(); + } + } + + @Override + public Tuple2 join(Tuple2 first, Tuple2 second) throws Exception { + return new Tuple2<>(first.f0, second.f0); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java index 36db34d2b3866..55b19111e68fc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java @@ -18,20 +18,14 @@ package org.apache.flink.test.cancelling; -//import org.junit.Test; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.operators.GenericDataSink; -import org.apache.flink.api.java.record.operators.GenericDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.util.DiscardingOutputFormat; -import org.apache.flink.test.recordJobs.util.InfiniteIntegerInputFormat; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; +import org.apache.flink.test.util.InfiniteIntegerInputFormat; +import org.junit.Test; -@SuppressWarnings("deprecation") public class MapCancelingITCase extends CancellingTestBase { private static final int parallelism = 4; @@ -39,119 +33,85 @@ public MapCancelingITCase() { setTaskManagerNumSlots(parallelism); } -// @Test + @Test public void testMapCancelling() throws Exception { - GenericDataSource source = new GenericDataSource( - new InfiniteIntegerInputFormat(), "Source"); - MapOperator mapper = MapOperator.builder(IdentityMapper.class) - .input(source) - .name("Identity Mapper") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), mapper, "Sink"); - - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 5 * 1000, 10 * 1000); + executeTask(new IdentityMapper()); } -// @Test + @Test public void testSlowMapCancelling() throws Exception { - GenericDataSource source = new GenericDataSource( - new InfiniteIntegerInputFormat(), "Source"); - MapOperator mapper = MapOperator.builder(DelayingIdentityMapper.class) - .input(source) - .name("Delay Mapper") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), mapper, "Sink"); - - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 5 * 1000, 10 * 1000); + executeTask(new DelayingIdentityMapper()); } -// @Test + @Test public void testMapWithLongCancellingResponse() throws Exception { - GenericDataSource source = new GenericDataSource( - new InfiniteIntegerInputFormat(), "Source"); - MapOperator mapper = MapOperator.builder(LongCancelTimeIdentityMapper.class) - .input(source) - .name("Long Cancelling Time Mapper") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), mapper, "Sink"); - - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 10 * 1000, 10 * 1000); + executeTask(new LongCancelTimeIdentityMapper()); } -// @Test + @Test public void testMapPriorToFirstRecordReading() throws Exception { - GenericDataSource source = new GenericDataSource( - new InfiniteIntegerInputFormat(), "Source"); - MapOperator mapper = MapOperator.builder(StuckInOpenIdentityMapper.class) - .input(source) - .name("Stuck-In-Open Mapper") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), mapper, "Sink"); - - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 10 * 1000, 10 * 1000); + executeTask(new StuckInOpenIdentityMapper()); + } + + public void executeTask(MapFunction mapper) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + env + .createInput(new InfiniteIntegerInputFormat(false)) + .map(mapper) + .output(new DiscardingOutputFormat()); + + env.setParallelism(parallelism); + + runAndCancelJob(env.createProgramPlan(), 5 * 1000, 10 * 1000); } // -------------------------------------------------------------------------------------------- - public static final class IdentityMapper extends MapFunction { + public static final class IdentityMapper implements MapFunction { private static final long serialVersionUID = 1L; @Override - public void map(Record record, Collector out) throws Exception { - out.collect(record); + public IN map(IN value) throws Exception { + return value; } } - public static final class DelayingIdentityMapper extends MapFunction { + public static final class DelayingIdentityMapper implements MapFunction { private static final long serialVersionUID = 1L; - - private static final int WAIT_TIME_PER_RECORD = 10 * 1000; // 10 sec. + + private static final int WAIT_TIME_PER_VALUE = 10 * 1000; // 10 sec. @Override - public void map(Record record, Collector out) throws Exception { - Thread.sleep(WAIT_TIME_PER_RECORD); - out.collect(record); + public IN map(IN value) throws Exception { + Thread.sleep(WAIT_TIME_PER_VALUE); + return value; } } - public static final class LongCancelTimeIdentityMapper extends MapFunction { + public static final class LongCancelTimeIdentityMapper implements MapFunction { private static final long serialVersionUID = 1L; - - private static final int WAIT_TIME_PER_RECORD = 5 * 1000; // 5 sec. + + private static final int WAIT_TIME_PER_VALUE = 5 * 1000; // 5 sec. @Override - public void map(Record record, Collector out) throws Exception { + public IN map(IN value) throws Exception { final long start = System.currentTimeMillis(); - long remaining = WAIT_TIME_PER_RECORD; + long remaining = WAIT_TIME_PER_VALUE; do { try { Thread.sleep(remaining); - } catch (InterruptedException iex) {} - } while ((remaining = WAIT_TIME_PER_RECORD - System.currentTimeMillis() + start) > 0); - - out.collect(record); + } catch (InterruptedException iex) { + } + } while ((remaining = WAIT_TIME_PER_VALUE - System.currentTimeMillis() + start) > 0); + + return value; } } - public static final class StuckInOpenIdentityMapper extends MapFunction { + public static final class StuckInOpenIdentityMapper extends RichMapFunction { private static final long serialVersionUID = 1L; - + @Override public void open(Configuration parameters) throws Exception { synchronized (this) { @@ -160,8 +120,8 @@ public void open(Configuration parameters) throws Exception { } @Override - public void map(Record record, Collector out) throws Exception { - out.collect(record); + public IN map(IN value) throws Exception { + return value; } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java deleted file mode 100644 index 8197e27356426..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java +++ /dev/null @@ -1,277 +0,0 @@ -/* - * 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.cancelling; - -//import org.junit.Test; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.operators.GenericDataSink; -import org.apache.flink.api.java.record.operators.GenericDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.util.DiscardingOutputFormat; -import org.apache.flink.test.recordJobs.util.InfiniteIntegerInputFormat; -import org.apache.flink.test.recordJobs.util.InfiniteIntegerInputFormatWithDelay; -import org.apache.flink.test.recordJobs.util.UniformIntInput; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class MatchJoinCancelingITCase extends CancellingTestBase { - private static final int parallelism = 4; - - public MatchJoinCancelingITCase(){ - setTaskManagerNumSlots(parallelism); - } - - // --------------- Test Sort Matches that are canceled while still reading / sorting ----------------- -// @Test - public void testCancelSortMatchWhileReadingSlowInputs() throws Exception { - GenericDataSource source1 = - new GenericDataSource(new InfiniteIntegerInputFormatWithDelay(), "Source 1"); - - GenericDataSource source2 = - new GenericDataSource(new InfiniteIntegerInputFormatWithDelay(), "Source 2"); - - JoinOperator matcher = JoinOperator.builder(SimpleMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 3000, 10*1000); - } - -// @Test - public void testCancelSortMatchWhileReadingFastInputs() throws Exception { - GenericDataSource source1 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 1"); - - GenericDataSource source2 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 2"); - - JoinOperator matcher = JoinOperator.builder(SimpleMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 5000, 10*1000); - } - -// @Test - public void testCancelSortMatchPriorToFirstRecordReading() throws Exception { - GenericDataSource source1 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 1"); - - GenericDataSource source2 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 2"); - - JoinOperator matcher = JoinOperator.builder(StuckInOpenMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Stuc-In-Open Match") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 5000); - - runAndCancelJob(p, 10 * 1000, 10 * 1000); - } - -// @Test - public void testCancelSortMatchWhileDoingHeavySorting() throws Exception { - GenericDataSource source1 = - new GenericDataSource(new UniformIntInput(), "Source 1"); - source1.setParameter(UniformIntInput.NUM_KEYS_KEY, 50000); - source1.setParameter(UniformIntInput.NUM_VALUES_KEY, 100); - - GenericDataSource source2 = - new GenericDataSource(new UniformIntInput(), "Source 2"); - source2.setParameter(UniformIntInput.NUM_KEYS_KEY, 50000); - source2.setParameter(UniformIntInput.NUM_VALUES_KEY, 100); - - JoinOperator matcher = JoinOperator.builder(SimpleMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Long Cancelling Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 30 * 1000, 30 * 1000); - } - - - // --------------- Test Sort Matches that are canceled while in the Matching Phase ----------------- - -// @Test - public void testCancelSortMatchWhileJoining() throws Exception { - GenericDataSource source1 = - new GenericDataSource(new UniformIntInput(), "Source 1"); - source1.setParameter(UniformIntInput.NUM_KEYS_KEY, 500); - source1.setParameter(UniformIntInput.NUM_VALUES_KEY, 3); - - GenericDataSource source2 = - new GenericDataSource(new UniformIntInput(), "Source 2"); - source2.setParameter(UniformIntInput.NUM_KEYS_KEY, 500); - source2.setParameter(UniformIntInput.NUM_VALUES_KEY, 3); - - JoinOperator matcher = JoinOperator.builder(DelayingMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Long Cancelling Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 10 * 1000, 20 * 1000); - } - -// @Test - public void testCancelSortMatchWithLongCancellingResponse() throws Exception { - - GenericDataSource source1 = - new GenericDataSource(new UniformIntInput(), "Source 1"); - source1.setParameter(UniformIntInput.NUM_KEYS_KEY, 500); - source1.setParameter(UniformIntInput.NUM_VALUES_KEY, 3); - - GenericDataSource source2 = - new GenericDataSource(new UniformIntInput(), "Source 2"); - source2.setParameter(UniformIntInput.NUM_KEYS_KEY, 500); - source2.setParameter(UniformIntInput.NUM_VALUES_KEY, 3); - - JoinOperator matcher = JoinOperator.builder(LongCancelTimeMatcher.class, IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Long Cancelling Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - - runAndCancelJob(p, 10 * 1000, 10 * 1000); - } - - // -------------------------------------- Test System corner cases --------------------------------- - -// @Test - public void testCancelSortMatchWithHighparallelism() throws Exception { - - GenericDataSource source1 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 1"); - - GenericDataSource source2 = - new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 2"); - - JoinOperator matcher = JoinOperator.builder(new SimpleMatcher(), IntValue.class, 0, 0) - .input1(source1) - .input2(source2) - .name("Sort Join") - .build(); - GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); - - Plan p = new Plan(sink); - p.setDefaultParallelism(64); - - runAndCancelJob(p, 3000, 20*1000); - } - - // -------------------------------------------------------------------------------------------- - - public static final class SimpleMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - @Override - public void join(Record value1, Record value2, Collector out) throws Exception { - value1.setField(1, value2.getField(0, IntValue.class)); - out.collect(value1); - } - } - - public static final class DelayingMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - private static final int WAIT_TIME_PER_RECORD = 10 * 1000; // 10 sec. - - @Override - public void join(Record value1, Record value2, Collector out) throws Exception { - Thread.sleep(WAIT_TIME_PER_RECORD); - value1.setField(1, value2.getField(0, IntValue.class)); - out.collect(value1); - } - } - - public static final class LongCancelTimeMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - private static final int WAIT_TIME_PER_RECORD = 5 * 1000; // 5 sec. - - @Override - public void join(Record value1, Record value2, Collector out) throws Exception { - value1.setField(1, value2.getField(0, IntValue.class)); - - final long start = System.currentTimeMillis(); - long remaining = WAIT_TIME_PER_RECORD; - do { - try { - Thread.sleep(remaining); - } catch (InterruptedException iex) {} - } while ((remaining = WAIT_TIME_PER_RECORD - System.currentTimeMillis() + start) > 0); - - out.collect(value1); - } - } - - public static final class StuckInOpenMatcher extends JoinFunction { - private static final long serialVersionUID = 1L; - - @Override - public void open(Configuration parameters) throws Exception { - synchronized (this) { - wait(); - } - } - - @Override - public void join(Record value1, Record value2, Collector out) throws Exception { - value1.setField(1, value2.getField(0, IntValue.class)); - out.collect(value1); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java index 3e2657c047fc5..5b05b67be75ec 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java @@ -23,9 +23,12 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.LocalExecutor; -import org.apache.flink.test.recordJobs.wordcount.WordCount; import org.apache.flink.test.testdata.WordCountData; +import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; @@ -46,17 +49,13 @@ public void testLocalExecutorWithWordCount() { FileWriter fw = new FileWriter(inFile); fw.write(WordCountData.TEXT); fw.close(); - - // run WordCount - WordCount wc = new WordCount(); LocalExecutor executor = new LocalExecutor(); executor.setDefaultOverwriteFiles(true); executor.setTaskManagerNumSlots(parallelism); executor.setPrintStatusDuringExecution(false); executor.start(); - Plan wcPlan = wc.getPlan(Integer.valueOf(parallelism).toString(), - inFile.toURI().toString(), outFile.toURI().toString()); + Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism); wcPlan.setExecutionConfig(new ExecutionConfig()); executor.executePlan(wcPlan); executor.stop(); @@ -65,4 +64,27 @@ public void testLocalExecutorWithWordCount() { Assert.fail(e.getMessage()); } } + + private Plan getWordCountPlan(File inFile, File outFile, int parallelism) { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(parallelism); + env.readTextFile(inFile.getAbsolutePath()) + .flatMap(new Tokenizer()) + .groupBy(0) + .sum(1) + .writeAsCsv(outFile.getAbsolutePath()); + return env.createProgramPlan(); + } + + public static final class Tokenizer implements FlatMapFunction> { + @Override + public void flatMap(String value, Collector> out) { + String[] tokens = value.toLowerCase().split("\\W+"); + for (String token : tokens) { + if (token.length() > 0) { + out.collect(new Tuple2<>(token, 1)); + } + } + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java b/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java index f625c57ccb502..4fb2d952778dc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.test.distributedCache; import java.io.BufferedReader; @@ -24,134 +23,69 @@ import java.io.FileNotFoundException; import java.io.FileReader; import java.util.HashSet; +import java.util.List; import java.util.Set; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; /** - * Test the distributed cache via using the cache file to do a selection on the input + * Tests the distributed cache by comparing a text file with a distributed copy. */ -@SuppressWarnings("deprecation") -public class DistributedCacheTest extends RecordAPITestBase { - - public static final String cacheData = "machen\n" + "zeit\n" + "heerscharen\n" + "keiner\n" + "meine\n" - + "fuehr\n" + "triumph\n" + "kommst\n" + "frei\n" + "schaffen\n" + "gesinde\n" - + "langbeinigen\n" + "schalk\n" + "besser\n" + "solang\n" + "meer\n" + "fragst\n" - + "gabriel\n" + "selbst\n" + "bin\n" + "sich\n" + "du\n" + "sogar\n" + "geht\n" - + "immer\n" + "mensch\n" + "befestigt\n" + "lebt\n" + "mag\n" + "engeln\n" + "breiten\n" - + "blitzendes\n" + "tags\n" + "sie\n" + "plagen\n" + "allzu\n" + "meisten\n" + "o\n" - + "pfade\n" + "kennst\n" + "nichts\n" + "gedanken\n" + "befriedigt\n" + "mich\n" + "s\n" - + "es\n" + "verneinen\n" + "er\n" + "gleich\n" + "baeumchen\n" + "donnergang\n"; +public class DistributedCacheTest extends JavaProgramTestBase { - public static final String selectedCounts = "machen 1\n" + "zeit 2\n" + "heerscharen 1\n" + "keiner 2\n" + "meine 3\n" - + "fuehr 1\n" + "triumph 1\n" + "kommst 1\n" + "frei 1\n" + "schaffen 1\n" + "gesinde 1\n" - + "langbeinigen 1\n" + "schalk 1\n" + "besser 1\n" + "solang 1\n" + "meer 4\n" + "fragst 1\n" - + "gabriel 1\n" + "selbst 2\n" + "bin 1\n" + "sich 7\n" + "du 11\n" + "sogar 1\n" + "geht 1\n" - + "immer 4\n" + "mensch 2\n" + "befestigt 1\n" + "lebt 2\n" + "mag 3\n" + "engeln 2\n" + "breiten 1\n" - + "blitzendes 1\n" + "tags 1\n" + "sie 2\n" + "plagen 2\n" + "allzu 1\n" + "meisten 1\n" + "o 1\n" - + "pfade 1\n" + "kennst 1\n" + "nichts 3\n" + "gedanken 1\n" + "befriedigt 1\n" + "mich 6\n" + "s 3\n" - + "es 8\n" + "verneinen 1\n" + "er 13\n" + "gleich 1\n" + "baeumchen 1\n" + "donnergang 1\n"; + public static final String data + = "machen\n" + + "zeit\n" + + "heerscharen\n" + + "keiner\n" + + "meine\n"; protected String textPath; - protected String cachePath; - protected String resultPath; - public static class TokenizeLine extends MapFunction { + @Override + protected void preSubmit() throws Exception { + textPath = createTempFile("count.txt", data); + } + + @Override + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.registerCachedFile(textPath, "cache_test"); + + List> result = env + .readTextFile(textPath) + .flatMap(new WordChecker()) + .collect(); + + compareResultAsTuples(result, data); + } + + public static class WordChecker extends RichFlatMapFunction> { private static final long serialVersionUID = 1L; - private Set stringList = new HashSet(); - @Override - public void open(org.apache.flink.configuration.Configuration conf) { + private final Set wordList = new HashSet<>(); + @Override + public void open(Configuration conf) throws FileNotFoundException, IOException { File file = getRuntimeContext().getDistributedCache().getFile("cache_test"); - BufferedReader reader = null; - try { - reader = new BufferedReader(new FileReader(file)); - String tempString = null; - while ((tempString = reader.readLine()) != null) { - stringList.add(tempString); - } - reader.close(); - } catch (FileNotFoundException e) { - e.printStackTrace(); - } catch (IOException e) { - e.printStackTrace(); + BufferedReader reader = new BufferedReader(new FileReader(file)); + String tempString; + while ((tempString = reader.readLine()) != null) { + wordList.add(tempString); } + reader.close(); } @Override - public void map(Record record, Collector collector) { - String line = record.getField(0, StringValue.class).getValue(); - String [] element = line.split(" "); - String word = element[0]; - int count = Integer.parseInt(element[1]); - if (stringList.contains(word)) { - collector.collect(new Record(new StringValue(word), new IntValue(count))); + public void flatMap(String word, Collector> out) throws Exception { + if (wordList.contains(word)) { + out.collect(new Tuple1<>(word)); } } } - - public Plan getPlan(int numSubTasks, String dataInput, String output) { - - - // input is {word, count} pair - FileDataSource source = new FileDataSource(new TextInputFormat(), dataInput, "Input Lines"); - - //do a selection using cached file - MapOperator mapper = MapOperator.builder(new TokenizeLine()) - .input(source) - .name("Tokenize Lines") - .build(); - - FileDataSink out = new FileDataSink(new CsvOutputFormat(), output, mapper, "Selection"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "Distributed Cache"); - plan.setDefaultParallelism(numSubTasks); - plan.setExecutionConfig(new ExecutionConfig()); - return plan; - } - - - @Override - protected void preSubmit() throws Exception { - textPath = createTempFile("count.txt", WordCountData.COUNTS); - cachePath = createTempFile("cache.txt", cacheData); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - Plan plan = getPlan(1 , textPath, resultPath); - try { - plan.registerCachedFile("cache_test", new DistributedCacheEntry(cachePath, false)); - } catch (IOException ex) { - throw new RuntimeException(ex); - } - return plan; - } - - @Override - protected void postSubmit() throws Exception { - // Test results - compareResultsByLinesInMemory(selectedCounts, resultPath); - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java index fe98e186873ba..fef6c9f4d4036 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java @@ -18,168 +18,71 @@ package org.apache.flink.test.failingPrograms; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.FailingTestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; +import java.util.List; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.junit.Assert; /** - * Tests whether the system recovers from a runtime exception from the user code. + * + * Tests that both jobs, the failing and the working one, are handled correctly. The first (failing) job must be + * canceled and the client must report the failure. The second (working) job must finish successfully and compute the + * correct result. + * */ -public class TaskFailureITCase extends FailingTestBase { - - private static final int parallelism = 4; - - // input for map tasks - private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" + - "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n" + - "1 1\n2 2\n2 2\n3 0\n4 4\n5 9\n7 7\n8 8\n" + - "1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - // expected result of working map job - private static final String MAP_RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n" + - "3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - private String inputPath; - private String resultPath; - - public TaskFailureITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - inputPath = createTempFile("input", MAP_IN); - resultPath = getTempDirPath("result"); - } - - - @Override - protected JobGraph getFailingJobGraph() throws Exception { - - // init data source - FileDataSource input = new FileDataSource(new ContractITCaseInputFormat(), inputPath); - - // init failing map task - MapOperator testMapper = MapOperator.builder(FailingMapper.class).build(); - - // init data sink - FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultPath); - - // compose failing program - output.setInput(testMapper); - testMapper.setInput(input); - - // generate plan - Plan plan = new Plan(output); - plan.setExecutionConfig(new ExecutionConfig()); - plan.setDefaultParallelism(parallelism); - - // optimize and compile plan - Optimizer pc = new Optimizer(new DataStatistics(), this.config); - OptimizedPlan op = pc.compile(plan); - - // return job graph of failing job - JobGraphGenerator jgg = new JobGraphGenerator(); - return jgg.compileJobGraph(op); - } - +public class TaskFailureITCase extends JavaProgramTestBase { @Override - protected JobGraph getJobGraph() throws Exception { - - // init data source - FileDataSource input = new FileDataSource(new ContractITCaseInputFormat(), inputPath); - - // init (working) map task - MapOperator testMapper = MapOperator.builder(TestMapper.class).build(); - - // init data sink - FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultPath); - - // compose working program - output.setInput(testMapper); - testMapper.setInput(input); - - // generate plan - Plan plan = new Plan(output); - plan.setExecutionConfig(new ExecutionConfig()); - plan.setDefaultParallelism(4); - - // optimize and compile plan - Optimizer pc = new Optimizer(new DataStatistics(), this.config); - OptimizedPlan op = pc.compile(plan); - - // return job graph of working job - JobGraphGenerator jgg = new JobGraphGenerator(); - return jgg.compileJobGraph(op); - } - - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(MAP_RESULT, resultPath); + protected void testProgram() throws Exception { + //test failing version + try { + executeTask(new FailingTestMapper()); + } catch (RuntimeException e) { //expected for collection execution + if (!isCollectionExecution()) { + Assert.fail(); + } + } catch (JobExecutionException e) { //expected for cluster execution + if (isCollectionExecution()) { + Assert.fail(); + } + } + //test correct version + executeTask(new TestMapper()); } - @Override - protected int getTimeout() { - // time out for this job is 30 secs - return 30; + private void executeTask(MapFunction mapper) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + List result = env.generateSequence(1, 9) + .map(mapper) + .collect(); + MultipleProgramsTestBase.compareResultAsText(result, "1\n2\n3\n4\n5\n6\n7\n8\n9"); } /** - * working program + * working map function */ - public static class TestMapper extends MapFunction { - + public static class TestMapper implements MapFunction { private static final long serialVersionUID = 1L; - - private final StringValue string = new StringValue(); - private final IntValue integer = new IntValue(); @Override - public void map(Record record, Collector out) throws Exception { - - final StringValue keyString = record.getField(0, this.string); - final int key = Integer.parseInt(keyString.toString()); - - final StringValue valueString = record.getField(1, this.string); - final int value = Integer.parseInt(valueString.toString()); - - if (key + value < 10) { - record.setField(0, valueString); - this.integer.setValue(key + 10); - record.setField(1, this.integer); - out.collect(record); - } + public Long map(Long value) throws Exception { + return value; } } /** - * Map Failing program + * failing map function */ - public static class FailingMapper extends MapFunction { - + public static class FailingTestMapper implements MapFunction { private static final long serialVersionUID = 1L; @Override - public void map(Record record, Collector out) throws Exception { + public Long map(Long value) throws Exception { throw new RuntimeException("This is an expected Test Exception"); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java index 9023a1f4fb347..2a0b004f89209 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java @@ -19,32 +19,25 @@ package org.apache.flink.test.iterative; import java.io.BufferedReader; -import java.io.Serializable; import java.util.Iterator; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.CoGroupFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.CoGroupOperator; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; +import org.apache.flink.api.java.operators.CoGroupOperator; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.operators.JoinOperator; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class CoGroupConnectedComponentsITCase extends RecordAPITestBase { +public class CoGroupConnectedComponentsITCase extends JavaProgramTestBase { private static final long SEED = 0xBADC0FFEEBEEFL; @@ -53,13 +46,11 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase { private static final int NUM_EDGES = 10000; + private static final int MAX_ITERATIONS = 100; + protected String verticesPath; protected String edgesPath; protected String resultPath; - - public CoGroupConnectedComponentsITCase(){ - setTaskManagerNumSlots(parallelism); - } @Override @@ -68,11 +59,6 @@ protected void preSubmit() throws Exception { edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED)); resultPath = getTempFilePath("results"); } - - @Override - protected Plan getTestJob() { - return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100); - } @Override protected void postSubmit() throws Exception { @@ -85,83 +71,70 @@ protected void postSubmit() throws Exception { // The test program // -------------------------------------------------------------------------------------------- + @Override + protected void testProgram() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> initialVertices = env.readCsvFile(verticesPath).fieldDelimiter(" ").types(Long.class).name("Vertices"); + + DataSet> edges = env.readCsvFile(edgesPath).fieldDelimiter(" ").types(Long.class, Long.class).name("Edges"); - @ConstantFieldsFirst(0) - @ConstantFieldsSecond(0) - public static final class MinIdAndUpdate extends CoGroupFunction implements Serializable { + DataSet> verticesWithId = initialVertices.map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Tuple1 value) throws Exception { + return new Tuple2<>(value.f0, value.f0); + } + }).name("Assign Vertex Ids"); + + DeltaIteration, Tuple2> iteration = verticesWithId.iterateDelta(verticesWithId, MAX_ITERATIONS, 0); + + JoinOperator, Tuple2, Tuple2> joinWithNeighbors = iteration.getWorkset() + .join(edges).where(0).equalTo(0) + .with(new JoinFunction, Tuple2, Tuple2>() { + @Override + public Tuple2 join(Tuple2 first, Tuple2 second) throws Exception { + return new Tuple2<>(second.f1, first.f1); + } + }) + .name("Join Candidate Id With Neighbor"); + + CoGroupOperator, Tuple2, Tuple2> minAndUpdate = joinWithNeighbors + .coGroup(iteration.getSolutionSet()).where(0).equalTo(0) + .with(new MinIdAndUpdate()) + .name("min Id and Update"); + + iteration.closeWith(minAndUpdate, minAndUpdate).writeAsCsv(resultPath, "\n", " ").name("Result"); + + env.execute("Workset Connected Components"); + } + + @ForwardedFieldsFirst("f1->f1") + @ForwardedFieldsSecond("f0->f0") + public static final class MinIdAndUpdate implements CoGroupFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; - private final LongValue newComponentId = new LongValue(); - @Override - public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { + public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { + Iterator> current = second.iterator(); if (!current.hasNext()) { throw new Exception("Error: Id not encountered before."); } - Record old = current.next(); - long oldId = old.getField(1, LongValue.class).getValue(); + Tuple2 old = current.next(); + long oldId = old.f1; long minimumComponentID = Long.MAX_VALUE; - while (candidates.hasNext()) { - Record candidate = candidates.next(); - long candidateComponentID = candidate.getField(1, LongValue.class).getValue(); + for (Tuple2 candidate : first) { + long candidateComponentID = candidate.f1; if (candidateComponentID < minimumComponentID) { minimumComponentID = candidateComponentID; } } if (minimumComponentID < oldId) { - newComponentId.setValue(minimumComponentID); - old.setField(1, newComponentId); - out.collect(old); + out.collect(new Tuple2<>(old.f0, minimumComponentID)); } } } - - @SuppressWarnings("unchecked") - public static Plan getPlan(int numSubTasks, String verticesInput, String edgeInput, String output, int maxIterations) { - - // data source for initial vertices - FileDataSource initialVertices = new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices"); - - MapOperator verticesWithId = MapOperator.builder(DuplicateLongMap.class).input(initialVertices).name("Assign Vertex Ids").build(); - - DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration"); - iteration.setInitialSolutionSet(verticesWithId); - iteration.setInitialWorkset(verticesWithId); - iteration.setMaximumNumberOfIterations(maxIterations); - - // create DataSourceContract for the edges - FileDataSource edges = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges"); - - // create CrossOperator for distance computation - JoinOperator joinWithNeighbors = JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(edges) - .name("Join Candidate Id With Neighbor") - .build(); - - CoGroupOperator minAndUpdate = CoGroupOperator.builder(new MinIdAndUpdate(), LongValue.class, 0, 0) - .input1(joinWithNeighbors) - .input2(iteration.getSolutionSet()) - .name("Min Id and Update") - .build(); - - iteration.setNextWorkset(minAndUpdate); - iteration.setSolutionSetDelta(minAndUpdate); - - // create DataSinkContract for writing the new cluster positions - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, iteration, "Result"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(LongValue.class, 0) - .field(LongValue.class, 1); - - // return the PACT plan - Plan plan = new Plan(result, "Workset Connected Components"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java index df3c00d32c3a7..61e08d4d22796 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java @@ -21,13 +21,19 @@ import java.io.BufferedReader; -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.RecordAPITestBase; +import org.apache.flink.test.util.JavaProgramTestBase; -public class ConnectedComponentsITCase extends RecordAPITestBase { +public class ConnectedComponentsITCase extends JavaProgramTestBase { private static final long SEED = 0xBADC0FFEEBEEFL; @@ -40,22 +46,44 @@ public class ConnectedComponentsITCase extends RecordAPITestBase { protected String edgesPath; protected String resultPath; - public ConnectedComponentsITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override protected void preSubmit() throws Exception { verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED)); resultPath = getTempFilePath("results"); } - + @Override - protected Plan getTestJob() { - WorksetConnectedComponents cc = new WorksetConnectedComponents(); - return cc.getPlan(Integer.valueOf(parallelism).toString(), verticesPath, edgesPath, resultPath, "100"); + protected void testProgram() throws Exception { + // set up execution environment + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // read vertex and edge data + DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); + + DataSet> edges = env.readCsvFile(edgesPath).fieldDelimiter(" ").types(Long.class, Long.class) + .flatMap(new ConnectedComponents.UndirectEdge()); + + // assign the initial components (equal to the vertex id) + DataSet> verticesWithInitialId = vertices.map(new DuplicateValue()); + + // open a delta iteration + DeltaIteration, Tuple2> iteration = + verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); + + // apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller + DataSet> changes = iteration.getWorkset().join(edges).where(0).equalTo(0).with(new ConnectedComponents.NeighborWithComponentIDJoin()) + .groupBy(0).aggregate(Aggregations.MIN, 1) + .join(iteration.getSolutionSet()).where(0).equalTo(0) + .with(new ConnectedComponents.ComponentIdFilter()); + + // close the delta iteration (delta and new workset are identical) + DataSet> result = iteration.closeWith(changes, changes); + + result.writeAsCsv(resultPath, "\n", " "); + + // execute program + env.execute("Connected Components Example"); } @Override @@ -64,4 +92,12 @@ protected void postSubmit() throws Exception { ConnectedComponentsData.checkOddEvenResult(reader); } } + + public static final class DuplicateValue implements MapFunction, Tuple2> { + + @Override + public Tuple2 map(Tuple1 vertex) { + return new Tuple2<>(vertex.f0, vertex.f0); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java index d5d150d4d61d3..bc4885f1dedf5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java @@ -20,36 +20,27 @@ package org.apache.flink.test.iterative; import java.io.BufferedReader; -import java.io.Serializable; import java.util.Collection; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -@SuppressWarnings("deprecation") @RunWith(Parameterized.class) -public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBase { +public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTestBase { private static final long SEED = 0xBADC0FFEEBEEFL; @@ -65,20 +56,64 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa public ConnectedComponentsWithDeferredUpdateITCase(Configuration config) { super(config); - setTaskManagerNumSlots(parallelism); } - + @Override protected void preSubmit() throws Exception { verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED)); resultPath = getTempFilePath("results"); } - + @Override - protected Plan getTestJob() { + protected void testProgram() throws Exception { boolean extraMapper = config.getBoolean("ExtraMapper", false); - return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100, extraMapper); + + // set up execution environment + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // read vertex and edge data + DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); + + DataSet> edges = env.readCsvFile(edgesPath).fieldDelimiter(" ").types(Long.class, Long.class) + .flatMap(new ConnectedComponents.UndirectEdge()); + + // assign the initial components (equal to the vertex id) + DataSet> verticesWithInitialId = vertices.map(new ConnectedComponentsITCase.DuplicateValue()); + + // open a delta iteration + DeltaIteration, Tuple2> iteration = + verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); + + // apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller + DataSet> changes = iteration.getWorkset() + .join(edges).where(0).equalTo(0).with(new ConnectedComponents.NeighborWithComponentIDJoin()) + .groupBy(0).aggregate(Aggregations.MIN, 1) + .join(iteration.getSolutionSet()).where(0).equalTo(0) + .with(new UpdateComponentIdMatchNonPreserving()); + + DataSet> delta; + if(extraMapper) { + delta = changes.map( + // ID Mapper + new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Tuple2 v) throws Exception { + return v; + } + }); + } + else { + delta = changes; + } + + // close the delta iteration (delta and new workset are identical) + DataSet> result = iteration.closeWith(delta, changes); + + result.writeAsCsv(resultPath, "\n", " "); + + // execute program + env.execute("Connected Components Example"); } @Override @@ -98,84 +133,21 @@ public static Collection getConfigurations() { return toParameterList(config1, config2); } - - @SuppressWarnings("unchecked") - public static Plan getPlan(int numSubTasks, String verticesInput, String edgeInput, String output, int maxIterations, boolean extraMap) { - - // data source for initial vertices - FileDataSource initialVertices = new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices"); - - MapOperator verticesWithId = MapOperator.builder(DuplicateLongMap.class).input(initialVertices).name("Assign Vertex Ids").build(); - - // the loop takes the vertices as the solution set and changed vertices as the workset - // initially, all vertices are changed - DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration"); - iteration.setInitialSolutionSet(verticesWithId); - iteration.setInitialWorkset(verticesWithId); - iteration.setMaximumNumberOfIterations(maxIterations); - - // data source for the edges - FileDataSource edges = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges"); - - // join workset (changed vertices) with the edges to propagate changes to neighbors - JoinOperator joinWithNeighbors = JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(edges) - .name("Join Candidate Id With Neighbor") - .build(); - - // find for each neighbor the smallest of all candidates - ReduceOperator minCandidateId = ReduceOperator.builder(new MinimumComponentIDReduce(), LongValue.class, 0) - .input(joinWithNeighbors) - .name("Find Minimum Candidate Id") - .build(); - - // join candidates with the solution set and update if the candidate component-id is smaller - JoinOperator updateComponentId = JoinOperator.builder(new UpdateComponentIdMatchNonPreserving(), LongValue.class, 0, 0) - .input1(minCandidateId) - .input2(iteration.getSolutionSet()) - .name("Update Component Id") - .build(); - - if (extraMap) { - MapOperator mapper = MapOperator.builder(IdentityMap.class).input(updateComponentId).name("idmap").build(); - iteration.setSolutionSetDelta(mapper); - } else { - iteration.setSolutionSetDelta(updateComponentId); - } - - iteration.setNextWorkset(updateComponentId); - // sink is the iteration result - FileDataSink result = new FileDataSink(new CsvOutputFormat("\n", " ", LongValue.class, LongValue.class), output, iteration, "Result"); - - // return the PACT plan - Plan plan = new Plan(result, "Workset Connected Components"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - public static final class UpdateComponentIdMatchNonPreserving extends JoinFunction implements Serializable { + public static final class UpdateComponentIdMatchNonPreserving + implements FlatJoinFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; @Override - public void join(Record newVertexWithComponent, Record currentVertexWithComponent, Collector out){ - - long candidateComponentID = newVertexWithComponent.getField(1, LongValue.class).getValue(); - long currentComponentID = currentVertexWithComponent.getField(1, LongValue.class).getValue(); - - if (candidateComponentID < currentComponentID) { - out.collect(newVertexWithComponent); + public void join( + Tuple2 candidate, + Tuple2 current, + Collector> out) throws Exception { + + if(candidate.f1 < current.f1) { + out.collect(candidate); } } } - - public static final class IdentityMap extends MapFunction { - private static final long serialVersionUID = 1L; - @Override - public void map(Record record, Collector out) throws Exception { - out.collect(record); - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java index f0a1dd7832c30..a8a28f1f95d16 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java @@ -76,7 +76,7 @@ protected void testProgram() throws Exception { .flatMap(new UndirectEdge()); // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = vertices.map(new DuplicateValue()); + DataSet> verticesWithInitialId = vertices.map(new ConnectedComponentsITCase.DuplicateValue()); // open a delta iteration DeltaIteration, Tuple2> iteration = @@ -98,11 +98,4 @@ protected void testProgram() throws Exception { env.execute("Connected Components Example"); } - public static final class DuplicateValue implements MapFunction, Tuple2> { - - @Override - public Tuple2 map(Tuple1 vertex) { - return new Tuple2(vertex.f0, vertex.f0); - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java index b97d0ad70f7e2..c2dd434f371d6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java @@ -20,33 +20,24 @@ package org.apache.flink.test.iterative; import java.io.BufferedReader; -import java.io.Serializable; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecondExcept; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.functions.FunctionAnnotation; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; /** * Tests a bug that prevented that the solution set can be on both sides of the match/cogroup function. */ -@SuppressWarnings("deprecation") -public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITestBase { +public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTestBase { private static final long SEED = 0xBADC0FFEEBEEFL; @@ -59,11 +50,6 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest protected String edgesPath; protected String resultPath; - public ConnectedComponentsWithSolutionSetFirstITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override protected void preSubmit() throws Exception { verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); @@ -72,90 +58,67 @@ protected void preSubmit() throws Exception { } @Override - protected Plan getTestJob() { - return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(parallelism, verticesPath, edgesPath, - resultPath, 100); + protected void testProgram() throws Exception { + // set up execution environment + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // read vertex and edge data + DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); + + DataSet> edges = env.readCsvFile(edgesPath).fieldDelimiter(" ").types(Long.class, Long.class) + .flatMap(new ConnectedComponents.UndirectEdge()); + + // assign the initial components (equal to the vertex id) + DataSet> verticesWithInitialId = vertices.map(new ConnectedComponentsITCase.DuplicateValue()); + + // open a delta iteration + DeltaIteration, Tuple2> iteration = + verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); + + // apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller + DataSet> minNeighbor = iteration.getWorkset() + .join(edges).where(0).equalTo(0).with(new ConnectedComponents.NeighborWithComponentIDJoin()) + .groupBy(0).aggregate(Aggregations.MIN, 1); + + DataSet> updatedIds = iteration.getSolutionSet() + .join(minNeighbor).where(0).equalTo(0).with(new UpdateComponentIdMatchMirrored()); + + // close the delta iteration (delta and new workset are identical) + DataSet> result = iteration.closeWith(updatedIds, updatedIds); + + result.writeAsCsv(resultPath, "\n", " "); + + // execute program + env.execute("Connected Components Example"); } + @Override protected void postSubmit() throws Exception { for (BufferedReader reader : getResultReader(resultPath)) { ConnectedComponentsData.checkOddEvenResult(reader); } } - + // -------------------------------------------------------------------------------------------- // Classes and methods for the test program // -------------------------------------------------------------------------------------------- - - @ConstantFieldsSecondExcept({}) - public static final class UpdateComponentIdMatchMirrored extends JoinFunction implements Serializable { - + + @FunctionAnnotation.ForwardedFieldsSecond("*") + public static final class UpdateComponentIdMatchMirrored + implements FlatJoinFunction, Tuple2, Tuple2> { private static final long serialVersionUID = 1L; @Override - public void join(Record currentVertexWithComponent, Record newVertexWithComponent, Collector out){ - - long candidateComponentID = newVertexWithComponent.getField(1, LongValue.class).getValue(); - long currentComponentID = currentVertexWithComponent.getField(1, LongValue.class).getValue(); - - if (candidateComponentID < currentComponentID) { - out.collect(newVertexWithComponent); + public void join( + Tuple2 current, + Tuple2 candidate, + Collector> out) throws Exception { + + if(candidate.f1 < current.f1) { + out.collect(candidate); } + } } - - @SuppressWarnings("unchecked") - private static Plan getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput( - int numSubTasks, String verticesInput, String edgeInput, String output, int maxIterations) - { - // data source for initial vertices - FileDataSource initialVertices = new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices"); - - MapOperator verticesWithId = MapOperator.builder(DuplicateLongMap.class).input(initialVertices).name("Assign Vertex Ids").build(); - - DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration"); - iteration.setInitialSolutionSet(verticesWithId); - iteration.setInitialWorkset(verticesWithId); - iteration.setMaximumNumberOfIterations(maxIterations); - - // create DataSourceContract for the edges - FileDataSource edges = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges"); - - // create CrossOperator for distance computation - JoinOperator joinWithNeighbors = JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(edges) - .name("Join Candidate Id With Neighbor") - .build(); - - // create ReduceOperator for finding the nearest cluster centers - ReduceOperator minCandidateId = ReduceOperator.builder(new MinimumComponentIDReduce(), LongValue.class, 0) - .input(joinWithNeighbors) - .name("Find Minimum Candidate Id") - .build(); - - // create CrossOperator for distance computation - JoinOperator updateComponentId = JoinOperator.builder(new UpdateComponentIdMatchMirrored(), LongValue.class, 0, 0) - .input1(iteration.getSolutionSet()) - .input2(minCandidateId) - .name("Update Component Id") - .build(); - - iteration.setNextWorkset(updateComponentId); - iteration.setSolutionSetDelta(updateComponentId); - - // create DataSinkContract for writing the new cluster positions - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, iteration, "Result"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(LongValue.class, 0) - .field(LongValue.class, 1); - - // return the PACT plan - Plan plan = new Plan(result, "Workset Connected Components"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java deleted file mode 100644 index bf459c613516a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.iterative; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents; -import org.apache.flink.test.util.RecordAPITestBase; - - -public class DeltaPageRankITCase extends RecordAPITestBase { - - protected String verticesPath; - protected String edgesPath; - protected String deltasPath; - - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = createTempFile("vertices.txt", INITIAL_VERTICES_WITH_RANK); - deltasPath = createTempFile("deltas.txt", INITIAL_DELTAS); - edgesPath = createTempFile("edges.txt", EDGES); - - resultPath = getTempFilePath("results"); - } - - @Override - protected Plan getTestJob() { - String[] params = { String.valueOf(parallelism) , verticesPath, edgesPath, resultPath, "3" }; - - WorksetConnectedComponents cc = new WorksetConnectedComponents(); - return cc.getPlan(params); - } - - @Override - protected void postSubmit() throws Exception { -// compareResultsByLinesInMemory(RESULT_RANKS, resultPath); - } - - - private static final String INITIAL_VERTICES_WITH_RANK = "1 0.025\n" + - "2 0.125\n" + - "3 0.0833333333333333\n" + - "4 0.0833333333333333\n" + - "5 0.075\n" + - "6 0.075\n" + - "7 0.183333333333333\n" + - "8 0.15\n" + - "9 0.1\n"; - - private static final String INITIAL_DELTAS = "1 -0.075\n" + - "2 0.025\n" + - "3 -0.0166666666666667\n" + - "4 -0.0166666666666667\n" + - "5 -0.025\n" + - "6 -0.025\n" + - "7 0.0833333333333333\n" + - "8 0.05\n" + - "9 0\n"; - - private static final String EDGES = "1 2 2\n" + - "1 3 2\n" + - "2 3 3\n" + - "2 4 3\n" + - "3 1 4\n" + - "3 2 4\n" + - "4 2 2\n" + - "5 6 2\n" + - "6 5 2\n" + - "7 8 2\n" + - "7 9 2\n" + - "8 7 2\n" + - "8 9 2\n" + - "9 7 2\n" + - "9 8 2\n" + - "3 5 4\n" + - "3 6 4\n" + - "4 8 2\n" + - "2 7 3\n" + - "5 7 2\n" + - "6 4 2\n"; - -// private static final String RESULT_RANKS = "1 0.00698784722222222\n" + -// "2 0.0326822916666667\n" + -// "3 0.0186631944444444\n" + -// "4 0.0293402777777778\n" + -// "5 0.0220920138888889\n" + -// "6 0.0220920138888889\n" + -// "7 0.262152777777778\n" + -// "8 0.260763888888889\n" + -// "9 0.245225694444444\n"; -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java index 0915a42a8031c..6350533facba2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java @@ -18,115 +18,55 @@ package org.apache.flink.test.iterative; -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; +import java.util.List; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.IterativeDataSet; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; -import org.junit.Assert; - -@SuppressWarnings("deprecation") -public class IterationTerminationWithTerminationTail extends RecordAPITestBase { - private static final String INPUT = "1\n" + "2\n" + "3\n" + "4\n" + "5\n"; +public class IterationTerminationWithTerminationTail extends JavaProgramTestBase { private static final String EXPECTED = "22\n"; - protected String dataPath; - protected String resultPath; - - public IterationTerminationWithTerminationTail(){ - setTaskManagerNumSlots(parallelism); - } - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", INPUT); - resultPath = getTempFilePath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); - @Override - protected Plan getTestJob() { - return getTestPlanPlan(parallelism, dataPath, resultPath); - } - - private static Plan getTestPlanPlan(int numSubTasks, String input, String output) { + DataSet initialInput = env.fromElements("1", "2", "3", "4", "5").name("input"); + + IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - FileDataSource initialInput = new FileDataSource(TextInputFormat.class, input, "input"); - - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(5); - Assert.assertTrue(iteration.getMaximumNumberOfIterations() > 1); + DataSet sumReduce = iteration.reduceGroup(new SumReducer()).name("Compute sum (GroupReduce"); - ReduceOperator sumReduce = ReduceOperator.builder(new SumReducer()) - .input(iteration.getPartialSolution()) - .name("Compute sum (Reduce)") - .build(); - - iteration.setNextPartialSolution(sumReduce); - - MapOperator terminationMapper = MapOperator.builder(new TerminationMapper()) - .input(sumReduce) - .name("Compute termination criterion (Map)") - .build(); - - iteration.setTerminationCriterion(terminationMapper); + DataSet terminationFilter = sumReduce.filter(new TerminationFilter()).name("Compute termination criterion (Map)"); - FileDataSink finalResult = new FileDataSink(CsvOutputFormat.class, output, iteration, "Output"); - CsvOutputFormat.configureRecordFormat(finalResult) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0); + List result = iteration.closeWith(sumReduce, terminationFilter).collect(); - Plan plan = new Plan(finalResult, "Iteration with AllReducer (keyless Reducer)"); - plan.setDefaultParallelism(numSubTasks); - Assert.assertTrue(plan.getDefaultParallelism() > 1); - return plan; + containsResultAsText(result, EXPECTED); } - - public static final class SumReducer extends ReduceFunction implements Serializable { - + + public static final class SumReducer implements GroupReduceFunction { private static final long serialVersionUID = 1L; - + @Override - public void reduce(Iterator it, Collector out) { - // Compute the sum + public void reduce(Iterable values, Collector out) throws Exception { int sum = 0; - while (it.hasNext()) { - sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; + for (String value : values) { + sum += Integer.parseInt(value) + 1; } - - out.collect(new Record(new StringValue(Integer.toString(sum)))); + out.collect("" + sum); } } - - public static class TerminationMapper extends MapFunction implements Serializable { + + public static class TerminationFilter implements FilterFunction { private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector collector) { - - int currentSum = Integer.parseInt(record.getField(0, StringValue.class).getValue()); - if(currentSum < 22) - collector.collect(record); + @Override + public boolean filter(String value) throws Exception { + return Integer.parseInt(value) < 22; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java index 3ce021bf1672e..5a2df3f5b1693 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java @@ -18,116 +18,56 @@ package org.apache.flink.test.iterative; -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; +import java.util.List; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.IterativeDataSet; +import org.apache.flink.test.util.JavaProgramTestBase; +import static org.apache.flink.test.util.TestBaseUtils.containsResultAsText; import org.apache.flink.util.Collector; -import org.junit.Assert; - -@SuppressWarnings("deprecation") -public class IterationTerminationWithTwoTails extends RecordAPITestBase { - private static final String INPUT = "1\n" + "2\n" + "3\n" + "4\n" + "5\n"; +public class IterationTerminationWithTwoTails extends JavaProgramTestBase { private static final String EXPECTED = "22\n"; - protected String dataPath; - protected String resultPath; - - public IterationTerminationWithTwoTails(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", INPUT); - resultPath = getTempFilePath("result"); - } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); - @Override - protected Plan getTestJob() { - return getTestPlanPlan(parallelism, dataPath, resultPath); - } - - private static Plan getTestPlanPlan(int numSubTasks, String input, String output) { + DataSet initialInput = env.fromElements("1", "2", "3", "4", "5").name("input"); - FileDataSource initialInput = new FileDataSource(TextInputFormat.class, input, "input"); - - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(5); - Assert.assertTrue(iteration.getMaximumNumberOfIterations() > 1); + IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - ReduceOperator sumReduce = ReduceOperator.builder(new SumReducer()) - .input(iteration.getPartialSolution()) - .name("Compute sum (Reduce)") - .build(); - - iteration.setNextPartialSolution(sumReduce); - - MapOperator terminationMapper = MapOperator.builder(new TerminationMapper()) - .input(iteration.getPartialSolution()) - .name("Compute termination criterion (Map)") - .build(); - - iteration.setTerminationCriterion(terminationMapper); + DataSet sumReduce = iteration.reduceGroup(new SumReducer()).name("Compute sum (GroupReduce"); - FileDataSink finalResult = new FileDataSink(CsvOutputFormat.class, output, iteration, "Output"); - CsvOutputFormat.configureRecordFormat(finalResult) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0); + DataSet terminationFilter = iteration.filter(new TerminationFilter()).name("Compute termination criterion (Map)"); - Plan plan = new Plan(finalResult, "Iteration with AllReducer (keyless Reducer)"); - plan.setDefaultParallelism(4); - Assert.assertTrue(plan.getDefaultParallelism() > 1); - return plan; + List result = iteration.closeWith(sumReduce, terminationFilter).collect(); + + containsResultAsText(result, EXPECTED); } - - static final class SumReducer extends ReduceFunction implements Serializable { - + + public static final class SumReducer implements GroupReduceFunction { private static final long serialVersionUID = 1L; - + @Override - public void reduce(Iterator it, Collector out) { - // Compute the sum + public void reduce(Iterable values, Collector out) throws Exception { int sum = 0; - - while (it.hasNext()) { - sum += Integer.parseInt(it.next().getField(0, StringValue.class).getValue()) + 1; + for (String value : values) { + sum += Integer.parseInt(value) + 1; } - - out.collect(new Record(new StringValue(Integer.toString(sum)))); + out.collect("" + sum); } } - - public static class TerminationMapper extends MapFunction implements Serializable { + + public static class TerminationFilter implements FilterFunction { private static final long serialVersionUID = 1L; - + @Override - public void map(Record record, Collector collector) { - - int currentSum = Integer.parseInt(record.getField(0, StringValue.class).getValue()); - - if(currentSum < 21) - collector.collect(record); + public boolean filter(String value) throws Exception { + return Integer.parseInt(value) < 21; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java index cb16c154a2744..ab66f31f250f9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java @@ -18,91 +18,34 @@ package org.apache.flink.test.iterative; -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.Assert; - -@SuppressWarnings("deprecation") -public class IterationWithAllReducerITCase extends RecordAPITestBase { - - private static final String INPUT = "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n"; +import java.util.List; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.IterativeDataSet; +import org.apache.flink.test.util.JavaProgramTestBase; + +public class IterationWithAllReducerITCase extends JavaProgramTestBase { private static final String EXPECTED = "1\n"; - protected String dataPath; - protected String resultPath; - - public IterationWithAllReducerITCase(){ - setTaskManagerNumSlots(4); - } - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", INPUT); - resultPath = getTempFilePath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } - - @Override - protected Plan getTestJob() { - Plan plan = getTestPlanPlan(parallelism, dataPath, resultPath); - return plan; - } + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); - - private static Plan getTestPlanPlan(int numSubTasks, String input, String output) { + DataSet initialInput = env.fromElements("1", "1", "1", "1", "1", "1", "1", "1"); - FileDataSource initialInput = new FileDataSource(TextInputFormat.class, input, "input"); - - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(5); - - Assert.assertTrue(iteration.getMaximumNumberOfIterations() > 1); + IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - ReduceOperator sumReduce = ReduceOperator.builder(new PickOneReducer()) - .input(iteration.getPartialSolution()) - .name("Compute sum (Reduce)") - .build(); - - iteration.setNextPartialSolution(sumReduce); + DataSet sumReduce = iteration.reduce(new ReduceFunction(){ + @Override + public String reduce(String value1, String value2) throws Exception { + return value1; + } + }).name("Compute sum (Reduce)"); - FileDataSink finalResult = new FileDataSink(CsvOutputFormat.class, output, iteration, "Output"); - CsvOutputFormat.configureRecordFormat(finalResult) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0); + List result = iteration.closeWith(sumReduce).collect(); - Plan plan = new Plan(finalResult, "Iteration with AllReducer (keyless Reducer)"); - - plan.setDefaultParallelism(numSubTasks); - Assert.assertTrue(plan.getDefaultParallelism() > 1); - - return plan; - } - - public static final class PickOneReducer extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator it, Collector out) { - out.collect(it.next()); - } + compareResultAsText(result, EXPECTED); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java index c11c9eab3460f..c283df1dc68eb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java @@ -18,43 +18,25 @@ package org.apache.flink.test.iterative; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.IterativeDataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.test.util.PointFormatter; +import org.apache.flink.test.util.PointInFormat; +import org.apache.flink.test.util.CoordVector; +import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class IterationWithChainingITCase extends RecordAPITestBase { +public class IterationWithChainingITCase extends JavaProgramTestBase { private static final String DATA_POINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; private String dataPath; private String resultPath; - public IterationWithChainingITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(parallelism); - } - @Override protected void preSubmit() throws Exception { dataPath = createTempFile("data_points.txt", DATA_POINTS); @@ -62,63 +44,35 @@ protected void preSubmit() throws Exception { } @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(DATA_POINTS, resultPath); - } - - @Override - protected Plan getTestJob() { - return getTestPlan(config.getInteger("ChainedMapperITCase#NoSubtasks", 1), dataPath, resultPath); - } + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); - @Parameters - public static Collection getConfigurations() { - Configuration config1 = new Configuration(); - config1.setInteger("ChainedMapperITCase#NoSubtasks", parallelism); - return toParameterList(config1); - } + DataSet> initialInput + = env.readFile(new PointInFormat(), dataPath).setParallelism(1).name("Input"); - public static final class IdentityMapper extends MapFunction implements Serializable { + IterativeDataSet> iteration = initialInput.iterate(2).name("Loop"); - private static final long serialVersionUID = 1L; + DataSet> identity + = iteration.groupBy(0).reduceGroup(new GroupReduceFunction, Tuple2>() { + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + for (Tuple2 value : values) { + out.collect(value); + } + } + }).map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Tuple2 value) throws Exception { + return value; + } - @Override - public void map(Record rec, Collector out) { - out.collect(rec); - } - } + }); - public static final class DummyReducer extends ReduceFunction implements Serializable { + iteration.closeWith(identity).writeAsFormattedText(resultPath, new PointFormatter()); - private static final long serialVersionUID = 1L; + env.execute("Iteration with chained map test"); - @Override - public void reduce(Iterator it, Collector out) { - while (it.hasNext()) { - out.collect(it.next()); - } - } - } - - static Plan getTestPlan(int numSubTasks, String input, String output) { - - FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input"); - initialInput.setParallelism(1); - - BulkIteration iteration = new BulkIteration("Loop"); - iteration.setInput(initialInput); - iteration.setMaximumNumberOfIterations(2); - - ReduceOperator dummyReduce = ReduceOperator.builder(new DummyReducer(), IntValue.class, 0).input(iteration.getPartialSolution()) - .name("Reduce something").build(); - - MapOperator dummyMap = MapOperator.builder(new IdentityMapper()).input(dummyReduce).build(); - iteration.setNextPartialSolution(dummyMap); - - FileDataSink finalResult = new FileDataSink(new PointOutFormat(), output, iteration, "Output"); - - Plan plan = new Plan(finalResult, "Iteration with chained map test"); - plan.setDefaultParallelism(numSubTasks); - return plan; + compareResultsByLinesInMemory(DATA_POINTS, resultPath); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java index 2a4a4b7dd34a1..8756429e966f2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java @@ -25,10 +25,11 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.test.util.CoordVector; +import org.apache.flink.test.util.PointFormatter; +import org.apache.flink.test.util.PointInFormat; import org.apache.flink.test.util.JavaProgramTestBase; -import org.apache.flink.types.Record; import org.apache.flink.util.Collector; public class IterationWithUnionITCase extends JavaProgramTestBase { @@ -54,32 +55,32 @@ protected void postSubmit() throws Exception { protected void testProgram() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet initialInput = env.readFile(new PointInFormat(), this.dataPath).setParallelism(1); + DataSet> initialInput = env.readFile(new PointInFormat(), this.dataPath).setParallelism(1); - IterativeDataSet iteration = initialInput.iterate(2); + IterativeDataSet> iteration = initialInput.iterate(2); - DataSet result = iteration.union(iteration).map(new IdentityMapper()); + DataSet> result = iteration.union(iteration).map(new IdentityMapper()); - iteration.closeWith(result).write(new PointOutFormat(), this.resultPath); + iteration.closeWith(result).writeAsFormattedText(this.resultPath, new PointFormatter()); env.execute(); } - static final class IdentityMapper implements MapFunction, Serializable { + static final class IdentityMapper implements MapFunction, Tuple2>, Serializable { private static final long serialVersionUID = 1L; @Override - public Record map(Record rec) { + public Tuple2 map(Tuple2 rec) { return rec; } } - static class DummyReducer implements GroupReduceFunction, Serializable { + static class DummyReducer implements GroupReduceFunction, Tuple2>, Serializable { private static final long serialVersionUID = 1L; @Override - public void reduce(Iterable it, Collector out) { - for (Record r : it) { + public void reduce(Iterable> it, Collector> out) { + for (Tuple2 r : it) { out.collect(r); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java deleted file mode 100644 index ac3659a8193e7..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.iterative; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast; -import org.apache.flink.test.testdata.KMeansData; -import org.apache.flink.test.util.RecordAPITestBase; - - -public class IterativeKMeansITCase extends RecordAPITestBase { - - protected String dataPath; - protected String clusterPath; - protected String resultPath; - - public IterativeKMeansITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS); - clusterPath = createTempFile("initial_centers.txt", KMeansData.INITIAL_CENTERS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - KMeansBroadcast kmi = new KMeansBroadcast(); - return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20"); - } - - - @Override - protected void postSubmit() throws Exception { - List resultLines = new ArrayList(); - readAllResultLines(resultLines, resultPath); - - KMeansData.checkResultsWithDelta(KMeansData.CENTERS_AFTER_20_ITERATIONS_SINGLE_DIGIT, resultLines, 0.1); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java deleted file mode 100644 index fcf43df0cbb51..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.iterative; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast; -import org.apache.flink.test.testdata.KMeansData; -import org.apache.flink.test.util.RecordAPITestBase; - - -public class KMeansITCase extends RecordAPITestBase { - - protected String dataPath; - protected String clusterPath; - protected String resultPath; - - public KMeansITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS); - clusterPath = createTempFile("initial_centers.txt", KMeansData.INITIAL_CENTERS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - KMeansBroadcast kmi = new KMeansBroadcast(); - return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20"); - } - - - @Override - protected void postSubmit() throws Exception { - List resultLines = new ArrayList(); - readAllResultLines(resultLines, resultPath); - - KMeansData.checkResultsWithDelta(KMeansData.CENTERS_AFTER_20_ITERATIONS_SINGLE_DIGIT, resultLines, 0.1); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java index 5b78a51388d69..35d615bcdb1af 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.operators; +package org.apache.flink.test.javaApiOperators; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.Order; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java index 3cb5580ad1974..38010b35d74d6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.operators; +package org.apache.flink.test.javaApiOperators; import java.util.ArrayList; import java.util.List; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java index faf6de5c8d8b2..9019b8f9c54af 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.test.operators; +package org.apache.flink.test.javaApiOperators; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.ReduceFunction; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java deleted file mode 100644 index be0518692651c..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.operators; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.CoGroupFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.api.java.record.operators.CoGroupOperator; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class CoGroupITCase extends RecordAPITestBase { - - String leftInPath = null; - String rightInPath = null; - String resultPath = null; - - public CoGroupITCase(Configuration testConfig) { - super(testConfig); - } - - private static final String LEFT_IN = "1 1\n2 2\n3 3\n4 4\n1 2\n2 3\n3 4\n4 5\n" + - "1 3\n2 4\n3 5\n4 6\n1 4\n2 5\n3 6\n4 7\n"; - - private static final String RIGHT_IN = "1 1\n2 2\n3 3\n5 1\n1 1\n2 2\n3 3\n6 1\n" + - "1 1\n2 2\n2 2\n7 1\n1 1\n2 2\n2 2\n8 1\n"; - - private static final String RESULT = "1 6\n2 2\n3 12\n4 22\n5 -1\n6 -1\n7 -1\n8 -1\n"; - - @Override - protected void preSubmit() throws Exception { - leftInPath = createTempFile("left_in.txt", LEFT_IN); - rightInPath = createTempFile("right_in.txt", RIGHT_IN); - resultPath = getTempDirPath("result"); - } - - public static class CoGroupTestInFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final StringValue keyString = new StringValue(); - private final StringValue valueString = new StringValue(); - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - this.keyString.setValueAscii(bytes, offset, 1); - this.valueString.setValueAscii(bytes, offset + 2, 1); - target.setField(0, keyString); - target.setField(1, valueString); - - return target; - } - - } - - public static class CoGroupOutFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - private final StringBuilder buffer = new StringBuilder(); - private final StringValue keyString = new StringValue(); - private final IntValue valueInteger = new IntValue(); - - @Override - public void writeRecord(Record record) throws IOException { - this.buffer.setLength(0); - this.buffer.append(record.getField(0, keyString).toString()); - this.buffer.append(' '); - this.buffer.append(record.getField(1, valueInteger).getValue()); - this.buffer.append('\n'); - - byte[] bytes = this.buffer.toString().getBytes(); - this.stream.write(bytes); - } - } - - public static class TestCoGrouper extends CoGroupFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue keyString = new StringValue(); - private StringValue valueString = new StringValue(); - - @Override - public void coGroup(Iterator records1, Iterator records2, Collector out) { - - Record record = null; - int sum = 0; - - while (records1.hasNext()) { - record = records1.next(); - keyString = record.getField(0, keyString); - valueString = record.getField(1, valueString); - sum += Integer.parseInt(valueString.getValue()); - } - - - while (records2.hasNext()) { - record = records2.next(); - keyString = record.getField(0, keyString); - valueString = record.getField(1, valueString); - sum -= Integer.parseInt(valueString.getValue()); - } - record.setField(1, new IntValue(sum)); - - out.collect(record); - } - - } - - @Override - protected Plan getTestJob() { - FileDataSource input_left = new FileDataSource(new CoGroupTestInFormat(), leftInPath); - DelimitedInputFormat.configureDelimitedFormat(input_left) - .recordDelimiter('\n'); - input_left.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); - - FileDataSource input_right = new FileDataSource(new CoGroupTestInFormat(), rightInPath); - DelimitedInputFormat.configureDelimitedFormat(input_right) - .recordDelimiter('\n'); - input_right.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); - - CoGroupOperator testCoGrouper = CoGroupOperator.builder(new TestCoGrouper(), StringValue.class, 0, 0) - .build(); - testCoGrouper.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); - testCoGrouper.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, - config.getString("CoGroupTest#LocalStrategy", "")); - testCoGrouper.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, - config.getString("CoGroupTest#ShipStrategy", "")); - - FileDataSink output = new FileDataSink(new CoGroupOutFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testCoGrouper); - testCoGrouper.setFirstInput(input_left); - testCoGrouper.setSecondInput(input_right); - - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - - LinkedList tConfigs = new LinkedList(); - - String[] localStrategies = { Optimizer.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE }; - - String[] shipStrategies = { Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH }; - - for (String localStrategy : localStrategies) { - for (String shipStrategy : shipStrategies) { - - Configuration config = new Configuration(); - config.setString("CoGroupTest#LocalStrategy", localStrategy); - config.setString("CoGroupTest#ShipStrategy", shipStrategy); - config.setInteger("CoGroupTest#NoSubtasks", 4); - - tConfigs.add(config); - } - } - - return toParameterList(tConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java deleted file mode 100644 index 3fde5a9f81758..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * 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.operators; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.CrossFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.CrossOperator; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.LinkedList; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class CrossITCase extends RecordAPITestBase { - - private String leftInPath = null; - private String rightInPath = null; - private String resultPath = null; - - public CrossITCase(Configuration testConfig) { - super(testConfig); - } - - //private static final String LEFT_IN = "1 1\n2 2\n1 1\n2 2\n3 3\n4 4\n3 3\n4 4\n"; - - //private static final String RIGHT_IN = "1 1\n1 2\n2 2\n2 4\n3 3\n3 6\n4 4\n4 8\n"; - - //private static final String RESULT = "4 1\n4 1\n4 2\n4 2\n5 2\n5 2\n5 4\n5 4\n6 3\n6 3\n7 4\n7 4\n" - // + "5 0\n5 0\n5 1\n5 1\n6 1\n6 1\n6 3\n6 3\n7 2\n7 2\n8 3\n8 3\n" - // + "6 -1\n6 -1\n6 0\n6 0\n7 0\n7 0\n8 1\n8 1\n" + "7 -2\n7 -2\n7 -1\n7 -1\n8 -1\n8 -1\n"; - - //private static final String RESULT = "10 1\n10 1\n10 5\n10 5\n4 1\n4 1\n4 2\n4 2\n5 0\n5 0\n5 1\n," + - // "5 1\n5 2\n5 2\n5 4\n5 4\n6 -1\n6 -1\n6 0\n6 0\n6 1\n6 1\n6 3\n6 3\n6 3\n6 3\n6 6\n6 6\n7 -1\n" + - // "7 -1\n7 -2\n7 -2\n7 0\n7 0\n7 2\n7 2\n7 2\n7 2\n7 4\n7 4\n7 5\n7 5\n7 8\n7 8\n8 -1\n8 -1\n8 1\n" + - // "8 1\n8 1\n8 1\n8 3\n8 3\n8 4\n8 4\n8 7\n8 7\n9 0\n9 0\n9 2\n9 2\n9 3\n9 3\n9 6\n9 6\n"; - - //private static final String RESULT = "2 2\n4 4\n1 1\n3 3\n2 2\n4 4\n1 1\n3 3\n5 0\n5 1\n6 1\n 6 3\n" + - // "7 2\n7 5\n8 3\n8 7\n7 -2\n7 -1\n8 -1\n8 1\n9 0\n9 3\n10 1\n10 5\n4 1\n4 2\n5 2\n5 4\n6 3\n" + - // "6 6\n7 4\n7 8\n6 -1\n6 0\n7 0\n7 2\n8 1\n8 4\n9 2\n9 6\n5 0\n5 1\n6 1\n6 3\n7 2\n7 5\n 8 3\n" + - // "8 7\n7 -2\n7 -1\n8 -1\n8 1\n9 0\n9 3\n10 1\n10 5\n4 1\n4 2\n5 2\n5 4\n6 3\n6 6\n7 4\n7 8\n" + - // "6 -1\n6 0\n7 0\n7 2\n8 1\n8 4\n9 2\n9 6"; - - - private static final String LEFT_IN = "1 1\n2 2\n3 3\n"; - private static final String RIGHT_IN = "3 6\n4 4\n4 8\n"; - - private static final String RESULT = "6 6\n7 5\n7 8\n7 4\n8 3\n8 7\n8 4\n9 2\n9 6\n"; - - @Override - protected void preSubmit() throws Exception { - leftInPath = createTempFile("left_in.txt", LEFT_IN); - rightInPath = createTempFile("right_in.txt", RIGHT_IN); - resultPath = getTempDirPath("result"); - } - - - public static class TestCross extends CrossFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue string = new StringValue(); - private IntValue integer = new IntValue(); - - @Override - public Record cross(Record record1, Record record2) throws Exception { - string = record1.getField(1, string); - int val1 = Integer.parseInt(string.toString()); - string = record2.getField(1, string); - int val2 = Integer.parseInt(string.toString()); - string = record1.getField(0, string); - int key1 = Integer.parseInt(string.toString()); - string = record2.getField(0, string); - int key2 = Integer.parseInt(string.toString()); - - string.setValue((key1 + key2 + 2) + ""); - integer.setValue(val2 - val1 + 1); - - record1.setField(0, string); - record1.setField(1, integer); - - return record1; - } - - } - - @Override - protected Plan getTestJob() { - - FileDataSource input_left = new FileDataSource( - new ContractITCaseInputFormat(), leftInPath); - DelimitedInputFormat.configureDelimitedFormat(input_left) - .recordDelimiter('\n'); - input_left.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); - - FileDataSource input_right = new FileDataSource( - new ContractITCaseInputFormat(), rightInPath); - DelimitedInputFormat.configureDelimitedFormat(input_right) - .recordDelimiter('\n'); - input_right.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); - - CrossOperator testCross = CrossOperator.builder(new TestCross()).build(); - testCross.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); - testCross.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, - config.getString("CrossTest#LocalStrategy", "")); - if (config.getString("CrossTest#ShipStrategy", "").equals("BROADCAST_FIRST")) { - testCross.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_BROADCAST); - testCross.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_FORWARD); - } else if (config.getString("CrossTest#ShipStrategy", "").equals("BROADCAST_SECOND")) { - testCross.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_BROADCAST); - testCross.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_FORWARD); - } else { - testCross.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, - config.getString("CrossTest#ShipStrategy", "")); - } - - FileDataSink output = new FileDataSink( - new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testCross); - testCross.setFirstInput(input_left); - testCross.setSecondInput(input_right); - - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - - LinkedList tConfigs = new LinkedList(); - - String[] localStrategies = { Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND }; - - String[] shipStrategies = { "BROADCAST_FIRST", "BROADCAST_SECOND" - // PactCompiler.HINT_SHIP_STRATEGY_BROADCAST - // PactCompiler.HINT_SHIP_STRATEGY_SFR - }; - - for (String localStrategy : localStrategies) { - for (String shipStrategy : shipStrategies) { - - Configuration config = new Configuration(); - config.setString("CrossTest#LocalStrategy", localStrategy); - config.setString("CrossTest#ShipStrategy", shipStrategy); - config.setInteger("CrossTest#NoSubtasks", 4); - - tConfigs.add(config); - } - } - - return toParameterList(tConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java deleted file mode 100644 index c2ec55ad4e1b4..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java +++ /dev/null @@ -1,182 +0,0 @@ -/* - * 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.operators; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.LinkedList; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class JoinITCase extends RecordAPITestBase { - - private static final Logger LOG = LoggerFactory.getLogger(JoinITCase.class); - - String leftInPath = null; - String rightInPath = null; - String resultPath = null; - - public JoinITCase(Configuration testConfig) { - super(testConfig); - } - - private static final String LEFT_IN = "1 1\n2 2\n3 3\n4 4\n1 2\n2 3\n3 4\n4 5\n" + - "1 3\n2 4\n3 5\n4 6\n1 4\n2 5\n3 6\n4 7\n"; - - private static final String RIGHT_IN = "1 1\n2 2\n3 3\n5 1\n1 1\n2 2\n3 3\n6 1\n" + - "1 1\n2 2\n2 2\n7 1\n1 1\n2 2\n2 2\n8 1\n"; - - private static final String RESULT = "2 1\n2 1\n2 1\n2 1\n2 2\n2 2\n2 2\n2 2\n2 3\n2 3\n2 3\n2 3\n2 4\n2 4\n2 4\n2 4\n" - + "4 1\n4 1\n4 2\n4 2\n4 3\n4 3\n4 4\n4 4\n" - + "3 1\n3 2\n3 3\n3 4\n3 1\n3 2\n3 3\n3 4\n3 1\n3 2\n3 3\n3 4\n3 1\n3 2\n3 3\n3 4\n3 1\n3 2\n3 3\n3 4\n3 1\n3 2\n3 3\n3 4\n"; - - @Override - protected void preSubmit() throws Exception { - leftInPath = createTempFile("left_in.txt", LEFT_IN); - rightInPath = createTempFile("right_in.txt", RIGHT_IN); - resultPath = getTempDirPath("result"); - } - - public static class TestMatcher extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue keyString = new StringValue(); - private StringValue valueString = new StringValue(); - - @Override - public void join(Record value1, Record value2, Collector out) - throws Exception { - keyString = value1.getField(0, keyString); - keyString.setValue(""+ (Integer.parseInt(keyString.getValue())+1)); - value1.setField(0, keyString); - valueString = value1.getField(1, valueString); - int val1 = Integer.parseInt(valueString.getValue())+2; - valueString = value2.getField(1, valueString); - int val2 = Integer.parseInt(valueString.getValue())+1; - - value1.setField(1, new IntValue(val1 - val2)); - - out.collect(value1); - - if (LOG.isDebugEnabled()) - LOG.debug("Processed: [" + keyString.toString() + "," + val1 + "] + " + - "[" + keyString.toString() + "," + val2 + "]"); - } - - } - - @Override - protected Plan getTestJob() { - FileDataSource input_left = new FileDataSource( - new ContractITCaseInputFormat(), leftInPath); - DelimitedInputFormat.configureDelimitedFormat(input_left) - .recordDelimiter('\n'); - input_left.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); - - FileDataSource input_right = new FileDataSource( - new ContractITCaseInputFormat(), rightInPath); - DelimitedInputFormat.configureDelimitedFormat(input_right) - .recordDelimiter('\n'); - input_right.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); - - JoinOperator testMatcher = JoinOperator.builder(new TestMatcher(), StringValue.class, 0, 0) - .build(); - testMatcher.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); - testMatcher.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, - config.getString("MatchTest#LocalStrategy", "")); - if (config.getString("MatchTest#ShipStrategy", "").equals("BROADCAST_FIRST")) { - testMatcher.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_BROADCAST); - testMatcher.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_FORWARD); - } else if (config.getString("MatchTest#ShipStrategy", "").equals("BROADCAST_SECOND")) { - testMatcher.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_FORWARD); - testMatcher.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_BROADCAST); - } else { - testMatcher.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, - config.getString("MatchTest#ShipStrategy", "")); - } - - FileDataSink output = new FileDataSink( - new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testMatcher); - testMatcher.setFirstInput(input_left); - testMatcher.setSecondInput(input_right); - - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - - LinkedList tConfigs = new LinkedList(); - - String[] localStrategies = { Optimizer.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE, - Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST, Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND }; - - String[] shipStrategies = { Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH, "BROADCAST_FIRST", "BROADCAST_SECOND"}; - - for (String localStrategy : localStrategies) { - for (String shipStrategy : shipStrategies) { - - Configuration config = new Configuration(); - config.setString("MatchTest#LocalStrategy", localStrategy); - config.setString("MatchTest#ShipStrategy", shipStrategy); - config.setInteger("MatchTest#NoSubtasks", 4); - - tConfigs.add(config); - } - } - - return toParameterList(tConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java deleted file mode 100644 index 28b9501add035..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.operators; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.LinkedList; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class MapITCase extends RecordAPITestBase { - - private static final Logger LOG = LoggerFactory.getLogger(MapITCase.class); - - String inPath = null; - String resultPath = null; - - public MapITCase(Configuration testConfig) { - super(testConfig); - } - - private static final String IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" + - "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n" + - "5 9\n7 7\n8 8\n1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - private static final String RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - @Override - protected void preSubmit() throws Exception { - inPath = createTempFile("in.txt", IN); - resultPath = getTempDirPath("result"); - } - - public static class TestMapper extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue keyString = new StringValue(); - private StringValue valueString = new StringValue(); - - @Override - public void map(Record record, Collector out) throws Exception { - keyString = record.getField(0, keyString); - valueString = record.getField(1, valueString); - - LOG.debug("Processed: [" + keyString.toString() + "," + valueString.getValue() + "]"); - - if (Integer.parseInt(keyString.toString()) + Integer.parseInt(valueString.toString()) < 10) { - - record.setField(0, valueString); - record.setField(1, new IntValue(Integer.parseInt(keyString.toString()) + 10)); - - out.collect(record); - } - - } - } - - @Override - protected Plan getTestJob() { - FileDataSource input = new FileDataSource( - new ContractITCaseInputFormat(), inPath); - DelimitedInputFormat.configureDelimitedFormat(input) - .recordDelimiter('\n'); - input.setParallelism(config.getInteger("MapTest#NoSubtasks", 1)); - - MapOperator testMapper = MapOperator.builder(new TestMapper()).build(); - testMapper.setParallelism(config.getInteger("MapTest#NoSubtasks", 1)); - - FileDataSink output = new FileDataSink( - new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testMapper); - testMapper.setInput(input); - - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - LinkedList testConfigs = new LinkedList(); - - Configuration config = new Configuration(); - config.setInteger("MapTest#NoSubtasks", 4); - testConfigs.add(config); - - return toParameterList(testConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java deleted file mode 100644 index f5511c80426d5..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.operators; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class ReduceITCase extends RecordAPITestBase { - - String inPath = null; - String resultPath = null; - - public ReduceITCase(Configuration testConfig) { - super(testConfig); - } - - private static final String IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n1 1\n" + - "2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n5 9\n7 7\n8 8\n" + - "1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - private static final String RESULT = "1 4\n2 18\n3 0\n4 28\n5 27\n6 15\n7 21\n8 32\n9 1\n"; - - @Override - protected void preSubmit() throws Exception { - inPath = createTempFile("in.txt", IN); - resultPath = getTempDirPath("result"); - } - - @ReduceOperator.Combinable - public static class TestReducer extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue reduceValue = new StringValue(); - private StringValue combineValue = new StringValue(); - - @Override - public void combine(Iterator records, Collector out) { - Record record = null; - int sum = 0; - - while (records.hasNext()) { - record = records.next(); - combineValue = record.getField(1, combineValue); - sum += Integer.parseInt(combineValue.toString()); - } - combineValue.setValue(sum + ""); - record.setField(1, combineValue); - out.collect(record); - } - - @Override - public void reduce(Iterator records, Collector out) { - Record record = null; - int sum = 0; - - while (records.hasNext()) { - record = records.next(); - reduceValue = record.getField(1, reduceValue); - sum += Integer.parseInt(reduceValue.toString()); - } - record.setField(1, new IntValue(sum)); - out.collect(record); - } - } - - @Override - protected JobGraph getJobGraph() throws Exception { - FileDataSource input = new FileDataSource( - new ContractITCaseInputFormat(), inPath); - DelimitedInputFormat.configureDelimitedFormat(input) - .recordDelimiter('\n'); - input.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); - - ReduceOperator testReducer = ReduceOperator.builder(new TestReducer(), StringValue.class, 0) - .build(); - testReducer.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); - testReducer.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, - config.getString("ReduceTest#LocalStrategy", "")); - testReducer.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, - config.getString("ReduceTest#ShipStrategy", "")); - - FileDataSink output = new FileDataSink( - new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testReducer); - testReducer.setInput(input); - - Plan plan = new Plan(output); - plan.setExecutionConfig(new ExecutionConfig()); - Optimizer pc = new Optimizer(new DataStatistics(), this.config); - OptimizedPlan op = pc.compile(plan); - - JobGraphGenerator jgg = new JobGraphGenerator(); - return jgg.compileJobGraph(op); - - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - - LinkedList tConfigs = new LinkedList(); - - String[] localStrategies = { Optimizer.HINT_LOCAL_STRATEGY_SORT }; - String[] shipStrategies = { Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH }; - - for (String localStrategy : localStrategies) { - for (String shipStrategy : shipStrategies) { - - Configuration config = new Configuration(); - config.setString("ReduceTest#LocalStrategy", localStrategy); - config.setString("ReduceTest#ShipStrategy", shipStrategy); - config.setInteger("ReduceTest#NoSubtasks", 4); - tConfigs.add(config); - } - } - - return toParameterList(tConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java deleted file mode 100644 index b833421b406d5..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java +++ /dev/null @@ -1,182 +0,0 @@ -/* - * 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.operators; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.LinkedList; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class UnionITCase extends RecordAPITestBase { - private static final Logger LOG = LoggerFactory.getLogger(UnionITCase.class); - - String inPath = null; - String emptyInPath = null; - String resultPath = null; - - public UnionITCase(Configuration testConfig) { - super(testConfig); - } - - private static final String IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n1 1\n" + - "2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n5 9\n7 7\n8 8\n" + - "1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - private static final String RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n" + - "4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - private static final String EMPTY_RESULT = ""; - - private static final String MAP_RESULT_TWICE = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n" + - "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - @Override - protected void preSubmit() throws Exception { - inPath = createTempFile("in.txt", IN); - emptyInPath = createTempFile("empty_in.txt", ""); - resultPath = getTempDirPath("result"); - } - - public static class TestMapper extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue keyString = new StringValue(); - private StringValue valueString = new StringValue(); - - @Override - public void map(Record record, Collector out) throws Exception { - keyString = record.getField(0, keyString); - valueString = record.getField(1, valueString); - - if (LOG.isDebugEnabled()) - LOG.debug("Processed: [" + keyString.toString() + "," + valueString.getValue() + "]"); - - if (Integer.parseInt(keyString.toString()) + Integer.parseInt(valueString.toString()) < 10) { - - record.setField(0, valueString); - record.setField(1, new IntValue(Integer.parseInt(keyString.toString()) + 10)); - - out.collect(record); - } - - } - } - - @SuppressWarnings("unchecked") - @Override - protected Plan getTestJob() { - String input1Path = config.getString("UnionTest#Input1Path", "").equals("empty") ? emptyInPath : inPath; - String input2Path = config.getString("UnionTest#Input2Path", "").equals("empty") ? emptyInPath : inPath; - - FileDataSource input1 = new FileDataSource( - new ContractITCaseInputFormat(), input1Path); - DelimitedInputFormat.configureDelimitedFormat(input1) - .recordDelimiter('\n'); - input1.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); - - FileDataSource input2 = new FileDataSource( - new ContractITCaseInputFormat(), input2Path); - DelimitedInputFormat.configureDelimitedFormat(input2) - .recordDelimiter('\n'); - input2.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); - - MapOperator testMapper = MapOperator.builder(new TestMapper()).build(); - testMapper.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); - - FileDataSink output = new FileDataSink( - new ContractITCaseOutputFormat(), resultPath); - output.setParallelism(1); - - output.setInput(testMapper); - - testMapper.addInput(input1); - testMapper.addInput(input2); - - return new Plan(output); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(config.getString("UnionTest#ExpectedResult", ""), resultPath); - - } - - @Parameters - public static Collection getConfigurations() throws IOException { - LinkedList testConfigs = new LinkedList(); - - //second input empty - Configuration config = new Configuration(); - config.setInteger("UnionTest#NoSubtasks", 4); - config.setString("UnionTest#ExpectedResult", RESULT); - config.setString("UnionTest#Input1Path", "non-empty"); - config.setString("UnionTest#Input2Path", "empty"); - testConfigs.add(config); - - - //first input empty - config = new Configuration(); - config.setInteger("UnionTest#NoSubtasks", 4); - config.setString("UnionTest#ExpectedResult", RESULT); - config.setString("UnionTest#Input1Path", "empty"); - config.setString("UnionTest#Input2Path", "non-empty"); - testConfigs.add(config); - - //both inputs full - config = new Configuration(); - config.setInteger("UnionTest#NoSubtasks", 4); - config.setString("UnionTest#ExpectedResult", MAP_RESULT_TWICE); - config.setString("UnionTest#Input1Path", "non-empty"); - config.setString("UnionTest#Input2Path", "non-empty"); - testConfigs.add(config); - - //both inputs empty - config = new Configuration(); - config.setInteger("UnionTest#NoSubtasks", 4); - config.setString("UnionTest#ExpectedResult", EMPTY_RESULT); - config.setString("UnionTest#Input1Path", "empty"); - config.setString("UnionTest#Input2Path", "empty"); - testConfigs.add(config); - - return toParameterList(testConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java deleted file mode 100644 index 35cb8af4b8236..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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.operators; - -import java.io.Serializable; -import java.util.Collection; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseInputFormat; -import org.apache.flink.test.operators.io.ContractITCaseIOFormats.ContractITCaseOutputFormat; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@SuppressWarnings("deprecation") -@RunWith(Parameterized.class) -public class UnionSinkITCase extends RecordAPITestBase { - - public UnionSinkITCase(Configuration testConfig) { - super(testConfig); - setTaskManagerNumSlots(parallelism); - } - - private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" + - "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n" + - "1 1\n2 2\n2 2\n3 0\n4 4\n5 9\n7 7\n8 8\n" + - "1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - private static final String MAP_RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - private static final String EMPTY_MAP_RESULT = ""; - - private static final String MAP_RESULT_TWICE = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n" + - "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - private String textInput; - - private String emptyInput; - - private String resultDir; - - @Override - protected void preSubmit() throws Exception { - textInput = createTempFile("textdata.txt", MAP_IN); - emptyInput = createTempFile("emptyfile.txt", ""); - resultDir = getTempDirPath("result"); - } - - public static class TestMapper extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private StringValue keyString = new StringValue(); - private StringValue valueString = new StringValue(); - - @Override - public void map(Record record, Collector out) throws Exception { - keyString = record.getField(0, keyString); - valueString = record.getField(1, valueString); - - - if (Integer.parseInt(keyString.toString()) + Integer.parseInt(valueString.toString()) < 10) { - - record.setField(0, valueString); - record.setField(1, new IntValue(Integer.parseInt(keyString.toString()) + 10)); - - out.collect(record); - } - } - } - - @SuppressWarnings("unchecked") - @Override - protected JobGraph getJobGraph() throws Exception { - - String path1 = config.getBoolean("input1PathHasData", false) ? textInput : emptyInput; - String path2 = config.getBoolean("input2PathHasData", false) ? textInput : emptyInput; - - FileDataSource input1 = new FileDataSource(new ContractITCaseInputFormat(), path1); - FileDataSource input2 = new FileDataSource(new ContractITCaseInputFormat(), path2); - - MapOperator testMapper1 = MapOperator.builder(new TestMapper()).build(); - MapOperator testMapper2 = MapOperator.builder(new TestMapper()).build(); - - FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultDir); - - testMapper1.setInput(input1); - testMapper2.setInput(input2); - - output.addInput(testMapper1); - output.addInput(testMapper2); - - Plan plan = new Plan(output); - plan.setExecutionConfig(new ExecutionConfig()); - plan.setDefaultParallelism(parallelism); - - Optimizer pc = new Optimizer(new DataStatistics(), this.config); - OptimizedPlan op = pc.compile(plan); - - JobGraphGenerator jgg = new JobGraphGenerator(); - return jgg.compileJobGraph(op); - } - - @Override - protected void postSubmit() throws Exception { - String expectedResult = config.getString("UnionTest#ExpectedResult", null); - if (expectedResult == null) { - throw new Exception("Test corrupt, no expected return data set."); - } - compareResultsByLinesInMemory(expectedResult, resultDir); - } - - @Parameters - public static Collection getConfigurations() { - - //second input empty - Configuration config1 = new Configuration(); - config1.setString("UnionTest#ExpectedResult", MAP_RESULT); - config1.setBoolean("input1PathHasData", true); - config1.setBoolean("input2PathHasData", false); - - - //first input empty - Configuration config2 = new Configuration(); - config2.setString("UnionTest#ExpectedResult", MAP_RESULT); - config2.setBoolean("input1PathHasData", false); - config2.setBoolean("input2PathHasData", true); - - //both inputs full - Configuration config3 = new Configuration(); - config3.setString("UnionTest#ExpectedResult", MAP_RESULT_TWICE); - config3.setBoolean("input1PathHasData", true); - config3.setBoolean("input2PathHasData", true); - - //both inputs empty - Configuration config4 = new Configuration(); - config4.setString("UnionTest#ExpectedResult", EMPTY_MAP_RESULT); - config4.setBoolean("input1PathHasData", false); - config4.setBoolean("input2PathHasData", false); - - return toParameterList(config1, config2, config3, config4); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/io/ContractITCaseIOFormats.java b/flink-tests/src/test/java/org/apache/flink/test/operators/io/ContractITCaseIOFormats.java deleted file mode 100644 index 764929939cc49..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/io/ContractITCaseIOFormats.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.operators.io; - -import java.io.IOException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; - -public class ContractITCaseIOFormats { - - private static final Logger LOG = LoggerFactory.getLogger(ContractITCaseIOFormats.class); - - public static class ContractITCaseInputFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final StringValue keyString = new StringValue(); - private final StringValue valueString = new StringValue(); - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - this.keyString.setValueAscii(bytes, offset, 1); - this.valueString.setValueAscii(bytes, offset + 2, 1); - target.setField(0, keyString); - target.setField(1, valueString); - - if (LOG.isDebugEnabled()) - LOG.debug("Read in: [" + keyString.getValue() + "," + valueString.getValue() + "]"); - - return target; - } - } - - public static class ContractITCaseOutputFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - private final StringBuilder buffer = new StringBuilder(); - private final StringValue keyString = new StringValue(); - private final IntValue valueInteger = new IntValue(); - - - public ContractITCaseOutputFormat() { - setWriteMode(WriteMode.OVERWRITE); - } - - @Override - public void writeRecord(Record record) throws IOException { - this.buffer.setLength(0); - this.buffer.append(record.getField(0, keyString).toString()); - this.buffer.append(' '); - this.buffer.append(record.getField(1, valueInteger).getValue()); - this.buffer.append('\n'); - - byte[] bytes = this.buffer.toString().getBytes(); - - if (LOG.isDebugEnabled()) - LOG.debug("Writing out: [" + keyString.toString() + "," + valueInteger.getValue() + "]"); - - this.stream.write(bytes); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java index ab8ff45f8c403..959a17a825e6b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java @@ -21,11 +21,24 @@ import static org.junit.Assert.*; import java.util.Arrays; +import java.util.Collection; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.record.operators.FileDataSource; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.client.program.OptimizerPlanEnvironment; +import org.apache.flink.client.program.PreviewPlanEnvironment; +import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode; @@ -34,11 +47,10 @@ import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.optimizer.util.CompilerTestBase; import org.apache.flink.optimizer.util.OperatorResolver; -import org.apache.flink.test.recordJobs.kmeans.KMeansSingleStep; +import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; -@SuppressWarnings("deprecation") public class KMeansSingleStepTest extends CompilerTestBase { private static final String DATAPOINTS = "Data Points"; @@ -54,16 +66,15 @@ public class KMeansSingleStepTest extends CompilerTestBase { @Test public void testCompileKMeansSingleStepWithStats() { - - KMeansSingleStep kmi = new KMeansSingleStep(); - Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); + + Plan p = getKMeansPlan(); p.setExecutionConfig(new ExecutionConfig()); // set the statistics OperatorResolver cr = getContractResolver(p); - FileDataSource pointsSource = cr.getNode(DATAPOINTS); - FileDataSource centersSource = cr.getNode(CENTERS); - setSourceStatistics(pointsSource, 100l*1024*1024*1024, 32f); - setSourceStatistics(centersSource, 1024*1024, 32f); + GenericDataSourceBase pointsSource = cr.getNode(DATAPOINTS); + GenericDataSourceBase centersSource = cr.getNode(CENTERS); + setSourceStatistics(pointsSource, 100l * 1024 * 1024 * 1024, 32f); + setSourceStatistics(centersSource, 1024 * 1024, 32f); OptimizedPlan plan = compileWithStats(p); checkPlan(plan); @@ -71,9 +82,8 @@ public void testCompileKMeansSingleStepWithStats() { @Test public void testCompileKMeansSingleStepWithOutStats() { - - KMeansSingleStep kmi = new KMeansSingleStep(); - Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); + + Plan p = getKMeansPlan(); p.setExecutionConfig(new ExecutionConfig()); OptimizedPlan plan = compileNoStats(p); checkPlan(plan); @@ -97,7 +107,7 @@ private void checkPlan(OptimizedPlan plan) { assertEquals(LocalStrategy.NONE, mapper.getInput().getLocalStrategy()); assertEquals(LocalStrategy.NONE, mapper.getBroadcastInputs().get(0).getLocalStrategy()); - assertEquals(DriverStrategy.COLLECTOR_MAP, mapper.getDriverStrategy()); + assertEquals(DriverStrategy.MAP, mapper.getDriverStrategy()); assertNull(mapper.getInput().getLocalStrategyKeys()); assertNull(mapper.getInput().getLocalStrategySortOrder()); @@ -127,4 +137,145 @@ private void checkPlan(OptimizedPlan plan) { assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); } + + public static Plan getKMeansPlan() { + // prepare the test environment + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + KMeans(new String[]{IN_FILE, IN_FILE, OUT_FILE, "20"}); + } catch (OptimizerPlanEnvironment.ProgramAbortException pae) { + // all good. + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("KMeans failed with an exception"); + } + return env.getPlan(); + } + + public static void KMeans(String[] args) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet points = env.readCsvFile(args[0]) + .fieldDelimiter(" ") + .includeFields(true, true) + .types(Double.class, Double.class) + .name(DATAPOINTS) + .map(new MapFunction, Point>() { + @Override + public Point map(Tuple2 value) throws Exception { + return new Point(value.f0, value.f1); + } + }); + + DataSet centroids = env.readCsvFile(args[1]) + .fieldDelimiter(" ") + .includeFields(true, true, true) + .types(Integer.class, Double.class, Double.class) + .name(CENTERS) + .map(new MapFunction, Centroid>() { + @Override + public Centroid map(Tuple3 value) throws Exception { + return new Centroid(value.f0, value.f1, value.f2); + } + }); + + DataSet> newCentroids = points + .map(new SelectNearestCenter()).name(MAPPER_NAME).withBroadcastSet(centroids, "centroids"); + + DataSet> recomputeClusterCenter + = newCentroids.groupBy(0).reduceGroup(new RecomputeClusterCenter()).name(REDUCER_NAME); + + recomputeClusterCenter.project(0, 1).writeAsCsv(args[2], "\n", " ").name(SINK); + + env.execute("KMeans Example"); + } + + public static class Point extends Tuple2 { + public Point(double x, double y) { + this.f0 = x; + this.f1 = y; + } + + public Point add(Point other) { + f0 += other.f0; + f1 += other.f1; + return this; + } + + public Point div(long val) { + f0 /= val; + f1 /= val; + return this; + } + + public double euclideanDistance(Point other) { + return Math.sqrt((f0 - other.f0) * (f0 - other.f0) + (f1 - other.f1) * (f1 - other.f1)); + } + + public double euclideanDistance(Centroid other) { + return Math.sqrt((f0 - other.f1.f0) * (f0 - other.f1.f0) + (f1 - other.f1.f1) * (f1 - other.f1.f1)); + } + } + + public static class Centroid extends Tuple2 { + public Centroid(int id, double x, double y) { + this.f0 = id; + this.f1 = new Point(x, y); + } + + public Centroid(int id, Point p) { + this.f0 = id; + this.f1 = p; + } + } + + /** + * Determines the closest cluster center for a data point. + */ + public static final class SelectNearestCenter extends RichMapFunction> { + private Collection centroids; + + @Override + public void open(Configuration parameters) throws Exception { + this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); + } + + @Override + public Tuple3 map(Point p) throws Exception { + double minDistance = Double.MAX_VALUE; + int closestCentroidId = -1; + for (Centroid centroid : centroids) { + double distance = p.euclideanDistance(centroid); + if (distance < minDistance) { + minDistance = distance; + closestCentroidId = centroid.f0; + } + } + return new Tuple3<>(closestCentroidId, p, 1); + } + } + + @Combinable + public static final class RecomputeClusterCenter extends RichGroupReduceFunction, Tuple3> { + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + int id = -1; + double x = 0; + double y = 0; + int count = 0; + for (Tuple3 value : values) { + id = value.f0; + x += value.f1.f0; + y += value.f1.f1; + count += value.f2; + } + out.collect(new Tuple3<>(id, new Point(x, y), count)); + } + + @Override + public void combine(Iterable> values, Collector> out) throws Exception { + reduce(values, out); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java index f4efb8a9abbed..e929913dc0b46 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java @@ -22,10 +22,23 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable; +import org.apache.flink.api.common.operators.DualInputOperator; +import org.apache.flink.api.common.operators.GenericDataSourceBase; +import org.apache.flink.api.common.operators.SingleInputOperator; import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.client.program.OptimizerPlanEnvironment; +import org.apache.flink.client.program.PreviewPlanEnvironment; import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.SingleInputPlanNode; @@ -35,23 +48,25 @@ import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.optimizer.util.CompilerTestBase; import org.apache.flink.optimizer.util.OperatorResolver; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3; +import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; /** * Tests TPCH Q3 (simplified) under various input conditions. */ -@SuppressWarnings("deprecation") +@SuppressWarnings("serial") public class RelationalQueryCompilerTest extends CompilerTestBase { private static final String ORDERS = "Orders"; private static final String LINEITEM = "LineItems"; private static final String MAPPER_NAME = "FilterO"; private static final String JOIN_NAME = "JoinLiO"; + private static final String REDUCE_NAME = "AggLiO"; + private static final String SINK = "Output"; private final FieldList set0 = new FieldList(0); - private final FieldList set01 = new FieldList(new int[] {0,1}); + private final FieldList set01 = new FieldList(0,1); private final ExecutionConfig defaultExecutionConfig = new ExecutionConfig(); // ------------------------------------------------------------------------ @@ -63,8 +78,7 @@ public class RelationalQueryCompilerTest extends CompilerTestBase { @Test public void testQueryNoStatistics() { try { - TPCHQuery3 query = new TPCHQuery3(); - Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE); + Plan p = getTPCH3Plan(); p.setExecutionConfig(defaultExecutionConfig); // compile final OptimizedPlan plan = compileNoStats(p); @@ -72,12 +86,12 @@ public void testQueryNoStatistics() { final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); // get the nodes from the final plan - final SinkPlanNode sink = or.getNode("Output"); - final SingleInputPlanNode reducer = or.getNode("AggLio"); + final SinkPlanNode sink = or.getNode(SINK); + final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; - final DualInputPlanNode join = or.getNode("JoinLiO"); - final SingleInputPlanNode filteringMapper = or.getNode("FilterO"); + final DualInputPlanNode join = or.getNode(JOIN_NAME); + final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME); // verify the optimizer choices checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); @@ -95,7 +109,7 @@ public void testQueryNoStatistics() { */ @Test public void testQueryAnyValidPlan() { - testQueryGeneric(1024*1024*1024L, 8*1024*1024*1024L, 0.05f, true, true, true, false, true); + testQueryGeneric(1024*1024*1024L, 8*1024*1024*1024L, 0.05f, 0.05f, true, true, true, false, true); } /** @@ -103,7 +117,7 @@ public void testQueryAnyValidPlan() { */ @Test public void testQueryWithSizeZeroInputs() { - testQueryGeneric(0, 0, 0.5f, true, true, true, false, true); + testQueryGeneric(0, 0, 0.1f, 0.5f, true, true, true, false, true); } /** @@ -111,7 +125,7 @@ public void testQueryWithSizeZeroInputs() { */ @Test public void testQueryWithStatsForBroadcastHash() { - testQueryGeneric(1024l*1024*1024*1024, 1024l*1024*1024*1024, 0.05f, true, false, true, false, false); + testQueryGeneric(1024l*1024*1024*1024, 1024l*1024*1024*1024, 0.01f, 0.05f, true, false, true, false, false); } /** @@ -119,7 +133,7 @@ public void testQueryWithStatsForBroadcastHash() { */ @Test public void testQueryWithStatsForRepartitionAny() { - testQueryGeneric(100l*1024*1024*1024*1024, 100l*1024*1024*1024*1024, 0.5f, false, true, true, true, true); + testQueryGeneric(100l*1024*1024*1024*1024, 100l*1024*1024*1024*1024, 0.1f, 0.5f, false, true, true, true, true); } /** @@ -128,34 +142,23 @@ public void testQueryWithStatsForRepartitionAny() { */ @Test public void testQueryWithStatsForRepartitionMerge() { - TPCHQuery3 query = new TPCHQuery3(); - Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE); + Plan p = getTPCH3Plan(); p.setExecutionConfig(defaultExecutionConfig); // set compiler hints OperatorResolver cr = getContractResolver(p); - JoinOperator match = cr.getNode("JoinLiO"); + DualInputOperator match = cr.getNode(JOIN_NAME); match.getCompilerHints().setFilterFactor(100f); - testQueryGeneric(100l*1024*1024*1024*1024, 100l*1024*1024*1024*1024, 0.05f, 100f, false, true, false, false, true); + testQueryGeneric(100l*1024*1024*1024*1024, 100l*1024*1024*1024*1024, 0.01f, 100f, false, true, false, false, true); } // ------------------------------------------------------------------------ - - private void testQueryGeneric(long orderSize, long lineItemSize, - float ordersFilterFactor, - boolean broadcastOkay, boolean partitionedOkay, - boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay) - { - testQueryGeneric(orderSize, lineItemSize, ordersFilterFactor, ordersFilterFactor, broadcastOkay, partitionedOkay, hashJoinFirstOkay, hashJoinSecondOkay, mergeJoinOkay); - } - private void testQueryGeneric(long orderSize, long lineItemSize, float ordersFilterFactor, float joinFilterFactor, boolean broadcastOkay, boolean partitionedOkay, boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay) { - TPCHQuery3 query = new TPCHQuery3(); - Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE); + Plan p = getTPCH3Plan(); p.setExecutionConfig(defaultExecutionConfig); testQueryGeneric(p, orderSize, lineItemSize, ordersFilterFactor, joinFilterFactor, broadcastOkay, partitionedOkay, hashJoinFirstOkay, hashJoinSecondOkay, mergeJoinOkay); } @@ -168,10 +171,10 @@ private void testQueryGeneric(Plan p, long orderSize, long lineitemSize, try { // set statistics OperatorResolver cr = getContractResolver(p); - FileDataSource ordersSource = cr.getNode(ORDERS); - FileDataSource lineItemSource = cr.getNode(LINEITEM); - MapOperator mapper = cr.getNode(MAPPER_NAME); - JoinOperator joiner = cr.getNode(JOIN_NAME); + GenericDataSourceBase ordersSource = cr.getNode(ORDERS); + GenericDataSourceBase lineItemSource = cr.getNode(LINEITEM); + SingleInputOperator mapper = cr.getNode(MAPPER_NAME); + DualInputOperator joiner = cr.getNode(JOIN_NAME); setSourceStatistics(ordersSource, orderSize, 100f); setSourceStatistics(lineItemSource, lineitemSize, 140f); mapper.getCompilerHints().setAvgOutputRecordSize(16f); @@ -183,12 +186,12 @@ private void testQueryGeneric(Plan p, long orderSize, long lineitemSize, final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); // get the nodes from the final plan - final SinkPlanNode sink = or.getNode("Output"); - final SingleInputPlanNode reducer = or.getNode("AggLio"); + final SinkPlanNode sink = or.getNode(SINK); + final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; - final DualInputPlanNode join = or.getNode("JoinLiO"); - final SingleInputPlanNode filteringMapper = or.getNode("FilterO"); + final DualInputPlanNode join = or.getNode(JOIN_NAME); + final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME); checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); @@ -230,7 +233,7 @@ else if (checkRepartitionShipStrategies(join, reducer, combiner)) { // ------------------------------------------------------------------------ // Checks for special conditions // ------------------------------------------------------------------------ - + private void checkStandardStrategies(SingleInputPlanNode map, DualInputPlanNode join, SingleInputPlanNode combiner, SingleInputPlanNode reducer, SinkPlanNode sink) { @@ -239,7 +242,7 @@ private void checkStandardStrategies(SingleInputPlanNode map, DualInputPlanNode Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); // check the driver strategies that are always fix - Assert.assertEquals(DriverStrategy.COLLECTOR_MAP, map.getDriverStrategy()); + Assert.assertEquals(DriverStrategy.FLAT_MAP, map.getDriverStrategy()); Assert.assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy()); Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); if (combiner != null) { @@ -348,4 +351,73 @@ private boolean checkRepartitionMergeJoin(DualInputPlanNode join, SingleInputPla return false; } } + + public static Plan getTPCH3Plan() { + // prepare the test environment + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + TCPH3(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE}); + } catch (OptimizerPlanEnvironment.ProgramAbortException pae) { + // all good. + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("TCPH3 failed with an exception"); + } + return env.getPlan(); + } + + public static void TCPH3(String[] args) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(Integer.parseInt(args[0])); + + //order id, order status, order data, order prio, ship prio + DataSet> orders + = env.readCsvFile(args[1]) + .fieldDelimiter("|").lineDelimiter("\n") + .includeFields("101011001").types(Long.class, String.class, String.class, String.class, Integer.class) + .name(ORDERS); + + //order id, extended price + DataSet> lineItems + = env.readCsvFile(args[2]) + .fieldDelimiter("|").lineDelimiter("\n") + .includeFields("100001").types(Long.class, Double.class) + .name(LINEITEM); + + DataSet> filterO = orders.flatMap(new FilterO()).name(MAPPER_NAME); + + DataSet> joinLiO = filterO.join(lineItems).where(0).equalTo(0).with(new JoinLiO()).name(JOIN_NAME); + + DataSet> aggLiO = joinLiO.groupBy(0, 1).reduceGroup(new AggLiO()).name(REDUCE_NAME); + + aggLiO.writeAsCsv(args[3], "\n", "|").name(SINK); + + env.execute(); + } + + @ForwardedFields("f0; f4->f1") + public static class FilterO implements FlatMapFunction, Tuple2> { + @Override + public void flatMap(Tuple5 value, Collector> out) throws Exception { + // not going to be executed + } + } + + @ForwardedFieldsFirst("f0; f1") + public static class JoinLiO implements FlatJoinFunction, Tuple2, Tuple3> { + @Override + public void join(Tuple2 first, Tuple2 second, Collector> out) throws Exception { + // not going to be executed + } + } + + @ForwardedFields("f0; f1") + @Combinable + public static class AggLiO extends RichGroupReduceFunction, Tuple3> { + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + // not going to be executed + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java index ce713832f615f..576717f77d8c1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java @@ -22,16 +22,15 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.distributions.SimpleDistribution; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; +import org.apache.flink.api.common.operators.base.FileDataSourceBase; import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.optimizer.plan.Channel; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.SingleInputPlanNode; @@ -40,17 +39,13 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.test.recordJobs.wordcount.WordCount; -import org.apache.flink.test.recordJobs.wordcount.WordCount.CountWords; -import org.apache.flink.test.recordJobs.wordcount.WordCount.TokenizeLine; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.StringValue; import org.junit.Assert; import org.junit.Test; -@SuppressWarnings("deprecation") public class WordCountCompilerTest extends CompilerTestBase { - + + private static final long serialVersionUID = 8988304231385358228L; + /** * This method tests the simple word count. */ @@ -61,124 +56,64 @@ public void testWordCount() { } private void checkWordCount(boolean estimates) { - try { - WordCount wc = new WordCount(); - ExecutionConfig ec = new ExecutionConfig(); - Plan p = wc.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, OUT_FILE); - p.setExecutionConfig(ec); - OptimizedPlan plan; - if (estimates) { - FileDataSource source = getContractResolver(p).getNode("Input Lines"); - setSourceStatistics(source, 1024*1024*1024*1024L, 24f); - plan = compileWithStats(p); - } else { - plan = compileNoStats(p); - } - - // get the optimizer plan nodes - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(plan); - SinkPlanNode sink = resolver.getNode("Word Counts"); - SingleInputPlanNode reducer = resolver.getNode("Count Words"); - SingleInputPlanNode mapper = resolver.getNode("Tokenize Lines"); - - // verify the strategies - Assert.assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - Channel c = reducer.getInput(); - Assert.assertEquals(LocalStrategy.COMBININGSORT, c.getLocalStrategy()); - FieldList l = new FieldList(0); - Assert.assertEquals(l, c.getShipStrategyKeys()); - Assert.assertEquals(l, c.getLocalStrategyKeys()); - Assert.assertTrue(Arrays.equals(c.getLocalStrategySortOrder(), reducer.getSortOrders(0))); - - // check the combiner - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); - Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - Assert.assertEquals(l, combiner.getKeys(0)); - Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(DEFAULT_PARALLELISM); + + // get input data + DataSet lines = env.readTextFile(IN_FILE).name("Input Lines"); + + lines + // dummy map + .map(new MapFunction>() { + private static final long serialVersionUID = -3952739820618875030L; + @Override + public Tuple2 map(String v) throws Exception { + return new Tuple2<>(v, 1); + } + }).name("Tokenize Lines") + // count + .groupBy(0).sum(1).name("Count Words") + // discard + .output(new DiscardingOutputFormat>()).name("Word Counts"); + + // get the plan and compile it + Plan p = env.createProgramPlan(); + p.setExecutionConfig(new ExecutionConfig()); + + OptimizedPlan plan; + if (estimates) { + GenericDataSourceBase source = getContractResolver(p).getNode("Input Lines"); + setSourceStatistics(source, 1024*1024*1024*1024L, 24f); + plan = compileWithStats(p); + } else { + plan = compileNoStats(p); } - } - - /** - * This method tests that with word count and a range partitioned sink, the range partitioner is pushed down. - */ - @Test - public void testWordCountWithSortedSink() { - checkWordCountWithSortedSink(true); - checkWordCountWithSortedSink(false); - } - - private void checkWordCountWithSortedSink(boolean estimates) { - try { - FileDataSource sourceNode = new FileDataSource(new TextInputFormat(), IN_FILE, "Input Lines"); - MapOperator mapNode = MapOperator.builder(new TokenizeLine()) - .input(sourceNode) - .name("Tokenize Lines") - .build(); - ReduceOperator reduceNode = ReduceOperator.builder(new CountWords(), StringValue.class, 0) - .input(mapNode) - .name("Count Words") - .build(); - FileDataSink out = new FileDataSink(new CsvOutputFormat(), OUT_FILE, reduceNode, "Word Counts"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .lenient(true) - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Ordering ordering = new Ordering(0, StringValue.class, Order.DESCENDING); - out.setGlobalOrder(ordering, new SimpleDistribution(new StringValue[] {new StringValue("N")})); - ExecutionConfig ec = new ExecutionConfig(); - Plan p = new Plan(out, "WordCount Example"); - p.setDefaultParallelism(DEFAULT_PARALLELISM); - p.setExecutionConfig(ec); - - OptimizedPlan plan; - if (estimates) { - setSourceStatistics(sourceNode, 1024*1024*1024*1024L, 24f); - plan = compileWithStats(p); - } else { - plan = compileNoStats(p); - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(plan); - SinkPlanNode sink = resolver.getNode("Word Counts"); - SingleInputPlanNode reducer = resolver.getNode("Count Words"); - SingleInputPlanNode mapper = resolver.getNode("Tokenize Lines"); - - Assert.assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.PARTITION_RANGE, reducer.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - Channel c = reducer.getInput(); - Assert.assertEquals(LocalStrategy.COMBININGSORT, c.getLocalStrategy()); - FieldList l = new FieldList(0); - Assert.assertEquals(l, c.getShipStrategyKeys()); - Assert.assertEquals(l, c.getLocalStrategyKeys()); - - // check that the sort orders are descending - Assert.assertFalse(c.getShipStrategySortOrder()[0]); - Assert.assertFalse(c.getLocalStrategySortOrder()[0]); + // get the optimizer plan nodes + OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(plan); + SinkPlanNode sink = resolver.getNode("Word Counts"); + SingleInputPlanNode reducer = resolver.getNode("Count Words"); + SingleInputPlanNode mapper = resolver.getNode("Tokenize Lines"); + + // verify the strategies + Assert.assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); + Assert.assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); + Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); + + Channel c = reducer.getInput(); + Assert.assertEquals(LocalStrategy.COMBININGSORT, c.getLocalStrategy()); + FieldList l = new FieldList(0); + Assert.assertEquals(l, c.getShipStrategyKeys()); + Assert.assertEquals(l, c.getLocalStrategyKeys()); + Assert.assertTrue(Arrays.equals(c.getLocalStrategySortOrder(), reducer.getSortOrders(0))); + + // check the combiner + SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); + Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); + Assert.assertEquals(l, combiner.getKeys(0)); + Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - // check the combiner - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); - Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - Assert.assertEquals(l, combiner.getKeys(0)); - Assert.assertEquals(l, combiner.getKeys(1)); - Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java index 99402a5545c7b..e134c7abff3db 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java @@ -20,7 +20,19 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.operators.util.FieldList; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.OptimizerPlanEnvironment.ProgramAbortException; +import org.apache.flink.client.program.PreviewPlanEnvironment; import org.apache.flink.optimizer.dag.TempMode; import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.OptimizedPlan; @@ -33,13 +45,14 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.test.recordJobs.graph.ConnectedComponentsWithCoGroup; +import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; /** * */ +@SuppressWarnings("serial") public class ConnectedComponentsCoGroupTest extends CompilerTestBase { private static final String VERTEX_SOURCE = "Vertices"; @@ -59,10 +72,7 @@ public class ConnectedComponentsCoGroupTest extends CompilerTestBase { @Test public void testWorksetConnectedComponents() { - ConnectedComponentsWithCoGroup cc = new ConnectedComponentsWithCoGroup(); - - Plan plan = cc.getPlan(String.valueOf(DEFAULT_PARALLELISM), - IN_FILE, IN_FILE, OUT_FILE, String.valueOf(100)); + Plan plan = getConnectedComponentsCoGroupPlan(); plan.setExecutionConfig(new ExecutionConfig()); OptimizedPlan optPlan = compileNoStats(plan); OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); @@ -134,4 +144,68 @@ public void testWorksetConnectedComponents() { JobGraphGenerator jgg = new JobGraphGenerator(); jgg.compileJobGraph(optPlan); } + + public static Plan getConnectedComponentsCoGroupPlan() { + // prepare the test environment + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + ConnectedComponentsWithCoGroup(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE, "100"}); + } catch (ProgramAbortException pae) { + // all good. + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("ConnectedComponentsWithCoGroup failed with an exception"); + } + return env.getPlan(); + } + + public static void ConnectedComponentsWithCoGroup(String[] args) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(Integer.parseInt(args[0])); + + DataSet> initialVertices = env.readCsvFile(args[1]).types(Long.class).name(VERTEX_SOURCE); + + DataSet> edges = env.readCsvFile(args[2]).types(Long.class, Long.class).name(EDGES_SOURCE); + + DataSet> verticesWithId = initialVertices.flatMap(new DummyMapFunction()); + + DeltaIteration, Tuple2> iteration + = verticesWithId.iterateDelta(verticesWithId, Integer.parseInt(args[4]), 0).name(ITERATION_NAME); + + DataSet> joinWithNeighbors = iteration.getWorkset().join(edges) + .where(0).equalTo(0) + .with(new DummyJoinFunction()).name(JOIN_NEIGHBORS_MATCH); + + DataSet> minAndUpdate = joinWithNeighbors.coGroup(iteration.getSolutionSet()) + .where(0).equalTo(0) + .with(new DummyCoGroupFunction()).name(MIN_ID_AND_UPDATE); + + iteration.closeWith(minAndUpdate, minAndUpdate).writeAsCsv(args[3]).name(SINK); + + env.execute(); + } + + public static class DummyMapFunction implements FlatMapFunction, Tuple2> { + @Override + public void flatMap(Tuple1 value, Collector> out) throws Exception { + // won't be executed + } + } + + public static class DummyJoinFunction implements FlatJoinFunction, Tuple2, Tuple2> { + @Override + public void join(Tuple2 first, Tuple2 second, Collector> out) throws Exception { + // won't be executed + } + } + + @ForwardedFieldsFirst("f0->f0") + @ForwardedFieldsSecond("f0->f0") + public static class DummyCoGroupFunction implements CoGroupFunction, Tuple2, Tuple2> { + @Override + public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { + // won't be executed + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java deleted file mode 100644 index 378527073b603..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/IterativeKMeansTest.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.optimizer.iterations; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.util.Arrays; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.optimizer.util.OperatorResolver; -import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast; -import org.junit.Assert; -import org.junit.Test; - -@SuppressWarnings("deprecation") -public class IterativeKMeansTest extends CompilerTestBase { - - private static final String DATAPOINTS = "Data Points"; - private static final String CENTERS = "Centers"; - - private static final String MAPPER_NAME = "Find Nearest Centers"; - private static final String REDUCER_NAME = "Recompute Center Positions"; - - private static final String ITERATION_NAME = "k-means loop"; - - private static final String SINK = "New Center Positions"; - - private final FieldList set0 = new FieldList(0); - - // -------------------------------------------------------------------------------------------- - // K-Means (Bulk Iteration) - // -------------------------------------------------------------------------------------------- - - @Test - public void testCompileKMeansSingleStepWithStats() { - - KMeansBroadcast kmi = new KMeansBroadcast(); - Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); - p.setExecutionConfig(new ExecutionConfig()); - // set the statistics - OperatorResolver cr = getContractResolver(p); - FileDataSource pointsSource = cr.getNode(DATAPOINTS); - FileDataSource centersSource = cr.getNode(CENTERS); - setSourceStatistics(pointsSource, 100l*1024*1024*1024, 32f); - setSourceStatistics(centersSource, 1024*1024, 32f); - - OptimizedPlan plan = compileWithStats(p); - checkPlan(plan); - - new JobGraphGenerator().compileJobGraph(plan); - } - - @Test - public void testCompileKMeansSingleStepWithOutStats() { - - KMeansBroadcast kmi = new KMeansBroadcast(); - Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); - p.setExecutionConfig(new ExecutionConfig()); - OptimizedPlan plan = compileNoStats(p); - checkPlan(plan); - - new JobGraphGenerator().compileJobGraph(plan); - } - - private void checkPlan(OptimizedPlan plan) { - - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); - - final SinkPlanNode sink = or.getNode(SINK); - final SingleInputPlanNode reducer = or.getNode(REDUCER_NAME); - final SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); - final SingleInputPlanNode mapper = or.getNode(MAPPER_NAME); - - final BulkIterationPlanNode iter = or.getNode(ITERATION_NAME); - - // -------------------- outside the loop ----------------------- - - // check the sink - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); - - // check the iteration - assertEquals(ShipStrategyType.FORWARD, iter.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, iter.getInput().getLocalStrategy()); - - - // -------------------- inside the loop ----------------------- - - // check the mapper - assertEquals(1, mapper.getBroadcastInputs().size()); - assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.BROADCAST, mapper.getBroadcastInputs().get(0).getShipStrategy()); - assertFalse(mapper.getInput().isOnDynamicPath()); - assertTrue(mapper.getBroadcastInputs().get(0).isOnDynamicPath()); - assertTrue(mapper.getInput().getTempMode().isCached()); - - assertEquals(LocalStrategy.NONE, mapper.getInput().getLocalStrategy()); - assertEquals(LocalStrategy.NONE, mapper.getBroadcastInputs().get(0).getLocalStrategy()); - - assertEquals(DriverStrategy.COLLECTOR_MAP, mapper.getDriverStrategy()); - - assertNull(mapper.getInput().getLocalStrategyKeys()); - assertNull(mapper.getInput().getLocalStrategySortOrder()); - assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategyKeys()); - assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategySortOrder()); - - // check the combiner - Assert.assertNotNull(combiner); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - assertTrue(combiner.getInput().isOnDynamicPath()); - - assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - assertNull(combiner.getInput().getLocalStrategyKeys()); - assertNull(combiner.getInput().getLocalStrategySortOrder()); - assertEquals(set0, combiner.getKeys(0)); - assertEquals(set0, combiner.getKeys(1)); - - // check the reducer - assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - assertTrue(reducer.getInput().isOnDynamicPath()); - assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy()); - assertEquals(set0, reducer.getKeys(0)); - assertEquals(set0, reducer.getInput().getLocalStrategyKeys()); - assertTrue(Arrays.equals(reducer.getInput().getLocalStrategySortOrder(), reducer.getSortOrders(0))); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java index 24d94167844d8..d186cbbfcbf37 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java @@ -25,13 +25,12 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.examples.java.clustering.KMeans; +import org.apache.flink.examples.java.graph.ConnectedComponents; +import org.apache.flink.examples.java.graph.PageRankBasic; +import org.apache.flink.examples.java.relational.TPCHQuery3; +import org.apache.flink.examples.java.relational.WebLogAnalysis; +import org.apache.flink.examples.java.wordcount.WordCount; import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.test.recordJobs.graph.DeltaPageRankWithInitialDeltas; -import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast; -import org.apache.flink.test.recordJobs.kmeans.KMeansSingleStep; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3; -import org.apache.flink.test.recordJobs.relational.WebLogAnalysis; -import org.apache.flink.test.recordJobs.wordcount.WordCount; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParseException; import org.codehaus.jackson.JsonParser; @@ -45,17 +44,34 @@ public class DumpCompiledPlanTest extends CompilerTestBase { @Test public void dumpWordCount() { - dump(new WordCount().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, OUT_FILE)); + // prepare the test environment + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + WordCount.main(new String[] {IN_FILE, OUT_FILE}); + } catch(OptimizerPlanEnvironment.ProgramAbortException pae) { + // all good. + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("WordCount failed with an exception"); + } + dump(env.getPlan()); } @Test public void dumpTPCH3() { - dump(new TPCHQuery3().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE)); - } - - @Test - public void dumpKMeans() { - dump(new KMeansSingleStep().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE)); + // prepare the test environment + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + TPCHQuery3.main(new String[] {IN_FILE, IN_FILE, OUT_FILE, "123"}); + } catch(OptimizerPlanEnvironment.ProgramAbortException pae) { + // all good. + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("TPCH3 failed with an exception"); + } + dump(env.getPlan()); } @Test @@ -64,7 +80,6 @@ public void dumpIterativeKMeans() { PreviewPlanEnvironment env = new PreviewPlanEnvironment(); env.setAsContext(); try { - // out) throws Exception { - out.collect(new Record(value1.getField(1, StringValue.class), value2.getField(1, IntValue.class))); - } - } - - public static class SerializableIteratorTest implements Iterator>, Serializable { - - private static final long serialVersionUID = 1L; - - private final String[] s = WordCountData.COUNTS.split("\n"); - - private int pos = 0; - - public void remove() { - throw new UnsupportedOperationException(); - } - - public List next() { - List tmp = new ArrayList(); - tmp.add(pos); - tmp.add(s[pos++].split(" ")[0]); - return tmp; - } - - public boolean hasNext() { - return pos < s.length; - } - } - - public Plan getPlan(int numSubTasks, String output) { - - List tmp = new ArrayList(); - int pos = 0; - for (String s : WordCountData.COUNTS.split("\n")) { - List tmpInner = new ArrayList(); - tmpInner.add(pos++); - tmpInner.add(Integer.parseInt(s.split(" ")[1])); - tmp.add(tmpInner); - } - - // test serializable iterator input, the input record is {id, word} - CollectionDataSource source = new CollectionDataSource(new SerializableIteratorTest(), "test_iterator"); - // test collection input, the input record is {id, count} - CollectionDataSource source2 = new CollectionDataSource(tmp, "test_collection"); - - JoinOperator join = JoinOperator.builder(Join.class, IntValue.class, 0, 0) - .input1(source).input2(source2).build(); - - FileDataSink out = new FileDataSink(new CsvOutputFormat(), output, join, "Collection Join"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "CollectionDataSource"); - plan.setExecutionConfig(new ExecutionConfig()); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - return getPlan(parallelism, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - // Test results - compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionValidationTest.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionValidationTest.java deleted file mode 100644 index 1ce7a73a38840..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionValidationTest.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.java.record.operators.CollectionDataSource; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Test the input field validation of CollectionDataSource - */ -@SuppressWarnings("deprecation") -public class CollectionValidationTest { - - @Test - public void TestArrayInputValidation() throws Exception { - - /* - * valid array input - */ - try { - new CollectionDataSource("test_1d_valid_array", "a", "b", "c"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - - try { - new CollectionDataSource("test_2d_valid_array", new Object[][] { { 1, "a" }, - { 2, "b" }, { 3, "c" } }); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - - /* - * invalid array input - */ - try { - new CollectionDataSource("test_1d_invalid_array", 1, "b", "c"); - Assert.fail("input type is different"); - } catch (Exception e) { - } - - try { - new CollectionDataSource("test_2d_invalid_array", new Object[][] { - { 1, "a" }, { 2, "b" }, { 3, 4 } }); - Assert.fail("input type is different"); - } catch (Exception e) { - } - } - - @Test - public void TestCollectionInputValidation() throws Exception { - /* - * valid collection input - */ - try { - List tmp = new ArrayList(); - for (int i = 0; i < 100; i++) { - tmp.add(i); - } - new CollectionDataSource(tmp, "test_valid_collection"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - - try { - List tmp = new ArrayList(); - for (int i = 0; i < 100; i++) { - List inner = new ArrayList(); - inner.add(i); - inner.add('a' + i); - tmp.add(inner); - } - new CollectionDataSource(tmp, "test_valid_double_collection"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - - /* - * invalid collection input - */ - try { - List tmp = new ArrayList(); - for (int i = 0; i < 100; i++) { - tmp.add(i); - } - tmp.add("a"); - new CollectionDataSource(tmp, "test_invalid_collection"); - Assert.fail("input type is different"); - } catch (Exception e) { - } - - try { - List tmp = new ArrayList(); - for (int i = 0; i < 100; i++) { - List inner = new ArrayList(); - inner.add(i); - inner.add('a' + i); - tmp.add(inner); - } - List inner = new ArrayList(); - inner.add('a'); - inner.add('a'); - tmp.add(inner); - new CollectionDataSource(tmp, "test_invalid_double_collection"); - Assert.fail("input type is different"); - } catch (Exception e) { - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java deleted file mode 100644 index 613de78b787eb..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.Collection; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.graph.ComputeEdgeDegrees; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class ComputeEdgeDegreesITCase extends RecordAPITestBase { - - protected String edgesPath = null; - protected String resultPath = null; - - private static final String EDGES = "1,2\n1,3\n1,4\n1,5\n2,3\n2,5\n3,4\n3,7\n4,3\n6,5\n8,3\n7,8\n5,6\n"; - private static final String EXPECTED = "1,4|2,3\n1,4|3,5\n1,4|4,2\n1,4|5,3\n2,3|3,5\n2,3|5,3\n3,5|4,2\n3,5|7,2\n5,3|6,1\n3,5|8,2\n7,2|8,2\n"; - - public ComputeEdgeDegreesITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - edgesPath = createTempFile("edges.txt", EDGES); - resultPath = getTempDirPath("edgesWithDegrees"); - } - - @Override - protected Plan getTestJob() { - ComputeEdgeDegrees computeDegrees = new ComputeEdgeDegrees(); - return computeDegrees.getPlan(String.valueOf(config.getInteger("NumSubtasks", 4)), - edgesPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("NumSubtasks", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java deleted file mode 100644 index 6b0f01cc47f30..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.Collection; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase { - - private static final String EDGES_WITH_DEGREES = "1,4|2,3\n1,4|3,5\n1,4|4,2\n1,4|5,3\n2,3|3,5\n2,3|5,3\n3,5|4,2\n3,5|7,2\n5,3|6,1\n3,5|8,2\n7,2|8,2\n"; - private static final String EXPECTED = "2,1,3\n4,1,3\n2,1,5\n7,3,8\n"; - - protected String edgesPath; - protected String resultPath; - - public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - edgesPath = createTempFile("edgesWithDegrees.txt", EDGES_WITH_DEGREES); - resultPath = getTempDirPath("triangles"); - } - - - @Override - protected Plan getTestJob() { - EnumTrianglesOnEdgesWithDegrees enumTriangles = new EnumTrianglesOnEdgesWithDegrees(); - return enumTriangles.getPlan( - String.valueOf(config.getInteger("NumSubtasks", 4)), - edgesPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("NumSubtasks", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java deleted file mode 100644 index b96a31c742b30..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesRdfFoaf; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.util.Collection; - -@RunWith(Parameterized.class) -public class EnumTrianglesRDFITCase extends RecordAPITestBase { - - String edgesPath = null; - String resultPath = null; - - private static final String EDGES = " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n"; - - private static final String EXPECTED = " \n" + " \n" + " \n" + - " \n" + " \n"; - - public EnumTrianglesRDFITCase(Configuration config) { - super(config); - } - - @Override - protected void preSubmit() throws Exception { - edgesPath = createTempFile("edges.txt", EDGES); - resultPath = getTempDirPath("triangles"); - } - - @Override - protected Plan getTestJob() { - EnumTrianglesRdfFoaf enumTriangles = new EnumTrianglesRdfFoaf(); - return enumTriangles.getPlan( - String.valueOf(config.getInteger("NumSubtasks", parallelism)), edgesPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("NumSubtasks", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java deleted file mode 100644 index 15486cb5fbdd1..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Random; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.distributions.UniformIntegerDistribution; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; - -@SuppressWarnings("deprecation") -public class GlobalSortingITCase extends RecordAPITestBase { - - private static final int NUM_RECORDS = 100000; - - private String recordsPath; - private String resultPath; - - private String sortedRecords; - - public GlobalSortingITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - - ArrayList records = new ArrayList(); - - //Generate records - Random rnd = new Random(1988); - - StringBuilder sb = new StringBuilder(NUM_RECORDS * 7); - - for (int i = 0; i < NUM_RECORDS; i++) { - int number = rnd.nextInt(); - - records.add(number); - - sb.append(number); - sb.append('\n'); - } - - recordsPath = createTempFile("records", sb.toString()); - resultPath = getTempDirPath("result"); - - - // create the expected sorted result - Collections.sort(records); - sb.setLength(0); - - for (Integer i : records) { - sb.append(i.intValue()); - sb.append('\n'); - } - - this.sortedRecords = sb.toString(); - } - - @Override - protected Plan getTestJob() { - GlobalSort globalSort = new GlobalSort(); - return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - // Test results - compareResultsByLinesInMemoryWithStrictOrder(this.sortedRecords, this.resultPath); - } - - - private static class GlobalSort implements Program { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - // parse program parameters - int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String recordsPath = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - FileDataSource source = new FileDataSource(CsvInputFormat.class, recordsPath); - source.setParallelism(numSubtasks); - CsvInputFormat.configureRecordFormat(source) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(IntValue.class, 0); - - FileDataSink sink = - new FileDataSink(CsvOutputFormat.class, output); - sink.setParallelism(numSubtasks); - CsvOutputFormat.configureRecordFormat(sink) - .recordDelimiter('\n') - .fieldDelimiter('|') - .lenient(true) - .field(IntValue.class, 0); - - sink.setGlobalOrder(new Ordering(0, IntValue.class, Order.ASCENDING), new UniformIntegerDistribution(Integer.MIN_VALUE, Integer.MAX_VALUE)); - sink.setInput(source); - - return new Plan(sink); - } - - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java deleted file mode 100644 index e79b8078eda2c..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java +++ /dev/null @@ -1,219 +0,0 @@ -/* - * 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.recordJobTests; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Random; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Key; - -@SuppressWarnings("deprecation") -public class GlobalSortingMixedOrderITCase extends RecordAPITestBase { - - private static final int NUM_RECORDS = 100000; - - private static final int RANGE_I1 = 100; - private static final int RANGE_I2 = 20; - private static final int RANGE_I3 = 20; - - private String recordsPath; - private String resultPath; - - private String sortedRecords; - - public GlobalSortingMixedOrderITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - - ArrayList records = new ArrayList(); - - //Generate records - final Random rnd = new Random(1988); - final StringBuilder sb = new StringBuilder(NUM_RECORDS * 7); - - - for (int j = 0; j < NUM_RECORDS; j++) { - TripleInt val = new TripleInt(rnd.nextInt(RANGE_I1), rnd.nextInt(RANGE_I2), rnd.nextInt(RANGE_I3)); - records.add(val); - sb.append(val); - sb.append('\n'); - } - - - this.recordsPath = createTempFile("records", sb.toString()); - this.resultPath = getTempDirPath("result"); - - // create the sorted result; - Collections.sort(records); - - sb.setLength(0); - for (TripleInt val : records) { - sb.append(val); - sb.append('\n'); - } - this.sortedRecords = sb.toString(); - } - - @Override - protected Plan getTestJob() { - GlobalSort globalSort = new GlobalSort(); - return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - // Test results - compareResultsByLinesInMemoryWithStrictOrder(this.sortedRecords, this.resultPath); - } - - - public static class TripleIntDistribution implements DataDistribution { - - private static final long serialVersionUID = 1L; - - private boolean ascendingI1, ascendingI2, ascendingI3; - - public TripleIntDistribution(Order orderI1, Order orderI2, Order orderI3) { - this.ascendingI1 = orderI1 != Order.DESCENDING; - this.ascendingI2 = orderI2 != Order.DESCENDING; - this.ascendingI3 = orderI3 != Order.DESCENDING; - } - - public TripleIntDistribution() {} - - @Override - public void write(DataOutputView out) throws IOException { - out.writeBoolean(this.ascendingI1); - out.writeBoolean(this.ascendingI2); - out.writeBoolean(this.ascendingI3); - } - - @Override - public void read(DataInputView in) throws IOException { - this.ascendingI1 = in.readBoolean(); - this.ascendingI2 = in.readBoolean(); - this.ascendingI3 = in.readBoolean(); - } - - @Override - public Key[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - - final float bucketWidth = ((float) RANGE_I1) / totalNumBuckets; - int boundVal = (int) ((bucketNum + 1) * bucketWidth); - if (!this.ascendingI1) { - boundVal = RANGE_I1 - boundVal; - } - - return new Key[] { new IntValue(boundVal), new IntValue(RANGE_I2), new IntValue(RANGE_I3) }; - } - - @Override - public int getNumberOfFields() { - return 3; - } - - } - - private static class GlobalSort implements Program { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - // parse program parameters - final int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String recordsPath = (args.length > 1 ? args[1] : ""); - final String output = (args.length > 2 ? args[2] : ""); - - @SuppressWarnings("unchecked") - FileDataSource source = new FileDataSource(new CsvInputFormat(',', IntValue.class, IntValue.class, IntValue.class), recordsPath); - - FileDataSink sink = new FileDataSink(CsvOutputFormat.class, output); - CsvOutputFormat.configureRecordFormat(sink) - .recordDelimiter('\n') - .fieldDelimiter(',') - .lenient(true) - .field(IntValue.class, 0) - .field(IntValue.class, 1) - .field(IntValue.class, 2); - - sink.setGlobalOrder( - new Ordering(0, IntValue.class, Order.DESCENDING) - .appendOrdering(1, IntValue.class, Order.ASCENDING) - .appendOrdering(2, IntValue.class, Order.DESCENDING), - new TripleIntDistribution(Order.DESCENDING, Order.ASCENDING, Order.DESCENDING)); - sink.setInput(source); - - Plan p = new Plan(sink); - p.setDefaultParallelism(numSubtasks); - return p; - } - } - - /** - * Three integers sorting descending, ascending, descending. - */ - private static final class TripleInt implements Comparable { - - private final int i1, i2, i3; - - - private TripleInt(int i1, int i2, int i3) { - this.i1 = i1; - this.i2 = i2; - this.i3 = i3; - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(32); - bld.append(this.i1); - bld.append(','); - bld.append(this.i2); - bld.append(','); - bld.append(this.i3); - return bld.toString(); - } - - @Override - public int compareTo(TripleInt o) { - return this.i1 < o.i1 ? 1 : this.i1 > o.i1 ? -1 : - this.i2 < o.i2 ? -1 : this.i2 > o.i2 ? 1 : - this.i3 < o.i3 ? 1 : this.i3 > o.i3 ? -1 : 0; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java deleted file mode 100644 index c1ad83dc7a33f..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.recordJobTests; - -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -@SuppressWarnings("deprecation") -public class GroupOrderReduceITCase extends RecordAPITestBase { - - private static final String INPUT = "1,3\n" + "2,1\n" + "5,1\n" + "3,1\n" + "1,8\n" + "1,9\n" + - "1,2\n" + "2,3\n" + "7,1\n" + "4,2\n" + "2,7\n" + "2,8\n" + - "1,1\n" + "2,7\n" + "5,4\n" + "4,3\n" + "3,6\n" + "3,7\n" + - "1,3\n" + "2,4\n" + "7,1\n" + "5,3\n" + "4,5\n" + "4,6\n" + - "1,4\n" + "3,9\n" + "8,5\n" + "5,3\n" + "5,4\n" + "5,5\n" + - "1,7\n" + "3,9\n" + "9,3\n" + "6,2\n" + "6,3\n" + "6,4\n" + - "1,8\n" + "3,8\n" + "8,7\n" + "6,2\n" + "7,2\n" + "7,3\n" + - "1,1\n" + "3,7\n" + "9,2\n" + "7,1\n" + "8,1\n" + "8,2\n" + - "1,2\n" + "2,6\n" + "8,7\n" + "7,1\n" + "9,1\n" + "9,1\n" + - "1,1\n" + "2,5\n" + "9,5\n" + "8,2\n" + "10,2\n" + "10,1\n" + - "1,1\n" + "2,6\n" + "2,7\n" + "8,3\n" + "11,3\n" + "11,2\n" + - "1,2\n" + "2,7\n" + "4,2\n" + "9,4\n" + "12,8\n" + "12,3\n" + - "1,2\n" + "4,8\n" + "1,7\n" + "9,5\n" + "13,9\n" + "13,4\n" + - "1,3\n" + "4,2\n" + "3,2\n" + "9,6\n" + "14,7\n" + "14,5\n"; - - protected String textPath; - protected String resultPath; - - - public GroupOrderReduceITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - textPath = createTempFile("pairs.csv", INPUT); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - - int parallelism = this.config.getInteger("GroupOrderTest#NumSubtasks", 1); - - @SuppressWarnings("unchecked") - CsvInputFormat format = new CsvInputFormat(',', IntValue.class, IntValue.class); - FileDataSource source = new FileDataSource(format, this.textPath, "Source"); - - ReduceOperator reducer = ReduceOperator.builder(CheckingReducer.class) - .keyField(IntValue.class, 0) - .input(source) - .name("Ordered Reducer") - .build(); - reducer.setGroupOrder(new Ordering(1, IntValue.class, Order.ASCENDING)); - - FileDataSink sink = new FileDataSink(CsvOutputFormat.class, this.resultPath, reducer, "Sink"); - CsvOutputFormat.configureRecordFormat(sink) - .recordDelimiter('\n') - .fieldDelimiter(',') - .field(IntValue.class, 0) - .field(IntValue.class, 1); - - Plan p = new Plan(sink); - p.setDefaultParallelism(parallelism); - return p; - } - - @Override - protected void postSubmit() throws Exception { - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("GroupOrderTest#NumSubtasks", parallelism); - return toParameterList(config); - } - - public static final class CheckingReducer extends ReduceFunction implements Serializable { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - int lastValue = records.next().getField(1, IntValue.class).getValue(); - - while (records.hasNext()) { - int nextValue = records.next().getField(1, IntValue.class).getValue(); - - if (nextValue < lastValue) { - throw new Exception("Group Order is violated!"); - } - - lastValue = nextValue; - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/MergeOnlyJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/MergeOnlyJoinITCase.java deleted file mode 100644 index ae3b8b7367b3b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/MergeOnlyJoinITCase.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.relational.MergeOnlyJoin; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.util.ArrayList; -import java.util.Collection; - -@RunWith(Parameterized.class) -public class MergeOnlyJoinITCase extends RecordAPITestBase { - - private String input1Path = null; - private String input2Path = null; - private String resultPath = null; - - private final String INPUT1 = "1|9|\n" - + "2|8\n" - + "3|7\n" - + "5|5\n" - + "6|4\n" - + "7|3\n" - + "4|6\n" - + "8|2\n" - + "2|1\n"; - - private final String INPUT2 = "2|2|\n" - + "2|6|\n" - + "2|1|\n" - + "4|1|\n" - + "5|1|\n" - + "2|1|\n"; - - - private final String EXPECTED_RESULT = "2|8|2\n" - + "2|8|6\n" - + "2|8|1\n" - + "2|8|1\n" - + "2|1|2\n" - + "2|1|6\n" - + "2|1|1\n" - + "2|1|1\n" - + "4|6|1\n" - + "5|5|1\n"; - - public MergeOnlyJoinITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(4); - } - - @Override - protected void preSubmit() throws Exception { - input1Path = createTempFile("input1.txt", INPUT1); - input2Path = createTempFile("input2.txt", INPUT2); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - MergeOnlyJoin mergeOnlyJoin = new MergeOnlyJoin(); - return mergeOnlyJoin.getPlan( - String.valueOf(config.getInteger("MergeOnlyJoinTest#NoSubtasks", 1)), - input1Path, - input2Path, - resultPath, - String.valueOf(config.getInteger("MergeOnlyJoinTest#NoSubtasksInput2", 1))); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - ArrayList tConfigs = new ArrayList(); - - Configuration config = new Configuration(); - config.setInteger("MergeOnlyJoinTest#NoSubtasks", 3); - config.setInteger("MergeOnlyJoinTest#NoSubtasksInput2", 3); - tConfigs.add(config); - - config = new Configuration(); - config.setInteger("MergeOnlyJoinTest#NoSubtasks", 3); - config.setInteger("MergeOnlyJoinTest#NoSubtasksInput2", 4); - tConfigs.add(config); - - config = new Configuration(); - config.setInteger("MergeOnlyJoinTest#NoSubtasks", 3); - config.setInteger("MergeOnlyJoinTest#NoSubtasksInput2", 2); - tConfigs.add(config); - - return toParameterList(tConfigs); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java deleted file mode 100644 index 1a0cae2dc3b7e..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.Collection; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.graph.PairwiseSP; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class PairwiseSPITCase extends RecordAPITestBase { - - String rdfDataPath = null; - String resultPath = null; - - /* - private String paths = "A|C|7| |\n" + "A|D|6| |\n" + "B|A|1| |\n" + "B|D|2| |\n" + "C|B|3| |\n" + "C|E|10| |\n" - + "C|F|12| |\n" + "C|G|9| |\n" + "D|F|5| |\n" + "E|H|2| |\n" + "F|E|3| |\n" + "G|F|1| |\n" + "H|D|2| |\n" - + "H|E|4| |\n"; - */ - - private static final String RDF_DATA = " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n" + - " \n" + " \n"; - - private static final String EXPECTED = "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + - "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + - "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + "||1|0| |\n" + - "||1|0| |\n" + "||1|0| |\n" + "||2|1||\n" + "||2|1||\n" + - "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + - "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + - "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + "||2|1||\n" + - "||2|1||\n"; - - public PairwiseSPITCase(Configuration config) { - super(config); - } - - @Override - protected void preSubmit() throws Exception { - rdfDataPath = createTempFile("rdf_data.txt", RDF_DATA); - resultPath = getTempDirPath("ITER_1"); - } - - @Override - protected Plan getTestJob() { - PairwiseSP a2aSP = new PairwiseSP(); - return a2aSP.getPlan( - String.valueOf(config.getInteger("All2AllSPTest#NoSubtasks", parallelism)), - rdfDataPath, - resultPath, - "true"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("All2AllSPTest#NoSubtasks", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java deleted file mode 100644 index 349275cf7e687..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java +++ /dev/null @@ -1,211 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.relational.TPCHQuery10; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.util.Collection; - -/** - */ -@RunWith(Parameterized.class) -public class TPCHQuery10ITCase extends RecordAPITestBase { - - private final String CUSTOMERS = "36900|Customer#000036900|ppktIUalnJ quTLD1fWZTEMBQwoEUpmI|8|18-347-285-7152|2667.45|MACHINERY|ts. slyly special packages are al|\n" - + "36901|Customer#000036901|TBb1yDZcf 8Zepk7apFJ|13|23-644-998-4944|4809.84|AUTOMOBILE|nstructions sleep final, regular deposits. quick accounts sleep furiously after the final accounts; instructions wa|\n" - + "36902|Customer#000036902|nCUCadobbPGA0pzd1yEX3RE|3|13-301-654-8016|8905.80|AUTOMOBILE|le blithely final packages. pending, pending foxes impress qu|\n" - + "16252|Customer#000016252|Ha0SZbzPcuno,WTyMl1ipU0YtpeuR1|15|25-830-891-9338|7140.55|BUILDING|furiously unusual packages! theodolites haggle along the quickly speci|\n" - + "130057|Customer#000130057|jQDBlCU2IlHmzkDfcqgIHg2eLsN|9|19-938-862-4157|5009.55|FURNITURE| blithely regular packages. carefully bold accounts sle|\n" - + "78002|Customer#000078002|v7Jkg5XIqM|10|20-715-308-7926|4128.41|AUTOMOBILE|ly after the special deposits. careful packages|\n" - + "81763|Customer#000081763|mZtn4M5r0KIw4aooP BXF3ReR RUlPJcAb|8|18-425-613-5972|8368.23|MACHINERY|ronic frays. slyly pending pinto beans are furiously grouches. permanen|\n" - + "86116|Customer#000086116|63BSp8bODm1dImPJEPTRmsSa4GqNA1SeRqFgx|0|10-356-493-3518|3205.60|AUTOMOBILE| ironic ideas. quickly pending ideas sleep blith|\n"; - - private final String ORDERS = "1|36901|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|78002|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|123314|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|136777|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|44485|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|55624|F|58749.59|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|39136|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|130057|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n" - + "33|66958|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|61001|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|127588|O|253724.56|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|115252|O|68289.96|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|86116|F|206680.66|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|124828|O|82500.05|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|81763|O|341734.47|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|32113|F|39414.99|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|16252|P|110643.60|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|129200|F|103740.67|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n"; - - private final String LINEITEMS = "1|155190|7706|1|17|21168.23|0.04|0.02|R|R|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|\n" - + "1|67310|7311|2|36|45983.16|0.09|0.06|R|R|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |\n" - + "1|63700|3701|3|8|13309.60|0.10|0.02|R|R|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|\n" - + "1|2132|4633|4|28|28955.64|0.09|0.06|R|R|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|\n" - + "1|24027|1534|5|24|22824.48|0.10|0.04|R|R|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|\n" - + "1|15635|638|6|32|49620.16|0.07|0.02|R|R|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|\n" - + "2|106170|1191|1|38|44694.46|0.00|0.05|R|R|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|\n" - + "3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|\n" - + "3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|\n" - + "3|128449|3474|3|27|39890.88|0.06|0.07|R|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |\n" - + "3|29380|1883|4|2|2618.76|0.01|0.06|R|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|\n" - + "3|183095|650|5|28|32986.52|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|\n" - + "3|62143|9662|6|26|28733.64|0.10|0.02|R|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|\n" - + "4|88035|5560|1|30|30690.90|0.03|0.08|R|R|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|\n" - + "5|108570|8571|1|15|23678.55|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |\n" - + "5|123927|3928|2|26|50723.92|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|\n" - + "5|37531|35|3|50|73426.50|0.08|0.03|R|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|\n" - + "6|139636|2150|1|37|61998.31|0.08|0.03|R|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|\n" - + "7|182052|9607|1|12|13608.60|0.07|0.03|R|R|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|\n" - + "7|145243|7758|2|9|11594.16|0.08|0.08|R|R|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|\n" - + "7|94780|9799|3|46|81639.88|0.10|0.07|R|R|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|\n" - + "7|163073|3074|4|28|31809.96|0.03|0.04|R|R|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|\n" - + "7|151894|9440|5|38|73943.82|0.08|0.01|R|R|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|\n" - + "7|79251|1759|6|35|43058.75|0.06|0.03|R|R|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |\n" - + "7|157238|2269|7|5|6476.15|0.04|0.02|R|R|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|\n" - + "32|82704|7721|1|28|47227.60|0.05|0.08|R|R|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|\n" - + "32|197921|441|2|32|64605.44|0.02|0.00|R|R|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |\n" - + "32|44161|6666|3|2|2210.32|0.09|0.02|R|R|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|\n" - + "32|2743|7744|4|4|6582.96|0.09|0.03|R|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|\n" - + "32|85811|8320|5|44|79059.64|0.05|0.06|R|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|\n" - + "32|11615|4117|6|6|9159.66|0.04|0.03|R|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|\n" - + "33|61336|8855|1|31|40217.23|0.09|0.04|R|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|\n" - + "33|60519|5532|2|32|47344.32|0.02|0.05|R|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|\n" - + "33|137469|9983|3|5|7532.30|0.05|0.03|R|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|\n" - + "33|33918|3919|4|41|75928.31|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|\n" - + "34|88362|871|1|13|17554.68|0.00|0.07|R|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|\n" - + "34|89414|1923|2|22|30875.02|0.08|0.06|R|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|\n" - + "34|169544|4577|3|6|9681.24|0.02|0.06|R|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |\n" - + "35|450|2951|1|24|32410.80|0.02|0.00|R|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|\n" - + "35|161940|4457|2|34|68065.96|0.06|0.08|R|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|\n" - + "35|120896|8433|3|7|13418.23|0.06|0.04|R|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |\n" - + "35|85175|7684|4|25|29004.25|0.06|0.05|R|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|\n" - + "35|119917|4940|5|34|65854.94|0.08|0.06|R|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|\n" - + "35|30762|3266|6|28|47397.28|0.03|0.02|R|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |\n" - + "36|119767|9768|1|42|75043.92|0.09|0.00|R|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |\n" - + "37|22630|5133|1|40|62105.20|0.09|0.03|R|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|\n" - + "37|126782|1807|2|39|70542.42|0.05|0.02|R|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|\n" - + "37|12903|5405|3|43|78083.70|0.05|0.08|R|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|\n" - + "38|175839|874|1|44|84252.52|0.04|0.02|R|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|\n" - + "39|2320|9821|1|44|53782.08|0.09|0.06|R|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|\n" - + "39|186582|4137|2|26|43383.08|0.08|0.04|R|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|\n" - + "39|67831|5350|3|46|82746.18|0.06|0.08|R|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|\n" - + "39|20590|3093|4|32|48338.88|0.07|0.05|R|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|\n" - + "39|54519|9530|5|43|63360.93|0.01|0.01|R|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|\n" - + "39|94368|6878|6|40|54494.40|0.06|0.05|R|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|\n" - + "64|85951|5952|1|21|40675.95|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|\n" - + "65|59694|4705|1|26|42995.94|0.03|0.03|R|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|\n" - + "65|73815|8830|2|22|39353.82|0.00|0.05|R|O|1995-07-17|1995-06-04|1995-07-19|COLLECT COD|FOB| ideas. special, r|\n" - + "65|1388|3889|3|21|27076.98|0.09|0.07|R|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|\n" - + "66|115118|7630|1|31|35126.41|0.00|0.08|R|F|1994-02-19|1994-03-11|1994-02-20|TAKE BACK RETURN|RAIL|ut the unusual accounts sleep at the bo|\n" - + "66|173489|3490|2|41|64061.68|0.04|0.07|R|F|1994-02-21|1994-03-01|1994-03-18|COLLECT COD|AIR| regular de|\n" - + "67|21636|9143|1|4|6230.52|0.09|0.04|R|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|\n" - + "67|20193|5198|2|12|13358.28|0.09|0.05|R|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|\n" - + "67|173600|6118|3|5|8368.00|0.03|0.07|R|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |\n" - + "67|87514|7515|4|44|66066.44|0.08|0.06|R|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|\n" - + "67|40613|8126|5|23|35733.03|0.05|0.07|R|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|\n" - + "67|178306|824|6|29|40144.70|0.02|0.05|R|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |\n"; - - private final String NATIONS = "0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|\n" - + "1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|\n" - + "2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |\n" - + "3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|\n" - + "4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|\n" - + "5|ETHIOPIA|0|ven packages wake quickly. regu|\n" - + "6|FRANCE|3|refully final requests. regular, ironi|\n" - + "7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|\n" - + "8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|\n" - + "9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|\n" - + "10|IRAN|4|efully alongside of the slyly final dependencies. |\n" - + "11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|\n" - + "12|JAPAN|2|ously. final, express gifts cajole a|\n" - + "13|JORDAN|4|ic deposits are blithely about the carefully regular pa|\n" - + "14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|\n" - + "15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|\n" - + "16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|\n" - + "17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|\n" - + "18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|\n" - + "19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|\n" - + "20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|\n" - + "21|VIETNAM|2|hely enticingly express accounts. even, final |\n" - + "22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|\n" - + "23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|\n" - + "24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|\n"; - - private final String EXPECTED_RESULT = "36901|Customer#000036901|167183.2296|4809.84|JORDAN|TBb1yDZcf 8Zepk7apFJ|23-644-998-4944|nstructions sleep final, regular deposits. quick accounts sleep furiously after the final accounts; instructions wa|\n" - + "16252|Customer#000016252|105699.9336|7140.55|MOROCCO|Ha0SZbzPcuno,WTyMl1ipU0YtpeuR1|25-830-891-9338|furiously unusual packages! theodolites haggle along the quickly speci|\n" - + "130057|Customer#000130057|200081.3676|5009.55|INDONESIA|jQDBlCU2IlHmzkDfcqgIHg2eLsN|19-938-862-4157| blithely regular packages. carefully bold accounts sle|\n" - + "78002|Customer#000078002|44694.46|4128.41|IRAN|v7Jkg5XIqM|20-715-308-7926|ly after the special deposits. careful packages|\n" - + "81763|Customer#000081763|325542.7507|8368.23|INDIA|mZtn4M5r0KIw4aooP BXF3ReR RUlPJcAb|18-425-613-5972|ronic frays. slyly pending pinto beans are furiously grouches. permanen|\n" - + "86116|Customer#000086116|197710.546|3205.60|ALGERIA|63BSp8bODm1dImPJEPTRmsSa4GqNA1SeRqFgx|10-356-493-3518| ironic ideas. quickly pending ideas sleep blith|\n"; - - private String ordersPath; - - private String lineitemsPath; - - private String customersPath; - - private String nationsPath; - - private String resultPath; - - public TPCHQuery10ITCase(Configuration testConfig) { - super(testConfig); - } - - @Override - protected Plan getTestJob() { - TPCHQuery10 tpchq10 = new TPCHQuery10(); - return tpchq10.getPlan( - String.valueOf(config.getInteger("TPCHQuery10Test#NoSubtasks", 1)), - ordersPath, - lineitemsPath, - customersPath, - nationsPath, - resultPath); - } - - @Override - protected void preSubmit() throws Exception { - ordersPath = createTempFile("orders.txt", ORDERS); - lineitemsPath = createTempFile("line_items.txt", LINEITEMS); - customersPath = createTempFile("customers.txt", CUSTOMERS); - nationsPath = createTempFile("nations.txt", NATIONS); - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("TPCHQuery10Test#NoSubtasks", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java deleted file mode 100644 index 159c93180aa1b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.Collection; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3; -import org.apache.flink.test.util.RecordAPITestBase; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -// ----------------------------------------------------------------------------- -// --- NOTE --- -// -// This class contains test data generated by tools from by the -// Transaction Processing Council (TPC), specifically the TPC-H benchmark's -// data generator. -// -// Any form of use and redistribution must happen in accordance with the TPC-H -// Software License Agreement. -// -// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf -// ----------------------------------------------------------------------------- - -@RunWith(Parameterized.class) -public class TPCHQuery3ITCase extends RecordAPITestBase { - - protected String ordersPath = null; - protected String lineitemsPath = null; - protected String resultPath = null; - - public static final String ORDERS = "1|36901|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|78002|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|123314|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|136777|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|44485|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|55624|F|58749.59|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|39136|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|130057|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n" - + "33|66958|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|61001|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|127588|O|253724.56|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|115252|O|68289.96|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|86116|F|206680.66|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|124828|O|82500.05|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|81763|O|341734.47|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|32113|F|39414.99|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|16252|P|110643.60|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|129200|F|103740.67|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n"; - - public static final String LINEITEMS = "1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|\n" - + "1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |\n" - + "1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|\n" - + "1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|\n" - + "1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|\n" - + "1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|\n" - + "2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|\n" - + "3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|\n" - + "3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|\n" - + "3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |\n" - + "3|29380|1883|4|2|2618.76|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|\n" - + "3|183095|650|5|28|32986.52|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|\n" - + "3|62143|9662|6|26|28733.64|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|\n" - + "4|88035|5560|1|30|30690.90|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|\n" - + "5|108570|8571|1|15|23678.55|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |\n" - + "5|123927|3928|2|26|50723.92|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|\n" - + "5|37531|35|3|50|73426.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|\n" - + "6|139636|2150|1|37|61998.31|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|\n" - + "7|182052|9607|1|12|13608.60|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|\n" - + "7|145243|7758|2|9|11594.16|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|\n" - + "7|94780|9799|3|46|81639.88|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|\n" - + "7|163073|3074|4|28|31809.96|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|\n" - + "7|151894|9440|5|38|73943.82|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|\n" - + "7|79251|1759|6|35|43058.75|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |\n" - + "7|157238|2269|7|5|6476.15|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|\n" - + "32|82704|7721|1|28|47227.60|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|\n" - + "32|197921|441|2|32|64605.44|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |\n" - + "32|44161|6666|3|2|2210.32|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|\n" - + "32|2743|7744|4|4|6582.96|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|\n" - + "32|85811|8320|5|44|79059.64|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|\n" - + "32|11615|4117|6|6|9159.66|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|\n" - + "33|61336|8855|1|31|40217.23|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|\n" - + "33|60519|5532|2|32|47344.32|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|\n" - + "33|137469|9983|3|5|7532.30|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|\n" - + "33|33918|3919|4|41|75928.31|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|\n" - + "34|88362|871|1|13|17554.68|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|\n" - + "34|89414|1923|2|22|30875.02|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|\n" - + "34|169544|4577|3|6|9681.24|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |\n" - + "35|450|2951|1|24|32410.80|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|\n" - + "35|161940|4457|2|34|68065.96|0.06|0.08|N|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|\n" - + "35|120896|8433|3|7|13418.23|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |\n" - + "35|85175|7684|4|25|29004.25|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|\n" - + "35|119917|4940|5|34|65854.94|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|\n" - + "35|30762|3266|6|28|47397.28|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |\n" - + "36|119767|9768|1|42|75043.92|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |\n" - + "37|22630|5133|1|40|62105.20|0.09|0.03|A|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|\n" - + "37|126782|1807|2|39|70542.42|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|\n" - + "37|12903|5405|3|43|78083.70|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|\n" - + "38|175839|874|1|44|84252.52|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|\n" - + "39|2320|9821|1|44|53782.08|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|\n" - + "39|186582|4137|2|26|43383.08|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|\n" - + "39|67831|5350|3|46|82746.18|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|\n" - + "39|20590|3093|4|32|48338.88|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|\n" - + "39|54519|9530|5|43|63360.93|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|\n" - + "39|94368|6878|6|40|54494.40|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|\n" - + "64|85951|5952|1|21|40675.95|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|\n" - + "65|59694|4705|1|26|42995.94|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|\n" - + "65|73815|8830|2|22|39353.82|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-07-19|COLLECT COD|FOB| ideas. special, r|\n" - + "65|1388|3889|3|21|27076.98|0.09|0.07|N|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|\n" - + "66|115118|7630|1|31|35126.41|0.00|0.08|R|F|1994-02-19|1994-03-11|1994-02-20|TAKE BACK RETURN|RAIL|ut the unusual accounts sleep at the bo|\n" - + "66|173489|3490|2|41|64061.68|0.04|0.07|A|F|1994-02-21|1994-03-01|1994-03-18|COLLECT COD|AIR| regular de|\n" - + "67|21636|9143|1|4|6230.52|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|\n" - + "67|20193|5198|2|12|13358.28|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|\n" - + "67|173600|6118|3|5|8368.00|0.03|0.07|N|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |\n" - + "67|87514|7515|4|44|66066.44|0.08|0.06|N|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|\n" - + "67|40613|8126|5|23|35733.03|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|\n" - + "67|178306|824|6|29|40144.70|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |\n"; - - public static final String EXPECTED_RESULT = "5|0|147828.97\n" + "66|0|99188.09\n"; - - public TPCHQuery3ITCase(Configuration config) { - super(config); - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - ordersPath = createTempFile("orders", ORDERS); - lineitemsPath = createTempFile("lineitems", LINEITEMS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - - TPCHQuery3 tpch3 = new TPCHQuery3(); - return tpch3.getPlan( - String.valueOf(config.getInteger("parallelism", 1)), - ordersPath, - lineitemsPath, - resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } - - @Parameters - public static Collection getConfigurations() { - Configuration config = new Configuration(); - config.setInteger("parallelism", parallelism); - return toParameterList(config); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java deleted file mode 100644 index 4c51ac7c0b03a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3Unioned; -import org.apache.flink.test.util.RecordAPITestBase; - -// ----------------------------------------------------------------------------- -// --- NOTE --- -// -// This class contains test data generated by tools from by the -// Transaction Processing Council (TPC), specifically the TPC-H benchmark's -// data generator. -// -// Any form of use and redistribution must happen in accordance with the TPC-H -// Software License Agreement. -// -// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf -// ----------------------------------------------------------------------------- - -public class TPCHQuery3WithUnionITCase extends RecordAPITestBase { - - private String orders1Path = null; - private String orders2Path = null; - private String partJoin1Path = null; - private String partJoin2Path = null; - private String lineitemsPath = null; - private String resultPath = null; - - private static final String ORDERS1 = "1|36901|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|78002|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|123314|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|136777|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|44485|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|55624|F|58749.59|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|39136|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|130057|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n"; - - private static final String ORDERS2 = "33|66958|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|61001|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|127588|O|253724.56|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|115252|O|68289.96|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|86116|F|206680.66|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|124828|O|82500.05|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|81763|O|341734.47|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|32113|F|39414.99|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|16252|P|110643.60|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|129200|F|103740.67|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n"; - - private static final String LINEITEMS = "1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|\n" - + "1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |\n" - + "1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|\n" - + "1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|\n" - + "1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|\n" - + "1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|\n" - + "2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|\n" - + "3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|\n" - + "3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|\n" - + "3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |\n" - + "3|29380|1883|4|2|2618.76|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|\n" - + "3|183095|650|5|28|32986.52|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|\n" - + "3|62143|9662|6|26|28733.64|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|\n" - + "4|88035|5560|1|30|30690.90|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|\n" - + "5|37531|35|3|50|73426.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|\n" - + "6|139636|2150|1|37|61998.31|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|\n" - + "7|182052|9607|1|12|13608.60|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|\n" - + "7|145243|7758|2|9|11594.16|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|\n" - + "7|94780|9799|3|46|81639.88|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|\n" - + "7|163073|3074|4|28|31809.96|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|\n" - + "7|151894|9440|5|38|73943.82|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|\n" - + "7|79251|1759|6|35|43058.75|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |\n" - + "7|157238|2269|7|5|6476.15|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|\n" - + "32|82704|7721|1|28|47227.60|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|\n" - + "32|197921|441|2|32|64605.44|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |\n" - + "32|44161|6666|3|2|2210.32|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|\n" - + "32|2743|7744|4|4|6582.96|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|\n" - + "32|85811|8320|5|44|79059.64|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|\n" - + "32|11615|4117|6|6|9159.66|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|\n" - + "33|61336|8855|1|31|40217.23|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|\n" - + "33|60519|5532|2|32|47344.32|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|\n" - + "33|137469|9983|3|5|7532.30|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|\n" - + "33|33918|3919|4|41|75928.31|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|\n" - + "34|88362|871|1|13|17554.68|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|\n" - + "34|89414|1923|2|22|30875.02|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|\n" - + "34|169544|4577|3|6|9681.24|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |\n" - + "35|450|2951|1|24|32410.80|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|\n" - + "35|161940|4457|2|34|68065.96|0.06|0.08|N|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|\n" - + "35|120896|8433|3|7|13418.23|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |\n" - + "35|85175|7684|4|25|29004.25|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|\n" - + "35|119917|4940|5|34|65854.94|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|\n" - + "35|30762|3266|6|28|47397.28|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |\n" - + "36|119767|9768|1|42|75043.92|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |\n" - + "37|22630|5133|1|40|62105.20|0.09|0.03|A|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|\n" - + "37|126782|1807|2|39|70542.42|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|\n" - + "37|12903|5405|3|43|78083.70|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|\n" - + "38|175839|874|1|44|84252.52|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|\n" - + "39|2320|9821|1|44|53782.08|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|\n" - + "39|186582|4137|2|26|43383.08|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|\n" - + "39|67831|5350|3|46|82746.18|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|\n" - + "39|20590|3093|4|32|48338.88|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|\n" - + "39|54519|9530|5|43|63360.93|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|\n" - + "39|94368|6878|6|40|54494.40|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|\n" - + "64|85951|5952|1|21|40675.95|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|\n" - + "65|59694|4705|1|26|42995.94|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|\n" - + "65|73815|8830|2|22|39353.82|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-07-19|COLLECT COD|FOB| ideas. special, r|\n" - + "65|1388|3889|3|21|27076.98|0.09|0.07|N|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|\n" - + "67|21636|9143|1|4|6230.52|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|\n" - + "67|20193|5198|2|12|13358.28|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|\n" - + "67|173600|6118|3|5|8368.00|0.03|0.07|N|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |\n" - + "67|87514|7515|4|44|66066.44|0.08|0.06|N|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|\n" - + "67|40613|8126|5|23|35733.03|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|\n" - + "67|178306|824|6|29|40144.70|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |\n"; - - - private static final String PART_JOIN_1 = "5|0|50723.92|\n"; - - private static final String PART_JOIN_2 = "5|0|23678.55|\n" + - "66|0|64061.68|\n" + - "66|0|35126.41|\n"; - - - - private static final String EXPECTED_RESULT = "5|0|147828.97\n" + "66|0|99188.09\n"; - - public TPCHQuery3WithUnionITCase(){ - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - orders1Path = createTempFile("orders1",ORDERS1); - orders2Path = createTempFile("orders2", ORDERS2); - partJoin1Path = createTempFile("partJoin1", PART_JOIN_1); - partJoin2Path = createTempFile("partJoin2", PART_JOIN_2); - lineitemsPath = createTempFile("lineitems", LINEITEMS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - TPCHQuery3Unioned tpch3 = new TPCHQuery3Unioned(); - return tpch3.getPlan( - Integer.valueOf(parallelism).toString(), - orders1Path, - orders2Path, - partJoin1Path, - partJoin2Path, - lineitemsPath, - resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java deleted file mode 100644 index 30c1b3e2f308e..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.relational.TPCHQuery4; -import org.apache.flink.test.util.RecordAPITestBase; - -public class TPCHQuery4ITCase extends RecordAPITestBase { - - private String ordersPath; - private String lineitemsPath; - private String resultPath ; - - private final String ORDERS = "1|36901|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|78002|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|123314|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|136777|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|44485|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|55624|F|58749.59|1995-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|39136|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|130057|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n" - + "33|66958|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|61001|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|127588|O|253724.56|1995-01-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|115252|O|68289.96|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|86116|F|206680.66|1995-02-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|124828|O|82500.05|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|81763|O|341734.47|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|32113|F|39414.99|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|16252|P|110643.60|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|129200|F|103740.67|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n"; - - private static final String LINEITEMS = "1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|\n" - + "1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |\n" - + "1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|\n" - + "1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|\n" - + "1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|\n" - + "1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|\n" - + "2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|\n" - + "3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|\n" - + "3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|\n" - + "3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |\n" - + "3|29380|1883|4|2|2618.76|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|\n" - + "3|183095|650|5|28|32986.52|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|\n" - + "3|62143|9662|6|26|28733.64|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|\n" - + "4|88035|5560|1|30|30690.90|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|\n" - + "5|108570|8571|1|15|23678.55|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |\n" - + "5|123927|3928|2|26|50723.92|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|\n" - + "5|37531|35|3|50|73426.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|\n" - + "6|139636|2150|1|37|61998.31|0.08|0.03|A|F|1995-04-27|1995-05-15|1995-05-20|TAKE BACK RETURN|TRUCK|p furiously special foxes|\n" - + "7|182052|9607|1|12|13608.60|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|\n" - + "7|145243|7758|2|9|11594.16|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|\n" - + "7|94780|9799|3|46|81639.88|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|\n" - + "7|163073|3074|4|28|31809.96|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|\n" - + "7|151894|9440|5|38|73943.82|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|\n" - + "7|79251|1759|6|35|43058.75|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |\n" - + "7|157238|2269|7|5|6476.15|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|\n" - + "32|82704|7721|1|28|47227.60|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|\n" - + "32|197921|441|2|32|64605.44|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |\n" - + "32|44161|6666|3|2|2210.32|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|\n" - + "32|2743|7744|4|4|6582.96|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|\n" - + "32|85811|8320|5|44|79059.64|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|\n" - + "32|11615|4117|6|6|9159.66|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|\n" - + "33|61336|8855|1|31|40217.23|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|\n" - + "33|60519|5532|2|32|47344.32|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|\n" - + "33|137469|9983|3|5|7532.30|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|\n" - + "33|33918|3919|4|41|75928.31|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|\n" - + "34|88362|871|1|13|17554.68|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|\n" - + "34|89414|1923|2|22|30875.02|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|\n" - + "34|169544|4577|3|6|9681.24|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |\n" - + "35|450|2951|1|24|32410.80|0.02|0.00|N|O|1995-02-21|1995-01-03|1995-03-18|TAKE BACK RETURN|FOB|, regular tithe|\n" - + "35|161940|4457|2|34|68065.96|0.06|0.08|N|O|1995-01-22|1995-01-06|1995-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|\n" - + "35|120896|8433|3|7|13418.23|0.06|0.04|N|O|1995-01-19|1995-12-22|1995-01-29|NONE|MAIL| the carefully regular |\n" - + "35|85175|7684|4|25|29004.25|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|\n" - + "35|119917|4940|5|34|65854.94|0.08|0.06|N|O|1995-11-08|1995-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|\n" - + "35|30762|3266|6|28|47397.28|0.03|0.02|N|O|1995-02-01|1995-12-24|1995-02-28|COLLECT COD|RAIL|ly alongside of |\n" - + "36|119767|9768|1|42|75043.92|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |\n" - + "37|22630|5133|1|40|62105.20|0.09|0.03|A|F|1995-07-21|1995-08-01|1995-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|\n" - + "37|126782|1807|2|39|70542.42|0.05|0.02|A|F|1995-07-02|1995-08-18|1995-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|\n" - + "37|12903|5405|3|43|78083.70|0.05|0.08|A|F|1995-07-10|1995-07-06|1995-08-02|DELIVER IN PERSON|TRUCK|iously ste|\n" - + "38|175839|874|1|44|84252.52|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|\n" - + "39|2320|9821|1|44|53782.08|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|\n" - + "39|186582|4137|2|26|43383.08|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|\n" - + "39|67831|5350|3|46|82746.18|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|\n" - + "39|20590|3093|4|32|48338.88|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|\n" - + "39|54519|9530|5|43|63360.93|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|\n" - + "39|94368|6878|6|40|54494.40|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|\n" - + "64|85951|5952|1|21|40675.95|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|\n" - + "65|59694|4705|1|26|42995.94|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|\n" - + "65|73815|8830|2|22|39353.82|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-05-19|COLLECT COD|FOB| ideas. special, r|\n" - + "65|1388|3889|3|21|27076.98|0.09|0.07|N|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|\n" - + "66|115118|7630|1|31|35126.41|0.00|0.08|R|F|1994-02-19|1994-03-11|1994-02-20|TAKE BACK RETURN|RAIL|ut the unusual accounts sleep at the bo|\n" - + "66|173489|3490|2|41|64061.68|0.04|0.07|A|F|1994-02-21|1994-03-01|1994-03-18|COLLECT COD|AIR| regular de|\n" - + "67|21636|9143|1|4|6230.52|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|\n" - + "67|20193|5198|2|12|13358.28|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|\n" - + "67|173600|6118|3|5|8368.00|0.03|0.07|N|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |\n" - + "67|87514|7515|4|44|66066.44|0.08|0.06|N|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|\n" - + "67|40613|8126|5|23|35733.03|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|\n" - + "67|178306|824|6|29|40144.70|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |\n"; - - private static final String EXPECTED_RESULT = "1-URGENT|2|\n" + "3-MEDIUM|2|\n" + "4-NOT SPECIFIED|4|"; - - public TPCHQuery4ITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - ordersPath = createTempFile("orders", ORDERS); - lineitemsPath = createTempFile("lineitems", LINEITEMS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - TPCHQuery4 tpch4 = new TPCHQuery4(); - return tpch4.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, lineitemsPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java deleted file mode 100644 index ab99e22eb2f74..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java +++ /dev/null @@ -1,391 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.relational.TPCHQuery9; -import org.apache.flink.test.util.RecordAPITestBase; - -// ----------------------------------------------------------------------------- -// --- NOTE --- -// -// This class contains test data generated by tools from by the -// Transaction Processing Council (TPC), specifically the TPC-H benchmark's -// data generator. -// -// Any form of use and redistribution must happen in accordance with the TPC-H -// Software License Agreement. -// -// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf -// ----------------------------------------------------------------------------- - -public class TPCHQuery9ITCase extends RecordAPITestBase { - - private String partInputPath; - private String partSuppInputPath; - private String ordersInputPath; - private String lineItemInputPath; - private String supplierInputPath; - private String nationInputPath; - private String resultPath; - - private static final String PART = "" - + "1|goldenrod lavender spring chocolate lace|Manufacturer#1|Brand#13|PROMO BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi|\n" - + "2|maroon sky cream royal snow|Manufacturer#1|Brand#13|LARGE BRUSHED BRASS|1|LG CASE|902.00|lar accounts amo|\n" - + "3|brown blue puff midnight black|Manufacturer#4|Brand#42|STANDARD POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag|\n" - + "4|orange goldenrod peach misty seashell|Manufacturer#3|Brand#34|SMALL PLATED BRASS|14|MED DRUM|904.00|p furiously r|\n" - + "5|midnight linen almond tomato plum|Manufacturer#3|Brand#32|STANDARD POLISHED TIN|15|SM PKG|905.00| wake carefully |\n" - + "6|deep ivory light pink cream|Manufacturer#2|Brand#24|PROMO PLATED STEEL|4|MED BAG|906.00|sual a|\n" - + "7|smoke magenta midnight purple blanched|Manufacturer#1|Brand#11|SMALL PLATED COPPER|45|SM BAG|907.00|lyly. ex|\n" - + "8|floral moccasin violet yellow sky|Manufacturer#4|Brand#44|PROMO BURNISHED TIN|41|LG DRUM|908.00|eposi|\n" - + "9|coral chiffon rose red saddle|Manufacturer#4|Brand#43|SMALL BURNISHED STEEL|12|WRAP CASE|909.00|ironic foxe|\n" - + "10|grey orchid mint purple misty|Manufacturer#5|Brand#54|LARGE BURNISHED STEEL|44|LG CAN|910.01|ithely final deposit|\n" - + "11|tomato navy linen grey maroon|Manufacturer#2|Brand#25|STANDARD BURNISHED NICKEL|43|WRAP BOX|911.01|ng gr|\n" - + "12|yellow salmon wheat blanched purple|Manufacturer#3|Brand#33|MEDIUM ANODIZED STEEL|25|JUMBO CASE|912.01| quickly|\n" - + "13|steel black tomato lemon aquamarine|Manufacturer#5|Brand#55|MEDIUM BURNISHED NICKEL|1|JUMBO PACK|913.01|osits.|\n" - + "14|sienna gainsboro cornsilk lavender blush|Manufacturer#1|Brand#13|SMALL POLISHED STEEL|28|JUMBO BOX|914.01|kages c|\n" - + "15|ivory khaki black plum medium|Manufacturer#1|Brand#15|LARGE ANODIZED BRASS|45|LG CASE|915.01|usual ac|\n" - + "16|maroon cornsilk steel slate navy|Manufacturer#3|Brand#32|PROMO PLATED TIN|2|MED PACK|916.01|unts a|\n" - + "17|spring grey turquoise cyan magenta|Manufacturer#4|Brand#43|ECONOMY BRUSHED STEEL|16|LG BOX|917.01| regular accounts|\n" - + "18|pale lace powder dim ivory|Manufacturer#1|Brand#11|SMALL BURNISHED STEEL|42|JUMBO PACK|918.01|s cajole slyly a|\n" - + "19|hot aquamarine green khaki light|Manufacturer#2|Brand#23|SMALL ANODIZED NICKEL|33|WRAP BOX|919.01| pending acc|\n" - + "20|sky chiffon burnished spring powder|Manufacturer#1|Brand#12|LARGE POLISHED NICKEL|48|MED BAG|920.02|are across the asympt|\n" - + "21|yellow snow spring sandy antique|Manufacturer#3|Brand#33|SMALL BURNISHED TIN|31|MED BAG|921.02|ss packages. pendin|\n" - + "22|dark gainsboro medium cream burnished|Manufacturer#4|Brand#43|PROMO POLISHED BRASS|19|LG DRUM|922.02| even p|\n" - + "23|pale seashell olive chartreuse tomato|Manufacturer#3|Brand#35|MEDIUM BURNISHED TIN|42|JUMBO JAR|923.02|nic, fina|\n" - + "24|violet lemon grey navajo turquoise|Manufacturer#5|Brand#52|MEDIUM PLATED STEEL|20|MED CASE|924.02| final the|\n" - + "25|grey chocolate antique dark ghost|Manufacturer#5|Brand#55|STANDARD BRUSHED COPPER|3|JUMBO BAG|925.02|requests wake|\n" - + "26|ghost violet maroon khaki saddle|Manufacturer#3|Brand#32|SMALL BRUSHED STEEL|32|SM CASE|926.02| instructions i|\n" - + "27|navy dim saddle indian midnight|Manufacturer#1|Brand#14|LARGE ANODIZED TIN|20|MED PKG|927.02|s wake. ir|\n" - + "28|orchid burnished metallic chiffon red|Manufacturer#4|Brand#44|SMALL PLATED COPPER|19|JUMBO PKG|928.02|x-ray pending, iron|\n" - + "29|aquamarine puff purple drab pale|Manufacturer#3|Brand#33|PROMO PLATED COPPER|7|LG DRUM|929.02| carefully fluffi|\n" - + "30|brown chiffon firebrick blanched smoke|Manufacturer#4|Brand#42|PROMO ANODIZED TIN|17|LG BOX|930.03|carefully bus|\n" - + "31|cream goldenrod linen almond tomato|Manufacturer#5|Brand#53|STANDARD BRUSHED TIN|10|LG BAG|931.03|uriously s|\n" - + "32|blanched purple maroon tan slate|Manufacturer#4|Brand#42|ECONOMY PLATED BRASS|31|LG CASE|932.03|urts. carefully fin|\n" - + "33|saddle cream tomato cyan chartreuse|Manufacturer#2|Brand#22|ECONOMY PLATED NICKEL|16|LG PKG|933.03|ly eve|\n" - + "34|purple lawn ghost steel azure|Manufacturer#1|Brand#13|LARGE BRUSHED STEEL|8|JUMBO BOX|934.03|riously ironic|\n" - + "35|puff white cornsilk green forest|Manufacturer#4|Brand#43|MEDIUM ANODIZED BRASS|14|JUMBO PACK|935.03|e carefully furi|\n" - + "36|slate frosted violet sienna dark|Manufacturer#2|Brand#25|SMALL BURNISHED COPPER|3|JUMBO CAN|936.03|olites o|\n" - + "37|floral khaki light drab almond|Manufacturer#4|Brand#45|LARGE POLISHED TIN|48|JUMBO BOX|937.03|silent |\n" - + "38|snow lavender slate midnight forest|Manufacturer#4|Brand#43|ECONOMY ANODIZED BRASS|11|SM JAR|938.03|structions inte|\n" - + "39|navy rosy antique olive burlywood|Manufacturer#5|Brand#53|SMALL POLISHED TIN|43|JUMBO JAR|939.03|se slowly above the fl|\n" - + "40|misty lace snow thistle saddle|Manufacturer#2|Brand#25|ECONOMY BURNISHED COPPER|27|SM CASE|940.04|! blithely specia|\n" - + "41|moccasin maroon dim cream frosted|Manufacturer#2|Brand#23|ECONOMY ANODIZED TIN|7|WRAP JAR|941.04|uriously. furiously cl|\n" - + "42|blanched lace magenta frosted rosy|Manufacturer#5|Brand#52|MEDIUM BURNISHED TIN|45|LG BOX|942.04|the slow|\n" - + "43|aquamarine steel indian cornflower chiffon|Manufacturer#4|Brand#44|PROMO POLISHED STEEL|5|WRAP CASE|943.04|e slyly along the ir|\n" - + "44|red light midnight wheat chartreuse|Manufacturer#4|Brand#45|MEDIUM PLATED TIN|48|SM PACK|944.04|pinto beans. carefully|\n" - + "45|green dodger beige peru navajo|Manufacturer#4|Brand#43|SMALL BRUSHED NICKEL|9|WRAP BAG|945.04|nts bo|\n" - + "46|black brown beige rose slate|Manufacturer#1|Brand#11|STANDARD POLISHED TIN|45|WRAP CASE|946.04|the blithely unusual |\n" - + "47|green cyan rose misty pale|Manufacturer#4|Brand#45|LARGE BURNISHED BRASS|14|JUMBO PACK|947.04| even plate|\n" - + "48|navajo almond royal forest cornflower|Manufacturer#5|Brand#53|STANDARD BRUSHED STEEL|27|JUMBO CASE|948.04|ng to the depo|\n" - + "49|cyan powder lime chartreuse goldenrod|Manufacturer#2|Brand#24|SMALL BURNISHED TIN|31|MED DRUM|949.04|ar pack|\n" - + "50|peach maroon chiffon lawn red|Manufacturer#3|Brand#33|LARGE ANODIZED TIN|25|WRAP PKG|950.05|kages m|\n" - + "51|rosy linen royal drab floral|Manufacturer#4|Brand#45|ECONOMY BURNISHED NICKEL|34|JUMBO PACK|951.05|n foxes|\n" - + "52|lemon salmon snow forest blush|Manufacturer#3|Brand#35|STANDARD BURNISHED TIN|25|WRAP CASE|952.05| final deposits. fu|\n" - + "53|blue burlywood magenta gainsboro sandy|Manufacturer#2|Brand#23|ECONOMY BURNISHED NICKEL|32|MED BAG|953.05|mptot|\n" - + "54|hot sienna cornsilk saddle dark|Manufacturer#2|Brand#21|LARGE BURNISHED COPPER|19|WRAP CASE|954.05|e blithely|\n" - + "55|honeydew chocolate magenta steel lavender|Manufacturer#2|Brand#23|ECONOMY BRUSHED COPPER|9|MED BAG|955.05|ly final pac|\n" - + "56|chocolate lavender forest tomato aquamarine|Manufacturer#1|Brand#12|MEDIUM PLATED STEEL|20|WRAP DRUM|956.05|ts. blithel|\n"; - - private static final String PARTSUPP = "" - + "1|2|3325|771.64|, even theodolites. regular, final theodolites eat after the carefully pending foxes. furiously regular deposits sleep slyly. carefully bold realms above the ironic dependencies haggle careful|\n" - + "1|252|8076|993.49|ven ideas. quickly even packages print. pending multipliers must have to are fluff|\n" - + "19|502|3956|337.09|after the fluffily ironic deposits? blithely special dependencies integrate furiously even excuses. blithely silent theodolites could have to haggle pending, express requests; fu|\n" - + "1|752|4069|357.84|al, regular dependencies serve carefully after the quickly final pinto beans. furiously even deposits sleep quickly final, silent pinto beans. fluffily reg|\n" - + "2|3|8895|378.49|nic accounts. final accounts sleep furiously about the ironic, bold packages. regular, regular accounts|\n" - + "2|253|4969|915.27|ptotes. quickly pending dependencies integrate furiously. fluffily ironic ideas impress blithely above the express accounts. furiously even epitaphs need to wak|\n" - + "2|503|8539|438.37|blithely bold ideas. furiously stealthy packages sleep fluffily. slyly special deposits snooze furiously carefully regular accounts. regular deposits according to the accounts nag carefully slyl|\n" - + "2|753|3025|306.39|olites. deposits wake carefully. even, express requests cajole. carefully regular ex|\n" - + "3|4|4651|920.92|ilent foxes affix furiously quickly unusual requests. even packages across the carefully even theodolites nag above the sp|\n" - + "3|254|4093|498.13|ending dependencies haggle fluffily. regular deposits boost quickly carefully regular requests. deposits affix furiously around the pinto beans. ironic, unusual platelets across the p|\n" - + "3|504|3917|645.40|of the blithely regular theodolites. final theodolites haggle blithely carefully unusual ideas. blithely even f|\n" - + "3|754|9942|191.92| unusual, ironic foxes according to the ideas detect furiously alongside of the even, express requests. blithely regular the|\n" - + "4|5|1339|113.97| carefully unusual ideas. packages use slyly. blithely final pinto beans cajole along the furiously express requests. regular orbits haggle carefully. care|\n" - + "4|255|6377|591.18|ly final courts haggle carefully regular accounts. carefully regular accounts could integrate slyly. slyly express packages about the accounts wake slyly|\n" - + "4|505|2694|51.37|g, regular deposits: quick instructions run across the carefully ironic theodolites-- final dependencies haggle into the dependencies. f|\n" - + "4|755|2480|444.37|requests sleep quickly regular accounts. theodolites detect. carefully final depths w|\n" - + "5|6|3735|255.88|arefully even requests. ironic requests cajole carefully even dolphin|\n" - + "5|256|9653|50.52|y stealthy deposits. furiously final pinto beans wake furiou|\n" - + "5|506|1329|219.83|iously regular deposits wake deposits. pending pinto beans promise ironic dependencies. even, regular pinto beans integrate|\n" - + "5|756|6925|537.98|sits. quickly fluffy packages wake quickly beyond the blithely regular requests. pending requests cajole among the final pinto beans. carefully busy theodolites affix quickly stealthily |\n" - + "6|7|8851|130.72|usly final packages. slyly ironic accounts poach across the even, sly requests. carefully pending request|\n" - + "6|257|1627|424.25| quick packages. ironic deposits print. furiously silent platelets across the carefully final requests are slyly along the furiously even instructi|\n" - + "6|507|3336|642.13|final instructions. courts wake packages. blithely unusual realms along the multipliers nag |\n" - + "6|757|6451|175.32| accounts alongside of the slyly even accounts wake carefully final instructions-- ruthless platelets wake carefully ideas. even deposits are quickly final,|\n" - + "7|8|7454|763.98|y express tithes haggle furiously even foxes. furiously ironic deposits sleep toward the furiously unusual|\n" - + "7|258|2770|149.66|hould have to nag after the blithely final asymptotes. fluffily spe|\n" - + "7|508|3377|68.77|usly against the daring asymptotes. slyly regular platelets sleep quickly blithely regular deposits. boldly regular deposits wake blithely ironic accounts|\n" - + "7|758|9460|299.58|. furiously final ideas hinder slyly among the ironic, final packages. blithely ironic dependencies cajole pending requests: blithely even packa|\n" - + "19|9|6834|249.63|lly ironic accounts solve express, unusual theodolites. special packages use quickly. quickly fin|\n" - + "8|259|396|957.34|r accounts. furiously pending dolphins use even, regular platelets. final|\n" - + "8|509|9845|220.62|s against the fluffily special packages snooze slyly slyly regular p|\n" - + "8|759|8126|916.91|final accounts around the blithely special asymptotes wake carefully beyond the bold dugouts. regular ideas haggle furiously after|\n" - + "9|10|7054|84.20|ts boost. evenly regular packages haggle after the quickly careful accounts. |\n" - + "9|260|7542|811.84|ate after the final pinto beans. express requests cajole express packages. carefully bold ideas haggle furiously. blithely express accounts eat carefully among the evenly busy accounts. carefully un|\n" - + "9|510|9583|381.31|d foxes. final, even braids sleep slyly slyly regular ideas. unusual ideas above|\n" - + "9|760|3063|291.84| the blithely ironic instructions. blithely express theodolites nag furiously. carefully bold requests shall have to use slyly pending requests. carefully regular instr|\n" - + "19|11|2952|996.12| bold foxes wake quickly even, final asymptotes. blithely even depe|\n" - + "10|261|3335|673.27|s theodolites haggle according to the fluffily unusual instructions. silent realms nag carefully ironic theodolites. furiously unusual instructions would detect fu|\n" - + "10|511|5691|164.00|r, silent instructions sleep slyly regular pinto beans. furiously unusual gifts use. silently ironic theodolites cajole final deposits! express dugouts are furiously. packages sleep |\n" - + "10|761|841|374.02|refully above the ironic packages. quickly regular packages haggle foxes. blithely ironic deposits a|\n" - + "11|12|4540|709.87|thely across the blithely unusual requests. slyly regular instructions wake slyly ironic theodolites. requests haggle blithely above the blithely brave p|\n" - + "45|27|4729|894.90|ters wake. sometimes bold packages cajole sometimes blithely final instructions. carefully ironic foxes after the furiously unusual foxes cajole carefully acr|\n" - + "11|512|3708|818.74|inal accounts nag quickly slyly special frays; bold, final theodolites play slyly after the furiously pending packages. f|\n" - + "11|762|3213|471.98|nusual, regular requests use carefully. slyly final packages haggle quickly. slyly express packages impress blithely across the blithely regular ideas. regular depe|\n" - + "12|13|3610|659.73|jole bold theodolites. final packages haggle! carefully regular deposits play furiously among the special ideas. quickly ironic packages detect quickly carefully final|\n" - + "12|263|7606|332.81|luffily regular courts engage carefully special realms. regular accounts across the blithely special pinto beans use carefully at the silent request|\n" - + "12|513|824|337.06|es are unusual deposits. fluffily even deposits across the blithely final theodolites doubt across the unusual accounts. regular, |\n" - + "12|763|5454|901.70|s across the carefully regular courts haggle fluffily among the even theodolites. blithely final platelets x-ray even ideas. fluffily express pinto beans sleep slyly. carefully even a|\n" - + "13|14|612|169.44|s. furiously even asymptotes use slyly blithely express foxes. pending courts integrate blithely among the ironic requests! blithely pending deposits integrate slyly furiously final packa|\n" - + "13|264|7268|862.70|s sleep slyly packages. final theodolites to the express packages haggle quic|\n" - + "13|514|864|38.64|s after the slyly pending instructions haggle even, express requests. permanently regular pinto beans are. slyly pending req|\n" - + "13|764|9736|327.18|tect after the express instructions. furiously silent ideas sleep blithely special ideas. attainments sleep furiously. carefully bold requests ab|\n" - + "14|15|5278|650.07|e quickly among the furiously ironic accounts. special, final sheaves against the|\n" - + "14|265|5334|889.50|ss dependencies are furiously silent excuses. blithely ironic pinto beans affix quickly according to the slyly ironic asymptotes. final packag|\n" - + "14|515|3676|893.39|sits are according to the fluffily silent asymptotes. final ideas are slyly above the regular instructions. furiousl|\n" - + "45|765|4947|310.13| final deposits boost slyly regular packages; carefully pending theodolites |\n"; - - private static final String ORDERS = "" - + "1|3691|O|194029.55|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|7801|O|60951.63|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|12332|F|247296.05|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|13678|O|53829.87|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|4450|F|139660.54|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|5563|F|65843.52|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|3914|O|231037.28|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|13006|O|166802.63|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n" - + "33|6697|F|118518.56|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|6101|O|75662.77|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|12760|O|192885.43|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|11527|O|72196.43|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|8612|F|156440.15|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|12484|O|64695.26|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|8177|O|307811.89|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|3212|F|30616.90|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|1627|P|99763.79|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|12920|F|100991.26|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n" - + "67|5662|O|167270.36|1996-12-19|4-NOT SPECIFIED|Clerk#000000547|0|symptotes haggle slyly around the furiously iron|\n" - + "68|2855|O|305815.83|1998-04-18|3-MEDIUM|Clerk#000000440|0| pinto beans sleep carefully. blithely ironic deposits haggle furiously acro|\n" - + "69|8449|F|228015.94|1994-06-04|4-NOT SPECIFIED|Clerk#000000330|0| depths atop the slyly thin deposits detect among the furiously silent accou|\n" - + "70|6434|F|133507.10|1993-12-18|5-LOW|Clerk#000000322|0| carefully ironic request|\n" - + "71|338|O|244449.86|1998-01-24|4-NOT SPECIFIED|Clerk#000000271|0| express deposits along the blithely regul|\n" - + "96|10778|F|72504.36|1994-04-17|2-HIGH|Clerk#000000395|0|oost furiously. pinto|\n" - + "97|2107|F|128590.11|1993-01-29|3-MEDIUM|Clerk#000000547|0|hang blithely along the regular accounts. furiously even ideas after the|\n" - + "98|10448|F|62956.90|1994-09-25|1-URGENT|Clerk#000000448|0|c asymptotes. quickly regular packages should have to nag re|\n" - + "99|8891|F|136624.34|1994-03-13|4-NOT SPECIFIED|Clerk#000000973|0|e carefully ironic packages. pending|\n" - + "100|14701|O|204408.59|1998-02-28|4-NOT SPECIFIED|Clerk#000000577|0|heodolites detect slyly alongside of the ent|\n" - + "101|2800|O|142434.13|1996-03-17|3-MEDIUM|Clerk#000000419|0|ding accounts above the slyly final asymptote|\n" - + "102|73|O|172239.95|1997-05-09|2-HIGH|Clerk#000000596|0| slyly according to the asymptotes. carefully final packages integrate furious|\n" - + "103|2911|O|147675.81|1996-06-20|4-NOT SPECIFIED|Clerk#000000090|0|ges. carefully unusual instructions haggle quickly regular f|\n" - + "128|7396|F|57495.50|1992-06-15|1-URGENT|Clerk#000000385|0|ns integrate fluffily. ironic asymptotes after the regular excuses nag around |\n" - + "129|7114|F|273469.52|1992-11-19|5-LOW|Clerk#000000859|0|ing tithes. carefully pending deposits boost about the silently express |\n" - + "130|3697|F|169065.69|1992-05-08|2-HIGH|Clerk#000000036|0|le slyly unusual, regular packages? express deposits det|\n" - + "131|9275|F|143898.91|1994-06-08|3-MEDIUM|Clerk#000000625|0|after the fluffily special foxes integrate s|\n" - + "132|2641|F|180897.80|1993-06-11|3-MEDIUM|Clerk#000000488|0|sits are daringly accounts. carefully regular foxes sleep slyly about the|\n" - + "133|4400|O|120535.39|1997-11-29|1-URGENT|Clerk#000000738|0|usly final asymptotes |\n" - + "134|620|F|203218.79|1992-05-01|4-NOT SPECIFIED|Clerk#000000711|0|lar theodolites boos|\n" - + "135|6049|O|280793.15|1995-10-21|4-NOT SPECIFIED|Clerk#000000804|0|l platelets use according t|\n" - + "160|8251|O|114252.21|1996-12-19|4-NOT SPECIFIED|Clerk#000000342|0|thely special sauternes wake slyly of t|\n" - + "161|1663|F|22632.04|1994-08-31|2-HIGH|Clerk#000000322|0|carefully! special instructions sin|\n" - + "162|1412|O|3658.13|1995-05-08|3-MEDIUM|Clerk#000000378|0|nts hinder fluffily ironic instructions. express, express excuses |\n" - + "163|8776|O|172394.12|1997-09-05|3-MEDIUM|Clerk#000000379|0|y final packages. final foxes since the quickly even|\n" - + "164|79|F|315228.81|1992-10-21|5-LOW|Clerk#000000209|0|cajole ironic courts. slyly final ideas are slyly. blithely final Tiresias sub|\n" - + "165|2725|F|204566.92|1993-01-30|4-NOT SPECIFIED|Clerk#000000292|0|across the blithely regular accounts. bold|\n" - + "166|10783|O|152280.99|1995-09-12|2-HIGH|Clerk#000000440|0|lets. ironic, bold asymptotes kindle|\n" - + "167|11941|F|53697.73|1993-01-04|4-NOT SPECIFIED|Clerk#000000731|0|s nag furiously bold excuses. fluffily iron|\n" - + "192|8257|O|151014.95|1997-11-25|5-LOW|Clerk#000000483|0|y unusual platelets among the final instructions integrate rut|\n" - + "193|7907|F|60599.97|1993-08-08|1-URGENT|Clerk#000000025|0|the furiously final pin|\n" - + "194|6173|F|169409.25|1992-04-05|3-MEDIUM|Clerk#000000352|0|egular requests haggle slyly regular, regular pinto beans. asymptote|\n" - + "195|13543|F|169343.52|1993-12-28|3-MEDIUM|Clerk#000000216|0|old forges are furiously sheaves. slyly fi|\n" - + "196|6484|F|55238.37|1993-03-17|2-HIGH|Clerk#000000988|0|beans boost at the foxes. silent foxes|\n" - + "197|3253|P|162567.01|1995-04-07|2-HIGH|Clerk#000000969|0|solve quickly about the even braids. carefully express deposits affix care|\n" - + "198|11023|O|167234.32|1998-01-02|4-NOT SPECIFIED|Clerk#000000331|0|its. carefully ironic requests sleep. furiously express fox|\n" - + "199|5297|O|93001.13|1996-03-07|2-HIGH|Clerk#000000489|0|g theodolites. special packag|\n" - + "224|248|F|232428.88|1994-06-18|4-NOT SPECIFIED|Clerk#000000642|0|r the quickly thin courts. carefully|\n"; - - private static final String LINEITEM = "" - + "1|19|785|1|17|24386.67|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|\n" - + "1|6731|732|2|36|58958.28|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |\n" - + "1|45|27|3|8|10210.96|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|\n" - + "1|214|465|4|28|31197.88|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|\n" - + "1|2403|160|5|24|31329.60|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|\n" - + "1|1564|67|6|32|46897.92|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|\n" - + "2|10617|138|1|38|58049.18|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|\n" - + "3|19|9|1|45|59869.35|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|\n" - + "3|19|11|2|49|88489.10|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|\n" - + "3|12845|370|3|27|47461.68|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |\n" - + "3|45|502|4|2|3681.86|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|\n" - + "3|45|765|5|28|34392.68|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|\n" - + "3|6215|984|6|26|29151.46|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|\n" - + "4|8804|579|1|30|51384.00|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|\n" - + "5|10857|858|1|15|26517.75|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |\n" - + "5|12393|394|2|26|33940.14|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|\n" - + "5|19|8|3|50|82887.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|\n" - + "6|45|27|1|37|69484.52|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|\n" - + "7|45|27|1|12|13490.40|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|\n" - + "7|19|9|2|9|12955.68|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|\n" - + "7|9478|997|3|46|63823.62|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|\n" - + "7|16308|309|4|28|34280.40|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|\n" - + "7|19|11|5|38|41997.22|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|\n" - + "7|7926|184|6|35|64187.20|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |\n" - + "7|45|765|7|5|8198.60|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|\n" - + "32|45|765|1|28|33019.56|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|\n" - + "32|19793|63|2|32|54809.28|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |\n" - + "32|19|502|3|2|2642.82|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|\n" - + "32|275|776|4|4|4701.08|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|\n" - + "32|19|11|5|44|65585.52|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|\n" - + "32|1162|414|6|6|6378.96|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|\n" - + "33|6134|903|1|31|32244.03|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|\n" - + "33|6052|565|2|32|30657.60|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|\n" - + "33|13747|11|3|5|8303.70|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|\n" - + "33|19|9|4|41|53110.99|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|\n" - + "34|8837|92|1|13|22695.79|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|\n" - + "34|45|502|2|22|40720.68|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|\n" - + "34|16955|488|3|6|11231.70|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |\n" - + "35|45|296|1|24|22680.96|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|\n" - + "35|19|765|2|34|37746.46|0.06|0.08|N|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|\n" - + "35|12090|877|3|7|7014.63|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |\n" - + "35|8518|777|4|25|35662.75|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|\n" - + "35|45|765|5|34|64735.66|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|\n" - + "35|3077|331|6|28|27441.96|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |\n" - + "36|11977|978|1|42|79336.74|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |\n" - + "37|2263|516|1|40|46610.40|0.09|0.03|A|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|\n" - + "37|12679|204|2|39|62075.13|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|\n" - + "37|19|9|3|43|51268.47|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|\n" - + "38|17584|119|1|44|66069.52|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|\n" - + "39|232|983|1|44|49818.12|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|\n" - + "39|18659|464|2|26|41018.90|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|\n" - + "39|45|27|3|46|77775.88|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|\n" - + "39|2059|312|4|32|30753.60|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|\n" - + "39|5452|963|5|43|58370.35|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|\n" - + "39|9437|697|6|40|53857.20|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|\n"; - - private static final String SUPPLIER = "" - + "1|Supplier#000000001| N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ|17|27-918-335-1736|5755.94|each slyly above the careful|\n" - + "2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen|\n" - + "3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl|\n" - + "4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp|\n" - + "5|Supplier#000000005|Gcdm2rJRzl5qlTVzc|11|21-151-690-3663|-283.84|. slyly regular pinto bea|\n" - + "6|Supplier#000000006|tQxuVm7s7CnK|14|24-696-997-4969|1365.79|final accounts. regular dolphins use against the furiously ironic decoys. |\n" - + "7|Supplier#000000007|s,4TicNGB4uO6PaSqNBUq|23|33-990-965-2201|6820.35|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit|\n" - + "8|Supplier#000000008|9Sq4bBH2FQEmaFOocY45sRTxo6yuoG|17|27-498-742-3860|7627.85|al pinto beans. asymptotes haggl|\n" - + "9|Supplier#000000009|1KhUgZegwM3ua7dsYmekYBsK|10|20-403-398-8662|5302.37|s. unusual, even requests along the furiously regular pac|\n" - + "10|Supplier#000000010|Saygah3gYWMp72i PY|24|34-852-489-8585|3891.91|ing waters. regular requests ar|\n" - + "11|Supplier#000000011|JfwTs,LZrV, M,9C|18|28-613-996-1505|3393.08|y ironic packages. slyly ironic accounts affix furiously; ironically unusual excuses across the flu|\n" - + "12|Supplier#000000012|aLIW q0HYd|8|18-179-925-7181|1432.69|al packages nag alongside of the bold instructions. express, daring accounts|\n" - + "13|Supplier#000000013|HK71HQyWoqRWOX8GI FpgAifW,2PoH|3|13-727-620-7813|9107.22|requests engage regularly instructions. furiously special requests ar|\n" - + "14|Supplier#000000014|EXsnO5pTNj4iZRm|15|25-656-247-5058|9189.82|l accounts boost. fluffily bold warhorses wake|\n" - + "15|Supplier#000000015|olXVbNBfVzRqgokr1T,Ie|8|18-453-357-6394|308.56| across the furiously regular platelets wake even deposits. quickly express she|\n" - + "16|Supplier#000000016|YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh|22|32-822-502-4215|2972.26|ously express ideas haggle quickly dugouts? fu|\n" - + "17|Supplier#000000017|c2d,ESHRSkK3WYnxpgw6aOqN0q|19|29-601-884-9219|1687.81|eep against the furiously bold ideas. fluffily bold packa|\n" - + "18|Supplier#000000018|PGGVE5PWAMwKDZw |16|26-729-551-1115|7040.82|accounts snooze slyly furiously bold |\n" - + "19|Supplier#000000019|edZT3es,nBFD8lBXTGeTl|24|34-278-310-2731|6150.38|refully final foxes across the dogged theodolites sleep slyly abou|\n" - + "20|Supplier#000000020|iybAE,RmTymrZVYaFZva2SH,j|3|13-715-945-6730|530.82|n, ironic ideas would nag blithely about the slyly regular accounts. silent, expr|\n" - + "21|Supplier#000000021|81CavellcrJ0PQ3CPBID0Z0JwyJm0ka5igEs|2|12-253-590-5816|9365.80|d. instructions integrate sometimes slyly pending instructions. accounts nag among the |\n" - + "22|Supplier#000000022|okiiQFk 8lm6EVX6Q0,bEcO|4|14-144-830-2814|-966.20| ironically among the deposits. closely expre|\n" - + "23|Supplier#000000023|ssetugTcXc096qlD7 2TL5crEEeS3zk|9|19-559-422-5776|5926.41|ges could have to are ironic deposits. regular, even request|\n" - + "24|Supplier#000000024|C4nPvLrVmKPPabFCj|0|10-620-939-2254|9170.71|usly pending deposits. slyly final accounts run |\n" - + "25|Supplier#000000025|RCQKONXMFnrodzz6w7fObFVV6CUm2q|22|32-431-945-3541|9198.31|ely regular deposits. carefully regular sauternes engage furiously above the regular accounts. idly |\n" - + "26|Supplier#000000026|iV,MHzAx6Z939uzFNkq09M0a1 MBfH7|21|31-758-894-4436|21.18| ideas poach carefully after the blithely bold asymptotes. furiously pending theodoli|\n" - + "27|Supplier#000000027|lC4CjKwNHUr6L4xIpzOBK4NlHkFTg|18|28-708-999-2028|1887.62|s according to the quickly regular hockey playe|\n" - + "28|Supplier#000000028|GBhvoRh,7YIN V|0|10-538-384-8460|-891.99|ld requests across the pinto beans are carefully against the quickly final courts. accounts sleep |\n" - + "29|Supplier#000000029|658tEqXLPvRd6xpFdqC2|1|11-555-705-5922|-811.62|y express ideas play furiously. even accounts sleep fluffily across the accounts. careful|\n" - + "30|Supplier#000000030|84NmC1rmQfO0fj3zkobLT|16|26-940-594-4852|8080.14|ias. carefully silent accounts cajole blithely. pending, special accounts cajole quickly above the f|\n" - + "31|Supplier#000000031|fRJimA7zchyApqRLHcQeocVpP|16|26-515-530-4159|5916.91|into beans wake after the special packages. slyly fluffy requests cajole furio|\n" - + "32|Supplier#000000032|yvoD3TtZSx1skQNCK8agk5bZlZLug|23|33-484-637-7873|3556.47|usly even depths. quickly ironic theodolites s|\n" - + "33|Supplier#000000033|gfeKpYw3400L0SDywXA6Ya1Qmq1w6YB9f3R|7|17-138-897-9374|8564.12|n sauternes along the regular asymptotes are regularly along the |\n" - + "34|Supplier#000000034|mYRe3KvA2O4lL4HhxDKkkrPUDPMKRCSp,Xpa|10|20-519-982-2343|237.31|eposits. slyly final deposits toward the slyly regular dependencies sleep among the excu|\n" - + "35|Supplier#000000035|QymmGXxjVVQ5OuABCXVVsu,4eF gU0Qc6|21|31-720-790-5245|4381.41| ironic deposits! final, bold platelets haggle quickly quickly pendin|\n" - + "36|Supplier#000000036|mzSpBBJvbjdx3UKTW3bLFewRD78D91lAC879|13|23-273-493-3679|2371.51|ular theodolites must haggle regular, bold accounts. slyly final pinto beans bo|\n" - + "37|Supplier#000000037|cqjyB5h1nV|0|10-470-144-1330|3017.47|iously final instructions. quickly special accounts hang fluffily above the accounts. deposits|\n" - + "38|Supplier#000000038|xEcx45vD0FXHT7c9mvWFY|4|14-361-296-6426|2512.41|ins. fluffily special accounts haggle slyly af|\n" - + "39|Supplier#000000039|ZM, nSYpEPWr1yAFHaC91qjFcijjeU5eH|8|18-851-856-5633|6115.65|le slyly requests. special packages shall are blithely. slyly unusual packages sleep |\n" - + "40|Supplier#000000040|zyIeWzbbpkTV37vm1nmSGBxSgd2Kp|22|32-231-247-6991|-290.06| final patterns. accounts haggle idly pas|\n"; - - private static final String NATION = "" - + "0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|\n" - + "1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|\n" - + "2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |\n" - + "3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|\n" - + "4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|\n" - + "5|ETHIOPIA|0|ven packages wake quickly. regu|\n" - + "6|FRANCE|3|refully final requests. regular, ironi|\n" - + "7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|\n" - + "8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|\n" - + "9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|\n" - + "10|IRAN|4|efully alongside of the slyly final dependencies. |\n" - + "11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|\n" - + "12|JAPAN|2|ously. final, express gifts cajole a|\n" - + "13|JORDAN|4|ic deposits are blithely about the carefully regular pa|\n" - + "14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|\n" - + "15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|\n" - + "16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|\n" - + "17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|\n" - + "18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|\n" - + "19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|\n" - + "20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|\n" - + "21|VIETNAM|2|hely enticingly express accounts. even, final |\n" - + "22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|\n" - + "23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|\n" - + "24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|\n"; - - private static final String EXPECTED_RESULT = "" - + "CHINA|1992|30814.46\n" - + "CHINA|1993|30830.309\n" - + "CHINA|1995|18476.965\n" - + "CHINA|1996|36566.742\n" - + "IRAN|1992|37970.953\n" - + "IRAN|1993|83140.0\n" - + "IRAN|1996|9672.556\n"; - - public TPCHQuery9ITCase(){ - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - partInputPath = createTempFile("part", PART); - partSuppInputPath = createTempFile("partSupp", PARTSUPP); - ordersInputPath = createTempFile("orders", ORDERS); - lineItemInputPath = createTempFile("lineItem", LINEITEM); - supplierInputPath = createTempFile("supplier", SUPPLIER); - nationInputPath = createTempFile("nation", NATION); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - TPCHQuery9 tpch9 = new TPCHQuery9(); - return tpch9.getPlan( - Integer.valueOf(parallelism).toString(), - partInputPath, - partSuppInputPath, - ordersInputPath, - lineItemInputPath, - supplierInputPath, - nationInputPath, - resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java deleted file mode 100644 index 881bd2c9f5a22..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.relational.TPCHQueryAsterix; -import org.apache.flink.test.util.RecordAPITestBase; - -// ----------------------------------------------------------------------------- -// --- NOTE --- -// -// This class contains test data generated by tools from by the -// Transaction Processing Council (TPC), specifically the TPC-H benchmark's -// data generator. -// -// Any form of use and redistribution must happen in accordance with the TPC-H -// Software License Agreement. -// -// For details, see http://www.tpc.org/tpch/dbgen/tpc-h%20license%20agreement.pdf -// ----------------------------------------------------------------------------- - -public class TPCHQueryAsterixITCase extends RecordAPITestBase { - - private String ordersPath; - private String custPath; - private String resultPath; - - private static final String ORDERS = - "1|1|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |\n" - + "2|6|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|\n" - + "3|2|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|\n" - + "4|8|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|\n" - + "5|8|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|\n" - + "6|1|F|58749.59|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|\n" - + "7|4|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |\n" - + "32|9|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|\n" - + "33|8|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|\n" - + "34|10|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|\n" - + "35|2|O|253724.56|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|\n" - + "36|9|O|68289.96|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|\n" - + "37|1|F|206680.66|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|\n" - + "38|7|O|82500.05|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|\n" - + "39|1|O|341734.47|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|\n" - + "64|4|F|39414.99|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|\n" - + "65|2|P|110643.60|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|\n" - + "66|3|F|103740.67|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|\n"; - - private static final String CUSTOMERS = - "1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|\n"+ - "2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|\n"+ - "3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|\n"+ - "4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|\n"+ - "5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|\n"+ - "6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|\n"+ - "7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|\n"+ - "8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|\n"+ - "9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|\n"+ - "10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|\n"; - - private static final String EXPECTED_RESULT = - "7|BUILDING\n" + - "1|HOUSEHOLD\n" + - "6|AUTOMOBILE\n" + - "2|MACHINERY\n" + - "2|FURNITURE\n"; - - public TPCHQueryAsterixITCase(){ - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - ordersPath = createTempFile("orders", ORDERS); - custPath = createTempFile("customers", CUSTOMERS); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - TPCHQueryAsterix tpchBench = new TPCHQueryAsterix(); - return tpchBench.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, custPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java deleted file mode 100644 index a45db2ffe4f85..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.recordJobTests; - -import java.io.File; -import java.io.FileInputStream; -import java.net.URI; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.sort.TeraSort; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.Assert; - - -public class TeraSortITCase extends RecordAPITestBase { - - private static final String INPUT_DATA_FILE = "/testdata/terainput.txt"; - - private String resultPath; - - public TeraSortITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - String testDataPath = getClass().getResource(INPUT_DATA_FILE).toString(); - - TeraSort ts = new TeraSort(); - return ts.getPlan(Integer.valueOf(parallelism).toString(), testDataPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - final byte[] line = new byte[100]; - final byte[] previous = new byte[10]; - for (int i = 0; i < previous.length; i++) { - previous[i] = -128; - } - - File parent = new File(new URI(resultPath).getPath()); - int num = 1; - while (true) { - File next = new File(parent, String.valueOf(num)); - if (!next.exists()) { - break; - } - FileInputStream inStream = new FileInputStream(next); - int read; - while ((read = inStream.read(line)) == 100) { - // check against the previous - for (int i = 0; i < previous.length; i++) { - if (line[i] > previous[i]) { - break; - } else if (line[i] < previous[i]) { - Assert.fail("Next record is smaller than previous record."); - } - } - - System.arraycopy(line, 0, previous, 0, 10); - } - - if (read != -1) { - Assert.fail("Inclomplete last record in result file."); - } - inStream.close(); - - num++; - } - - if (num == 1) { - Assert.fail("Empty result, nothing checked for Job!"); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java deleted file mode 100644 index f69567b7bff5f..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.relational.WebLogAnalysis; -import org.apache.flink.test.util.RecordAPITestBase; - -public class WebLogAnalysisITCase extends RecordAPITestBase { - - protected String docsPath; - protected String ranksPath; - protected String visitsPath; - protected String resultPath; - - private static final String docs = - "url_10|volutpat magna quis consectetuer volutpat ad erat editors exerci oscillations euismod volutpat Lorem convectionullamcorper Lorem volutpat enim tation elit |\n" + - "url_11|adipiscing enim diam ex tincidunt tincidunt nonummy volutpat minim euismod volutpat suscipit ex sed laoreet aliquip quis diam tincidunt wisi diam elit sed ut minim ad nonummy amet volutpat nostrud erat |\n" + - "url_12|ut nostrud adipiscing adipiscing ipsum nonummy amet volutpat volutpat sit enim Ut amet |\n" + - "url_13|euismod sit adipiscing ex suscipit ea veniam tincidunt laoreet nibh editors ullamcorper consectetuer convection commodo sed nostrud ea ex ullamcorper dolor dolore ad diam dolore amet ut tincidunt nonummy euismod enim ullamcorper tincidunt dolor sit volutpat dolor tincidunt aliquam nisl tation ullamcorper sed consectetuer sit sit laoreet ex |\n" + - "url_14|ad magna ipsum nonummy aliquip dolore aliquam veniam lobortis nostrud aliquip nibh amet aliquam editors magna aliquam volutpat nonummy sed tation erat adipiscing nostrud magna ut sit dolore volutpat laoreet nisl wisi Ut veniam nibh laoreet ad nostrud ut aliquip |\n" + - "url_15|enim ipsum veniam ex editor Lorem elit laoreet exerci ea wisi oscillations convection euismod euismod diam ut euismod ad Lorem ut Ut tation wisi diam suscipit nibh nostrud minim dolor |\n" + - "url_16|tation ad sed veniam lobortis editor nonummy Ut ea ipsum aliquip dolore ut laoreet tation ad ut Lorem ipsum minim nostrud quis Lorem enim |\n" + - "url_17|veniam editor veniam Lorem aliquam ipsum amet ut convection veniam enim commodo ad ex magna nibh Ut ex sed ut nostrud amet volutpat nibh Ut wisi ea laoreet Lorem amet minim ullamcorper tincidunt veniam laoreet laoreet |\n" + - "url_18|veniam diam tincidunt ullamcorper adipiscing oscillations adipiscing aliquip quis nibh suscipit ex sit sit wisi diam aliquam aliquip ea diam euismod ad erat ut ipsum lobortis ea exerci |\n" + - "url_19|adipiscing nisl oscillations ea editors tincidunt tation convection tincidunt tincidunt ut wisi tincidunt ut ut Ut nibh aliquam laoreet exerci enim ea nibh erat nonummy |\n" + - "url_20|ut diam aliquip ipsum laoreet elit volutpat suscipit nostrud convection tation nisl suscipit nonummy tation tation ut enim dolor nisl magna aliquam enim consectetuer sed tincidunt quis amet elit |\n" + - "url_21|tation ad exerci volutpat tincidunt sit ullamcorper oscillations ut minim sed diam Lorem adipiscing |\n" + - "url_22|sit exerci ad ut minim convection ad ea dolore Lorem ipsum amet sit wisi sed ullamcorper ipsum enim aliquam tincidunt elit nonummy laoreet laoreet dolore tation ullamcorper commodo Ut elit sit minim suscipit nisl laoreet |\n" + - "url_23|Lorem nibh ea ea ex ut tation euismod tincidunt lobortis enim ullamcorper euismod amet magna sit erat enim editor diam dolor volutpat diam nisl ut erat commodo amet veniam consectetuer ex commodo sed magna ea erat ullamcorper dolor sed diam enim euismod |\n" + - "url_24|nisl dolor magna editors ad consectetuer veniam commodo Ut dolor wisi dolore amet dolore dolore volutpat convection Ut oscillations |\n" + - "url_25|ut euismod quis ad nostrud volutpat dolor wisi oscillations nostrud sed nonummy wisi exerci convection sed erat nostrud quis editors Ut nostrud consectetuer aliquip exerci ut Ut tincidunt aliquam suscipit erat |\n" + - "url_26|nisl elit ea minim aliquam dolor consectetuer consectetuer quis Ut convection laoreet sit enim nostrud sed dolor magna dolor elit adipiscing |\n" + - "url_27|nisl euismod ipsum ex adipiscing erat euismod diam quis oscillations aliquip nisl ut sit dolor wisi enim tincidunt amet ullamcorper adipiscing nibh Ut volutpat sed nonummy ex ea wisi exerci aliquam elit Ut aliquip nostrud ad nibh ut sit suscipit ut commodo wisi Lorem nibh |\n" + - "url_28|laoreet sed editor dolor diam convection sed diam exerci laoreet oscillations consectetuer ullamcorper suscipit ut editors quis commodo editor veniam nibh ea diam ex magna ea elit sit sed nibh lobortis consectetuer erat erat sit minim ea ad ea nisl magna volutpat ut |\n" + - "url_29|consectetuer convection amet diam erat euismod erat editor oscillations ipsum tation tation editors minim wisi tation quis adipiscing euismod nonummy erat ut diam suscipit tincidunt ut consectetuer Lorem editor sit quis euismod veniam tincidunt aliquam ut veniam adipiscing magna Ut ut dolore euismod minim veniam |"; - - private static final String ranks = - "77|url_10|51|\n" + "7|url_11|24|\n" + "19|url_12|28|\n" + "13|url_13|50|\n" + - "81|url_14|41|\n" + "87|url_15|28|\n" + "78|url_16|29|\n" + "70|url_17|7|\n" + - "91|url_18|38|\n" + "99|url_19|11|\n" + "37|url_20|46|\n" + "64|url_21|46|\n" + - "25|url_22|14|\n" + "6|url_23|12|\n" + "87|url_24|39|\n" + "0|url_25|27|\n" + - "97|url_26|27|\n" + "9|url_27|54|\n" + "59|url_28|41|\n" + "40|url_29|11|"; - - private static final String visits = - "112.99.215.248|url_20|2011-5-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "189.69.147.166|url_19|2009-8-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "83.7.97.153|url_21|2010-3-20|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "215.57.124.59|url_10|2008-1-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "174.176.121.10|url_25|2007-12-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "66.225.121.11|url_16|2007-10-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "57.11.211.126|url_14|2010-12-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "129.127.115.129|url_28|2009-2-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "137.249.175.115|url_27|2011-5-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "101.92.9.112|url_12|2012-1-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "113.108.207.12|url_24|2011-3-20|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "179.141.87.220|url_18|2007-10-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "46.10.118.101|url_20|2009-7-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "91.67.84.155|url_20|2012-7-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "102.142.34.164|url_22|2010-7-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "203.67.94.61|url_14|2007-7-27|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "229.121.13.143|url_19|2010-4-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "209.154.189.12|url_16|2011-4-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "57.113.29.197|url_13|2008-9-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "132.90.139.12|url_16|2007-10-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "101.21.109.146|url_29|2009-2-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "171.69.34.255|url_29|2009-2-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "11.114.192.13|url_25|2008-9-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "37.188.212.141|url_15|2011-1-21|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "102.40.251.47|url_26|2010-7-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "136.85.230.245|url_17|2010-8-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "68.96.194.82|url_29|2007-10-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "62.126.40.210|url_29|2011-2-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "12.81.184.191|url_11|2008-1-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "12.232.205.52|url_18|2008-1-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "227.83.79.30|url_25|2008-6-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "88.222.17.152|url_21|2007-10-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "55.87.23.63|url_22|2008-8-16|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "54.41.77.25|url_18|2012-12-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "203.83.250.233|url_10|2009-4-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "42.13.134.196|url_15|2012-9-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "224.51.182.133|url_27|2008-11-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "180.209.212.98|url_27|2008-6-27|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "29.173.47.29|url_24|2011-9-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "251.153.164.73|url_18|2008-3-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "195.164.244.58|url_23|2010-12-21|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "96.167.244.51|url_18|2007-2-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "131.169.48.126|url_19|2009-6-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "171.151.109.83|url_12|2011-6-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "150.87.176.48|url_22|2008-10-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "56.172.201.54|TPCHQuery3ITCaseurl_25|2012-11-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "175.241.83.44|url_13|2008-2-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "105.222.1.251|url_18|2012-7-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "195.117.130.95|url_10|2012-4-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "10.139.236.176|url_14|2011-4-9|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "207.100.168.191|url_22|2007-5-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "196.129.126.177|url_25|2008-4-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "170.239.116.200|url_27|2010-3-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "99.79.118.85|url_13|2008-3-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "112.65.132.135|url_15|2010-8-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "109.15.235.196|url_14|2012-8-25|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "240.201.229.137|url_21|2008-9-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "74.53.245.171|url_18|2007-6-19|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "158.230.101.250|url_17|2011-4-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "198.197.213.210|url_11|2012-5-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "1.54.145.90|url_11|2009-6-16|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "2.236.14.86|url_13|2007-9-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "203.180.186.150|url_16|2010-2-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "135.83.247.101|url_22|2007-12-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "54.214.72.16|url_12|2007-5-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "238.115.186.34|url_10|2007-5-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "6.251.55.92|url_26|2007-5-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "196.96.11.111|url_16|2007-11-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "209.40.86.220|url_24|2009-5-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "130.208.66.140|url_19|2011-9-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "176.157.170.178|url_27|2008-12-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "103.4.119.128|url_15|2010-6-19|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "31.181.160.159|url_12|2009-8-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "166.238.60.79|url_23|2009-2-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "64.165.114.139|url_19|2007-9-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "11.45.238.55|url_26|2011-8-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "189.121.92.228|url_18|2008-1-16|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "137.114.181.208|url_14|2007-6-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "165.194.234.136|url_18|2012-3-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "210.145.85.234|url_10|2007-9-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "227.37.135.138|url_21|2010-5-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "39.87.120.86|url_25|2007-2-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "107.144.159.246|url_11|2010-7-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "160.38.166.236|url_15|2009-6-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "79.44.39.160|url_10|2008-12-19|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "143.52.62.175|url_19|2011-12-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "68.54.137.193|url_19|2007-6-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "199.137.128.158|url_27|2008-4-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "10.0.80.61|url_10|2011-3-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "4.30.48.164|url_29|2007-2-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "136.29.50.223|url_12|2008-3-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "83.101.200.110|url_23|2009-1-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "11.149.83.246|url_29|2010-9-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "154.119.20.134|url_14|2011-1-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "191.124.167.123|url_15|2009-6-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "27.158.167.38|url_11|2011-5-16|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "16.197.156.206|url_16|2009-2-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "222.150.206.233|url_20|2012-2-11|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "198.59.29.225|url_21|2007-2-25|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "182.91.244.40|url_18|2010-5-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "23.110.45.72|url_28|2008-6-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "94.241.8.36|url_11|2007-7-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "226.223.202.210|url_17|2007-10-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "73.16.194.132|url_14|2012-7-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "195.45.98.110|url_13|2012-2-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "189.123.80.178|url_29|2011-1-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "190.134.174.18|url_23|2011-12-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "133.54.252.245|url_13|2011-3-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "244.29.167.182|url_18|2012-10-11|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "125.32.251.232|url_26|2010-4-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "74.139.30.20|url_16|2012-2-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "100.73.180.209|url_29|2010-4-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "133.155.158.147|url_16|2011-11-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "185.53.212.102|url_15|2010-12-19|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "154.189.125.92|url_27|2007-10-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "12.209.166.107|url_26|2007-5-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "106.99.152.22|url_25|2011-12-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "37.120.32.53|url_22|2008-12-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "29.17.125.185|url_12|2011-8-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "233.125.12.14|url_19|2011-2-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "241.78.212.114|url_25|2007-12-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "153.183.247.90|url_18|2011-9-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "245.43.17.177|url_15|2012-4-21|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "55.13.54.220|url_24|2008-11-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "104.149.82.217|url_10|2011-9-26|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "36.15.167.145|url_24|2008-1-9|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "145.149.81.203|url_20|2008-5-27|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "68.89.4.115|url_18|2007-7-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "236.88.134.53|url_15|2010-5-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "180.140.58.209|url_29|2012-8-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "138.46.241.237|url_17|2010-11-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "254.217.79.58|url_29|2007-1-11|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "142.184.197.136|url_10|2007-11-3|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "216.118.94.220|url_12|2007-2-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "53.99.54.0|url_14|2008-3-9|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "95.151.221.37|url_28|2009-5-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "46.110.43.78|url_27|2007-11-9|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "172.96.203.180|url_11|2012-1-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "22.188.231.143|url_21|2007-10-23|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "216.46.80.239|url_22|2009-9-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "103.213.7.86|url_26|2007-12-25|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "186.204.3.180|url_27|2012-1-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "129.197.116.192|url_10|2012-5-1|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "8.164.181.242|url_24|2008-11-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "145.52.12.132|url_28|2012-8-6|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "20.22.62.146|url_11|2007-4-27|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "233.255.32.151|url_19|2008-6-25|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "31.233.226.148|url_24|2011-8-4|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "57.130.120.117|url_25|2008-12-9|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "29.98.249.152|url_21|2011-8-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "25.187.66.122|url_20|2008-12-17|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "236.205.44.215|url_26|2008-9-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "18.40.99.157|url_23|2012-1-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "204.221.88.206|url_24|2012-11-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "199.5.94.240|url_24|2011-9-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "57.175.34.24|url_14|2009-12-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "130.25.116.220|url_10|2007-6-5|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "134.174.12.250|url_22|2011-6-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "162.185.114.212|url_13|2011-8-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "8.113.137.95|url_14|2011-6-22|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "176.253.5.99|url_12|2010-4-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "3.110.24.0|url_12|2009-6-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "159.128.152.207|url_17|2012-2-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "104.145.150.244|url_10|2012-2-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "203.19.224.207|url_12|2012-9-24|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "225.163.125.108|url_20|2010-4-8|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "197.137.29.71|url_21|2007-10-25|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "29.207.144.144|url_12|2012-7-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "139.222.222.113|url_14|2010-4-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "169.225.20.141|url_29|2012-2-18|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "137.86.141.104|url_26|2012-3-10|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "193.100.2.122|url_24|2011-12-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "20.105.209.161|url_13|2010-2-7|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "116.81.52.207|url_11|2007-4-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "141.88.224.116|url_19|2009-4-12|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "72.198.58.37|url_26|2012-2-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "13.37.92.90|url_22|2011-5-16|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "191.163.173.142|url_24|2012-5-2|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "183.57.175.105|url_27|2007-8-13|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "230.212.34.87|url_10|2010-12-21|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + - "133.150.217.96|url_24|2012-7-27|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n" + "62.254.96.239|url_24|2011-5-15|0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|"; - - private static final String expected = "87|url_24|39\n" + "59|url_28|41\n"; - - public WebLogAnalysisITCase(){ - setTaskManagerNumSlots(parallelism); - } - - @Override - protected void preSubmit() throws Exception { - docsPath = createTempFile("docs", docs); - ranksPath = createTempFile("ranks", ranks); - visitsPath = createTempFile("visits", visits); - resultPath = getTempDirPath("results"); - } - - @Override - protected Plan getTestJob() { - WebLogAnalysis relOLAP = new WebLogAnalysis(); - return relOLAP.getPlan(Integer.valueOf(parallelism).toString(), docsPath, ranksPath, visitsPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expected, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java deleted file mode 100644 index 19f3decbb809b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.recordJobTests; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.test.recordJobs.wordcount.WordCount; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.RecordAPITestBase; - -public class WordCountITCase extends RecordAPITestBase { - - protected String textPath; - protected String resultPath; - - public WordCountITCase(){ - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - textPath = createTempFile("text.txt", WordCountData.TEXT); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - WordCount wc = new WordCount(); - return wc.getPlan(Integer.valueOf(parallelism).toString(), textPath, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - // Test results - compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java deleted file mode 100644 index 4518568250fcc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.recordJobTests; - -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.wordcount.WordCount.CountWords; -import org.apache.flink.test.recordJobs.wordcount.WordCount.TokenizeLine; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.RecordAPITestBase; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.StringValue; - -/** - * WordCount with multiple inputs to the reducer. - *

- * This test case is an adaption of issue #192 (and #124), which revealed problems with the union readers in Nephele. - * The problems have been fixed with commit 1228a5e. Without this commit the test will deadlock. - */ -@SuppressWarnings("deprecation") -public class WordCountUnionReduceITCase extends RecordAPITestBase { - - private static final int MULTIPLY = 1000; - - private String inputPath; - - private String outputPath; - - public WordCountUnionReduceITCase(){ - setTaskManagerNumSlots(parallelism); - } - - - @Override - protected void preSubmit() throws Exception { - // the fixed input is repeated this many times and the expected counts - // are multiplied by this factor, because the problem only occurs with - // inputs of a certain size - String input = repeatString(WordCountData.TEXT, MULTIPLY); - - this.inputPath = createTempFile("input.txt", input); - this.outputPath = getTempDirPath("output"); - } - - @Override - protected Plan getTestJob() { - WordCountUnionReduce wc = new WordCountUnionReduce(); - return wc.getPlan(this.inputPath, this.outputPath, parallelism); - } - - @Override - protected void postSubmit() throws Exception { - String expectedCounts = - multiplyIntegersInString(WordCountData.COUNTS, - // adjust counts to string repetition (InputSizeFactor) and two mappers (*2) - MULTIPLY * 2); - compareResultsByLinesInMemory(expectedCounts, this.outputPath); - } - - /** - * This is the adapted plan from issue #192. - */ - private class WordCountUnionReduce { - - /** - *

-		 *                   +-------------+
-		 *              //=> | MapOperator | =\\
-		 * +--------+  //    +-------------+   \\   +----------------+    +------+
-		 * | Source | =|                        |=> | ReduceOperator | => | Sink |
-		 * +--------+  \\    +-------------+   //   +----------------+    +------+
-		 *              \\=> | MapOperator | =//
-		 *                   +-------------+
-		 * 
- */ - public Plan getPlan(String inputPath, String outputPath, int numSubtasks) { - - FileDataSource source = new FileDataSource(TextInputFormat.class, inputPath, "First Input"); - - MapOperator wordsFirstInput = MapOperator.builder(TokenizeLine.class) - .input(source) - .name("Words (First Input)") - .build(); - - MapOperator wordsSecondInput = MapOperator.builder(TokenizeLine.class) - .input(source) - .name("Words (Second Input)") - .build(); - - @SuppressWarnings("unchecked") - ReduceOperator counts = ReduceOperator.builder(CountWords.class, StringValue.class, 0) - .input(wordsFirstInput, wordsSecondInput) - .name("Word Counts") - .build(); - - FileDataSink sink = new FileDataSink(CsvOutputFormat.class, outputPath, counts); - CsvOutputFormat.configureRecordFormat(sink) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(sink, "WordCount Union Reduce"); - plan.setDefaultParallelism(numSubtasks); - - return plan; - } - } - - /** - * Repeats the given String and returns the resulting String. - * - * @param str - * the string to repeat - * @param n - * the number of times to repeat the string - * @return repeated string if n > 1, otherwise the input string - */ - private String repeatString(String str, int n) { - if (n <= 1) { - return str; - } - - StringBuilder sb = new StringBuilder(str.length() * n + 1); - for (int i = 0; i < n; i++) { - sb.append(str); - } - return sb.toString(); - } - - /** - * Returns a new String with all occurring integers multiplied. - * - * @param str - * the string which contains integers to multiply - * @param n - * the factor to multiply each integer with - * @return new string with multiplied integers - */ - private String multiplyIntegersInString(String str, int n) { - Pattern counts = Pattern.compile("(\\d+)"); - Matcher matcher = counts.matcher(str); - - StringBuffer sb = new StringBuffer(str.length()); - - boolean hasMatch = false; - - while (matcher.find()) { - hasMatch = true; - matcher.appendReplacement(sb, String.valueOf(n * Integer.parseInt(matcher.group(1)))); - } - - return hasMatch ? sb.toString() : str; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java deleted file mode 100644 index 9ec8c7366c38c..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java +++ /dev/null @@ -1,231 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.EdgeInputFormat; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.EdgeWithDegreesOutputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class ComputeEdgeDegrees implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - // -------------------------------------------------------------------------------------------- - // Vertex Degree Computation - // -------------------------------------------------------------------------------------------- - - /** - * Duplicates each edge such that: (u,v) becomes (u,v),(v,u) - */ - public static final class ProjectEdge extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record copy = new Record(); - - @Override - public void map(Record record, Collector out) throws Exception { - this.copy.setField(0, record.getField(1, IntValue.class)); - this.copy.setField(1, record.getField(0, IntValue.class)); - - out.collect(this.copy); - out.collect(record); - } - } - - /** - * Creates for all records in the group a record of the form (v1, v2, c1, c2), where - * v1 is the lexicographically smaller vertex id and the count for the vertex that - * was the key contains the number of edges associated with it. The other count is zero. - * This reducer also eliminates duplicate edges. - */ - public static final class CountEdges extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record result = new Record(); - - private final IntValue firstVertex = new IntValue(); - private final IntValue secondVertex = new IntValue(); - private final IntValue firstCount = new IntValue(); - private final IntValue secondCount = new IntValue(); - - private int[] vals = new int[1024]; - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - int[] vals = this.vals; - int len = 0; - int key = -1; - - // collect all values - while (records.hasNext()) { - final Record rec = records.next(); - final int id = rec.getField(1, IntValue.class).getValue(); - if (key == -1) { - key = rec.getField(0, IntValue.class).getValue(); - } - - if (len >= vals.length) { - vals = new int[vals.length * 2]; - System.arraycopy(this.vals, 0, vals, 0, this.vals.length); - this.vals = vals; - } - vals[len++] = id; - } - - // sort the values to and uniquify them - Arrays.sort(vals, 0, len); - int k = 0; - for (int curr = -1, i = 0; i < len; i++) { - int val = vals[i]; - if (val != curr) { - curr = val; - vals[k] = vals[i]; - k++; - } - else { - vals[k] = vals[i]; - } - } - len = k; - - // create such that the vertex with the lower id is always the first - // both vertices contain a count, which is zero for the non-key vertices - for (int i = 0; i < len; i++) { - final int e2 = vals[i]; - if (key <= e2) { - firstVertex.setValue(key); - secondVertex.setValue(e2); - firstCount.setValue(len); - secondCount.setValue(0); - } else { - firstVertex.setValue(e2); - secondVertex.setValue(key); - firstCount.setValue(0); - secondCount.setValue(len); - } - this.result.setField(0, firstVertex); - this.result.setField(1, secondVertex); - this.result.setField(2, firstCount); - this.result.setField(3, secondCount); - out.collect(result); - } - } - } - - /** - * Takes the two separate edge entries (v1, v2, c1, 0) and (v1, v2, 0, c2) - * and creates an entry (v1, v2, c1, c2). - */ - public static final class JoinCountsAndUniquify extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue count1 = new IntValue(); - private final IntValue count2 = new IntValue(); - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - Record rec = null; - int c1 = 0, c2 = 0; - int numValues = 0; - - while (records.hasNext()) { - rec = records.next(); - final int f1 = rec.getField(2, IntValue.class).getValue(); - final int f2 = rec.getField(3, IntValue.class).getValue(); - c1 += f1; - c2 += f2; - numValues++; - } - - if (numValues != 2 || c1 == 0 || c2 == 0) { - throw new RuntimeException("JoinCountsAndUniquify Problem: key1=" + - rec.getField(0, IntValue.class).getValue() + ", key2=" + - rec.getField(1, IntValue.class).getValue() + - "values=" + numValues + ", c1=" + c1 + ", c2=" + c2); - } - - count1.setValue(c1); - count2.setValue(c2); - rec.setField(2, count1); - rec.setField(3, count2); - out.collect(rec); - } - } - - // -------------------------------------------------------------------------------------------- - // Triangle Enumeration - // -------------------------------------------------------------------------------------------- - - /** - * Assembles the Plan of the triangle enumeration example Pact program. - */ - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String edgeInput = args.length > 1 ? args[1] : ""; - final String output = args.length > 2 ? args[2] : ""; - final char delimiter = args.length > 3 ? (char) Integer.parseInt(args[3]) : ','; - - - FileDataSource edges = new FileDataSource(new EdgeInputFormat(), edgeInput, "Input Edges"); - edges.setParameter(EdgeInputFormat.ID_DELIMITER_CHAR, delimiter); - - MapOperator projectEdge = MapOperator.builder(new ProjectEdge()) - .input(edges).name("Project Edge").build(); - - ReduceOperator edgeCounter = ReduceOperator.builder(new CountEdges(), IntValue.class, 0) - .input(projectEdge).name("Count Edges for Vertex").build(); - - ReduceOperator countJoiner = ReduceOperator.builder(new JoinCountsAndUniquify()) - .keyField(IntValue.class, 0) - .keyField(IntValue.class, 1) - .input(edgeCounter) - .name("Join Counts") - .build(); - - FileDataSink triangles = new FileDataSink(new EdgeWithDegreesOutputFormat(), output, countJoiner, "Unique Edges With Degrees"); - - Plan p = new Plan(triangles, "Normalize Edges and compute Vertex Degrees"); - p.setDefaultParallelism(numSubTasks); - return p; - } - - @Override - public String getDescription() { - return "Parameters: [noSubStasks] [input file] [output file] [vertex delimiter]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java deleted file mode 100644 index cc6fd01ce8ea2..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.java.record.functions.CoGroupFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.CoGroupOperator; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; -import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class ConnectedComponentsWithCoGroup implements Program { - - private static final long serialVersionUID = 1L; - - @ConstantFieldsFirst(0) - @ConstantFieldsSecond(0) - public static final class MinIdAndUpdate extends CoGroupFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final LongValue newComponentId = new LongValue(); - - @Override - public void coGroup(Iterator candidates, Iterator current, Collector out) throws Exception { - if (!current.hasNext()) { - throw new Exception("Error: Id not encountered before."); - } - Record old = current.next(); - long oldId = old.getField(1, LongValue.class).getValue(); - - long minimumComponentID = Long.MAX_VALUE; - - while (candidates.hasNext()) { - long candidateComponentID = candidates.next().getField(1, LongValue.class).getValue(); - if (candidateComponentID < minimumComponentID) { - minimumComponentID = candidateComponentID; - } - } - - if (minimumComponentID < oldId) { - newComponentId.setValue(minimumComponentID); - old.setField(1, newComponentId); - out.collect(old); - } - } - - } - - @SuppressWarnings("unchecked") - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String verticesInput = (args.length > 1 ? args[1] : ""); - final String edgeInput = (args.length > 2 ? args[2] : ""); - final String output = (args.length > 3 ? args[3] : ""); - final int maxIterations = (args.length > 4 ? Integer.parseInt(args[4]) : 1); - - // data source for initial vertices - FileDataSource initialVertices = new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices"); - - MapOperator verticesWithId = MapOperator.builder(DuplicateLongMap.class).input(initialVertices).name("Assign Vertex Ids").build(); - - DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration"); - iteration.setInitialSolutionSet(verticesWithId); - iteration.setInitialWorkset(verticesWithId); - iteration.setMaximumNumberOfIterations(maxIterations); - - // create DataSourceContract for the edges - FileDataSource edges = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges"); - - // create CrossOperator for distance computation - JoinOperator joinWithNeighbors = JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(edges) - .name("Join Candidate Id With Neighbor") - .build(); - - CoGroupOperator minAndUpdate = CoGroupOperator.builder(new MinIdAndUpdate(), LongValue.class, 0, 0) - .input1(joinWithNeighbors) - .input2(iteration.getSolutionSet()) - .name("Min Id and Update") - .build(); - - iteration.setNextWorkset(minAndUpdate); - iteration.setSolutionSetDelta(minAndUpdate); - - // create DataSinkContract for writing the new cluster positions - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, iteration, "Result"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(LongValue.class, 0) - .field(LongValue.class, 1); - - // return the PACT plan - Plan plan = new Plan(result, "Workset Connected Components"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java deleted file mode 100644 index 81adbf565589a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class DeltaPageRankWithInitialDeltas implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - - @ConstantFieldsSecond(0) - public static final class RankComparisonMatch extends JoinFunction { - - private static final long serialVersionUID = 1L; - - private final DoubleValue newRank = new DoubleValue(); - - @Override - public void join(Record vertexWithDelta, Record vertexWithOldRank, Collector out) throws Exception { - DoubleValue deltaVal = vertexWithDelta.getField(1, DoubleValue.class); - DoubleValue currentVal = vertexWithOldRank.getField(1, DoubleValue.class); - - newRank.setValue(deltaVal.getValue() + currentVal.getValue()); - vertexWithOldRank.setField(1, newRank); - - out.collect(vertexWithOldRank); - } - } - - @Combinable - @ConstantFields(0) - public static final class UpdateRankReduceDelta extends ReduceFunction { - - private static final long serialVersionUID = 1L; - - private final DoubleValue newRank = new DoubleValue(); - - @Override - public void reduce(Iterator records, Collector out) { - - double rankSum = 0.0; - double rank; - Record rec = null; - - while (records.hasNext()) { - rec = records.next(); - rank = rec.getField(1, DoubleValue.class).getValue(); - rankSum += rank; - } - - // ignore small deltas - if (Math.abs(rankSum) > 0.00001) { - newRank.setValue(rankSum); - rec.setField(1, newRank); - out.collect(rec); - } - } - } - - public class PRDependenciesComputationMatchDelta extends JoinFunction { - - private static final long serialVersionUID = 1L; - - private final Record result = new Record(); - private final DoubleValue partRank = new DoubleValue(); - - /* - * (srcId, trgId, weight) x (vId, rank) => (trgId, rank / weight) - */ - @Override - public void join(Record vertexWithRank, Record edgeWithWeight, Collector out) throws Exception { - result.setField(0, edgeWithWeight.getField(1, LongValue.class)); - final long outLinks = edgeWithWeight.getField(2, LongValue.class).getValue(); - final double rank = vertexWithRank.getField(1, DoubleValue.class).getValue(); - partRank.setValue(rank / (double) outLinks); - result.setField(1, partRank); - - out.collect(result); - } - } - - @SuppressWarnings("unchecked") - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String solutionSetInput = (args.length > 1 ? args[1] : ""); - final String deltasInput = (args.length > 2 ? args[2] : ""); - final String dependencySetInput = (args.length > 3 ? args[3] : ""); - final String output = (args.length > 4 ? args[4] : ""); - final int maxIterations = (args.length > 5 ? Integer.parseInt(args[5]) : 1); - - // create DataSourceContract for the initalSolutionSet - FileDataSource initialSolutionSet = new FileDataSource(new CsvInputFormat(' ', LongValue.class, DoubleValue.class), solutionSetInput, "Initial Solution Set"); - - // create DataSourceContract for the initalDeltaSet - FileDataSource initialDeltaSet = new FileDataSource(new CsvInputFormat(' ', LongValue.class, DoubleValue.class), deltasInput, "Initial DeltaSet"); - - // create DataSourceContract for the edges - FileDataSource dependencySet = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class), dependencySetInput, "Dependency Set"); - - DeltaIteration iteration = new DeltaIteration(0, "Delta PageRank"); - iteration.setInitialSolutionSet(initialSolutionSet); - iteration.setInitialWorkset(initialDeltaSet); - iteration.setMaximumNumberOfIterations(maxIterations); - - JoinOperator dependenciesMatch = JoinOperator.builder(PRDependenciesComputationMatchDelta.class, - LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(dependencySet) - .name("calculate dependencies") - .build(); - - ReduceOperator updateRanks = ReduceOperator.builder(UpdateRankReduceDelta.class, LongValue.class, 0) - .input(dependenciesMatch) - .name("update ranks") - .build(); - - JoinOperator oldRankComparison = JoinOperator.builder(RankComparisonMatch.class, LongValue.class, 0, 0) - .input1(updateRanks) - .input2(iteration.getSolutionSet()) - .name("comparison with old ranks") - .build(); - - iteration.setNextWorkset(updateRanks); - iteration.setSolutionSetDelta(oldRankComparison); - - // create DataSinkContract for writing the final ranks - FileDataSink result = new FileDataSink(CsvOutputFormat.class, output, iteration, "Final Ranks"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(LongValue.class, 0) - .field(DoubleValue.class, 1); - - // return the PACT plan - Plan plan = new Plan(result, "Delta PageRank"); - plan.setDefaultParallelism(numSubTasks); - return plan; - - } - - - @Override - public String getDescription() { - return "Parameters: "; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java deleted file mode 100644 index b2328bae95375..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.EdgeWithDegreesInputFormat; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.TriangleOutputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -/** - * An implementation of the triangle enumeration, which expects its input to - * encode the degrees of the vertices. The algorithm selects the lower-degree vertex for the - * enumeration of open triads. - */ -@SuppressWarnings("deprecation") -public class EnumTrianglesOnEdgesWithDegrees implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - // -------------------------------------------------------------------------------------------- - // Triangle Enumeration - // -------------------------------------------------------------------------------------------- - - public static final class ProjectOutCounts extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - record.setNumFields(2); - out.collect(record); - } - } - - public static final class ProjectToLowerDegreeVertex extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - final int d1 = record.getField(2, IntValue.class).getValue(); - final int d2 = record.getField(3, IntValue.class).getValue(); - if (d1 > d2) { - IntValue first = record.getField(1, IntValue.class); - IntValue second = record.getField(0, IntValue.class); - record.setField(0, first); - record.setField(1, second); - } - record.setNumFields(2); - out.collect(record); - } - } - - public static final class BuildTriads extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue firstVertex = new IntValue(); - private final IntValue secondVertex = new IntValue(); - - private int[] edgeCache = new int[1024]; - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - int len = 0; - - Record rec = null; - while (records.hasNext()) { - rec = records.next(); - final int e1 = rec.getField(1, IntValue.class).getValue(); - - for (int i = 0; i < len; i++) { - final int e2 = this.edgeCache[i]; - - if (e1 <= e2) { - firstVertex.setValue(e1); - secondVertex.setValue(e2); - } else { - firstVertex.setValue(e2); - secondVertex.setValue(e1); - } - - rec.setField(1, firstVertex); - rec.setField(2, secondVertex); - out.collect(rec); - } - - if (len >= this.edgeCache.length) { - int[] na = new int[this.edgeCache.length * 2]; - System.arraycopy(this.edgeCache, 0, na, 0, this.edgeCache.length); - this.edgeCache = na; - } - this.edgeCache[len++] = e1; - } - } - } - - public static class CloseTriads extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - @Override - public void join(Record triangle, Record missingEdge, Collector out) throws Exception { - out.collect(triangle); - } - } - - /** - * Assembles the Plan of the triangle enumeration example Pact program. - */ - @Override - public Plan getPlan(String... args) { - - // parse job parameters - int numSubTasks = args.length > 0 ? Integer.parseInt(args[0]) : 1; - String edgeInput = args.length > 1 ? args[1] : ""; - String output = args.length > 2 ? args[2] : ""; - - FileDataSource edges = new FileDataSource(new EdgeWithDegreesInputFormat(), edgeInput, "Input Edges with Degrees"); - edges.setParameter(EdgeWithDegreesInputFormat.VERTEX_DELIMITER_CHAR, '|'); - edges.setParameter(EdgeWithDegreesInputFormat.DEGREE_DELIMITER_CHAR, ','); - - // =========================== Triangle Enumeration ============================ - - MapOperator toLowerDegreeEdge = MapOperator.builder(new ProjectToLowerDegreeVertex()) - .input(edges) - .name("Select lower-degree Edge") - .build(); - - MapOperator projectOutCounts = MapOperator.builder(new ProjectOutCounts()) - .input(edges) - .name("Project to vertex Ids only") - .build(); - - ReduceOperator buildTriads = ReduceOperator.builder(new BuildTriads(), IntValue.class, 0) - .input(toLowerDegreeEdge) - .name("Build Triads") - .build(); - - JoinOperator closeTriads = JoinOperator.builder(new CloseTriads(), IntValue.class, 1, 0) - .keyField(IntValue.class, 2, 1) - .input1(buildTriads) - .input2(projectOutCounts) - .name("Close Triads") - .build(); - closeTriads.setParameter("INPUT_SHIP_STRATEGY", "SHIP_REPARTITION_HASH"); - closeTriads.setParameter("LOCAL_STRATEGY", "LOCAL_STRATEGY_HASH_BUILD_SECOND"); - - FileDataSink triangles = new FileDataSink(new TriangleOutputFormat(), output, closeTriads, "Triangles"); - - Plan p = new Plan(triangles, "Enumerate Triangles"); - p.setDefaultParallelism(numSubTasks); - return p; - } - - @Override - public String getDescription() { - return "Parameters: [noSubStasks] [input file] [output file]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java deleted file mode 100644 index ba8b54a8fb2dc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java +++ /dev/null @@ -1,293 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirstExcept; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -/** - * Implementation of the triangle enumeration example Pact program. - * The program expects a file with RDF triples (in XML serialization) as input. Triples must be separated by linebrakes. - * - * The program filters for foaf:knows predicates to identify relationships between two entities (typically persons). - * Relationships are interpreted as edges in a social graph. Then the program enumerates all triangles which are build - * by edges in that graph. - * - * Usually, triangle enumeration is used as a pre-processing step to identify highly connected subgraphs. - * The algorithm was published as MapReduce job by J. Cohen in "Graph Twiddling in a MapReduce World". - * The Pact version was described in "MapReduce and PACT - Comparing Data Parallel Programming Models" (BTW 2011). - */ -@SuppressWarnings("deprecation") -public class EnumTrianglesRdfFoaf implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - /** - * Reads RDF triples and filters on the foaf:knows RDF predicate. - * The foaf:knows RDF predicate indicates that the RDF subject and object (typically of type foaf:person) know each - * other. - * Therefore, knowing connections between people are extracted and handles as graph edges. - * The EdgeInFormat filters all rdf triples with foaf:knows predicates. The subjects and objects URLs are - * compared. - * The lexicographically smaller URL is set as the first field of the output record, the greater one as the second field. - */ - public static class EdgeInFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final StringValue rdfSubj = new StringValue(); - private final StringValue rdfPred = new StringValue(); - private final StringValue rdfObj = new StringValue(); - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - final int limit = offset + numBytes; - int startPos = offset; - - // read RDF subject - startPos = parseVarLengthEncapsulatedStringField(bytes, startPos, limit, ' ', rdfSubj, '"'); - if (startPos < 0) { - // invalid record, exit - return null; - } - // read RDF predicate - startPos = parseVarLengthEncapsulatedStringField(bytes, startPos, limit, ' ', rdfPred, '"'); - if (startPos < 0 || !rdfPred.getValue().equals("")) { - // invalid record or predicate is not a foaf-knows predicate, exit - return null; - } - // read RDF object - startPos = parseVarLengthEncapsulatedStringField(bytes, startPos, limit, ' ', rdfObj, '"'); - if (startPos < 0) { - // invalid record, exit - return null; - } - - // compare RDF subject and object - if (rdfSubj.compareTo(rdfObj) <= 0) { - // subject is smaller, subject becomes first attribute, object second - target.setField(0, rdfSubj); - target.setField(1, rdfObj); - } else { - // object is smaller, object becomes first attribute, subject second - target.setField(0, rdfObj); - target.setField(1, rdfSubj); - } - - return target; - } - - /* - * Utility method to efficiently parse encapsulated, variable length strings - */ - private int parseVarLengthEncapsulatedStringField(byte[] bytes, int startPos, int limit, char delim, StringValue field, char encaps) { - - boolean isEncaps = false; - - // check whether string is encapsulated - if (bytes[startPos] == encaps) { - isEncaps = true; - } - - if (isEncaps) { - // string is encapsulated - for (int i = startPos; i < limit; i++) { - if (bytes[i] == encaps) { - if (bytes[i+1] == delim) { - field.setValueAscii(bytes, startPos, i-startPos+1); - return i+2; - } - } - } - return -1; - } else { - // string is not encapsulated - int i; - for (i = startPos; i < limit; i++) { - if (bytes[i] == delim) { - field.setValueAscii(bytes, startPos, i-startPos); - return i+1; - } - } - if (i == limit) { - field.setValueAscii(bytes, startPos, i-startPos); - return i+1; - } else { - return -1; - } - } - } - } - - /** - * Builds triads (open triangle) from all two edges that share a vertex. - * The common vertex is - */ - @ConstantFields(0) - public static class BuildTriads extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - // list of non-matching vertices - private final ArrayList otherVertices = new ArrayList(32); - - // matching vertex - private final StringValue matchVertex = new StringValue(); - - // mutable output record - private final Record result = new Record(); - - // initialize list of non-matching vertices for one vertex - public BuildTriads() { - this.otherVertices.add(new StringValue()); - } - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - // read the first edge - final Record rec = records.next(); - // read the matching vertex - rec.getFieldInto(0, this.matchVertex); - // read the non-matching vertex and add it to the list - rec.getFieldInto(1, this.otherVertices.get(0)); - - // set the matching vertex in the output record - this.result.setField(0, this.matchVertex); - - int numEdges = 1; - // while there are more edges - while (records.hasNext()) { - - // read the next edge - final Record next = records.next(); - - final StringValue myVertex; - // obtain an object to store the non-matching vertex - if (numEdges >= this.otherVertices.size()) { - // we need an additional vertex object - // create the object - myVertex = new StringValue(); - // and put it in the list - this.otherVertices.add(myVertex); - } else { - // we reuse a previously created object from the list - myVertex = this.otherVertices.get(numEdges); - } - // read the non-matching vertex into the obtained object - next.getFieldInto(1, myVertex); - - // combine the current edge with all vertices in the non-matching vertex list - for (int i = 0; i < numEdges; i++) { - // get the other non-matching vertex - final StringValue otherVertex = this.otherVertices.get(i); - // add my and other vertex to the output record depending on their ordering - if (otherVertex.compareTo(myVertex) < 0) { - this.result.setField(1, otherVertex); - this.result.setField(2, myVertex); - out.collect(this.result); - } else { - next.setField(2, otherVertex); - out.collect(next); - } - } - - numEdges++; - } - } - } - - /** - * Matches all missing edges with existing edges from input. - * If the missing edge for a triad is found, the triad is transformed to a triangle by adding the missing edge. - */ - @ConstantFieldsFirstExcept({}) - public static class CloseTriads extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void join(Record triad, Record missingEdge, Collector out) throws Exception { - // emit triangle (already contains missing edge at field 0 - out.collect(triad); - } - } - - /** - * Assembles the Plan of the triangle enumeration example Pact program. - */ - @Override - public Plan getPlan(String... args) { - - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String edgeInput = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - FileDataSource edges = new FileDataSource(new EdgeInFormat(), edgeInput, "BTC Edges"); - - ReduceOperator buildTriads = ReduceOperator.builder(new BuildTriads(), StringValue.class, 0) - .name("Build Triads") - .build(); - - JoinOperator closeTriads = JoinOperator.builder(new CloseTriads(), StringValue.class, 1, 0) - .keyField(StringValue.class, 2, 1) - .name("Close Triads") - .build(); - closeTriads.setParameter("INPUT_LEFT_SHIP_STRATEGY", "SHIP_REPARTITION_HASH"); - closeTriads.setParameter("INPUT_RIGHT_SHIP_STRATEGY", "SHIP_REPARTITION_HASH"); - closeTriads.setParameter("LOCAL_STRATEGY", "LOCAL_STRATEGY_HASH_BUILD_SECOND"); - - FileDataSink triangles = new FileDataSink(new CsvOutputFormat(), output, "Output"); - CsvOutputFormat.configureRecordFormat(triangles) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(StringValue.class, 1) - .field(StringValue.class, 2); - - triangles.setInput(closeTriads); - closeTriads.setSecondInput(edges); - closeTriads.setFirstInput(buildTriads); - buildTriads.setInput(edges); - - Plan plan = new Plan(triangles, "Enumerate Triangles"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: [numSubStasks] [inputRDFTriples] [outputTriangles]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesWithDegrees.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesWithDegrees.java deleted file mode 100644 index dc521584236ac..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesWithDegrees.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.recordJobs.graph; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.graph.ComputeEdgeDegrees.CountEdges; -import org.apache.flink.test.recordJobs.graph.ComputeEdgeDegrees.JoinCountsAndUniquify; -import org.apache.flink.test.recordJobs.graph.ComputeEdgeDegrees.ProjectEdge; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.BuildTriads; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.CloseTriads; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.ProjectOutCounts; -import org.apache.flink.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.ProjectToLowerDegreeVertex; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.EdgeInputFormat; -import org.apache.flink.test.recordJobs.graph.triangleEnumUtil.TriangleOutputFormat; -import org.apache.flink.types.IntValue; - -/** - * An implementation of the triangle enumeration, which includes the pre-processing step - * to compute the degrees of the vertices and to select the lower-degree vertex for the - * enumeration of open triads. - */ -@SuppressWarnings("deprecation") -public class EnumTrianglesWithDegrees implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String edgeInput = args.length > 1 ? args[1] : ""; - final String output = args.length > 2 ? args[2] : ""; - final char delimiter = args.length > 3 ? (char) Integer.parseInt(args[3]) : ','; - - - FileDataSource edges = new FileDataSource(new EdgeInputFormat(), edgeInput, "Input Edges"); - edges.setParameter(EdgeInputFormat.ID_DELIMITER_CHAR, delimiter); - - // =========================== Vertex Degree ============================ - - MapOperator projectEdge = MapOperator.builder(new ProjectEdge()) - .input(edges).name("Project Edge").build(); - - ReduceOperator edgeCounter = ReduceOperator.builder(new CountEdges(), IntValue.class, 0) - .input(projectEdge).name("Count Edges for Vertex").build(); - - ReduceOperator countJoiner = ReduceOperator.builder(new JoinCountsAndUniquify(), IntValue.class, 0) - .keyField(IntValue.class, 1) - .input(edgeCounter).name("Join Counts").build(); - - - // =========================== Triangle Enumeration ============================ - - MapOperator toLowerDegreeEdge = MapOperator.builder(new ProjectToLowerDegreeVertex()) - .input(countJoiner).name("Select lower-degree Edge").build(); - - MapOperator projectOutCounts = MapOperator.builder(new ProjectOutCounts()) - .input(countJoiner).name("Project out Counts").build(); - - ReduceOperator buildTriads = ReduceOperator.builder(new BuildTriads(), IntValue.class, 0) - .input(toLowerDegreeEdge).name("Build Triads").build(); - - JoinOperator closeTriads = JoinOperator.builder(new CloseTriads(), IntValue.class, 1, 0) - .keyField(IntValue.class, 2, 1) - .input1(buildTriads).input2(projectOutCounts) - .name("Close Triads").build(); - closeTriads.setParameter("INPUT_SHIP_STRATEGY", "SHIP_REPARTITION_HASH"); - closeTriads.setParameter("LOCAL_STRATEGY", "LOCAL_STRATEGY_HASH_BUILD_SECOND"); - - FileDataSink triangles = new FileDataSink(new TriangleOutputFormat(), output, closeTriads, "Triangles"); - - Plan p = new Plan(triangles, "Enumerate Triangles"); - p.setDefaultParallelism(numSubTasks); - return p; - } - - @Override - public String getDescription() { - return "Parameters: [noSubStasks] [input file] [output file]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java deleted file mode 100644 index 34d4b60775e99..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java +++ /dev/null @@ -1,435 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.IOException; -import java.io.Serializable; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; -import java.util.StringTokenizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.CoGroupFunction; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.api.java.record.operators.CoGroupOperator; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -/** - * Implementation of the Pairwise Shortest Path example PACT program. - * The program implements one iteration of the algorithm and must be run multiple times until no changes are computed. - * - * The pairwise shortest path algorithm comes from the domain graph problems. The goal is to find all shortest paths - * between any two transitively connected nodes in a graph. In this implementation edges are interpreted as directed and weighted. - * - * For the first iteration, the program allows two input formats: - * 1) RDF triples with foaf:knows predicates. A triple is interpreted as an edge from the RDF subject to the RDF object with weight 1. - * 2) The programs text-serialization for paths (see @see PathInFormat and @see PathOutFormat). - * - * The RDF input format is used if the 4th parameter of the getPlan() method is set to "true". If set to "false" the path input format is used. - */ -@SuppressWarnings("deprecation") -public class PairwiseSP implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - /** - * Reads RDF triples and filters on the foaf:knows RDF predicate. The triples elements must be separated by whitespaces. - * The foaf:knows RDF predicate indicates that the RDF subject knows the object (typically of type foaf:person). - * The connections between people are extracted and handles as graph edges. For the Pairwise Shortest Path algorithm the - * connection is interpreted as a directed edge, i.e. subject knows object, but the object does not necessarily know the subject. - * - * The RDFTripleInFormat filters all RDF triples with foaf:knows predicates. - * For each triple with foaf:knows predicate, a record is emitted with - * - from-node being the RDF subject at field position 0, - * - to-node being the RDF object at field position 1, - * - length being 1 at field position 2, and - * - hopList being an empty string at field position 3. - * - */ - public static class RDFTripleInFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final StringValue fromNode = new StringValue(); - private final StringValue toNode = new StringValue(); - private final IntValue pathLength = new IntValue(1); - private final IntValue hopCnt = new IntValue(0); - private final StringValue hopList = new StringValue(" "); - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - String lineStr = new String(bytes, offset, numBytes); - // replace reduce whitespaces and trim - lineStr = lineStr.replaceAll("\\s+", " ").trim(); - // build whitespace tokenizer - StringTokenizer st = new StringTokenizer(lineStr, " "); - - // line must have at least three elements - if (st.countTokens() < 3) { - return null; - } - - String rdfSubj = st.nextToken(); - String rdfPred = st.nextToken(); - String rdfObj = st.nextToken(); - - // we only want foaf:knows predicates - if (!rdfPred.equals("")) { - return null; - } - - // build node pair from subject and object - fromNode.setValue(rdfSubj); - toNode.setValue(rdfObj); - - target.setField(0, fromNode); - target.setField(1, toNode); - target.setField(2, pathLength); - target.setField(3, hopCnt); - target.setField(4, hopList); - - return target; - } - } - - /** - * The PathInFormat reads paths consisting of a from-node a to-node, a length, and hop node list serialized as a string. - * All four elements of the path must be separated by the pipe character ('|') and may not contain any pipe characters itself. - * - * PathInFormat returns records with: - * - from-node at field position 0, - * - to-node at field position 1, - * - length at field position 2, - * - hop list at field position 3. - */ - public static class PathInFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final StringValue fromNode = new StringValue(); - private final StringValue toNode = new StringValue(); - private final IntValue length = new IntValue(); - private final IntValue hopCnt = new IntValue(); - private final StringValue hopList = new StringValue(); - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - String lineStr = new String(bytes, offset, numBytes); - StringTokenizer st = new StringTokenizer(lineStr, "|"); - - // path must have exactly 5 tokens (fromNode, toNode, length, hopCnt, hopList) - if (st.countTokens() != 5) { - return null; - } - - this.fromNode.setValue(st.nextToken()); - this.toNode.setValue(st.nextToken()); - this.length.setValue(Integer.parseInt(st.nextToken())); - this.hopCnt.setValue(Integer.parseInt(st.nextToken())); - this.hopList.setValue(st.nextToken()); - - target.setField(0, fromNode); - target.setField(1, toNode); - target.setField(2, length); - target.setField(3, hopCnt); - target.setField(4, hopList); - - return target; - } - } - - /** - * The PathOutFormat serializes paths to text. - * In order, the from-node, the to-node, the length, the hop list are written out. - * Elements are separated by the pipe character ('|'). - * - * - */ - public static class PathOutFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - @Override - public void writeRecord(Record record) throws IOException { - StringBuilder line = new StringBuilder(); - - // append from-node - line.append(record.getField(0, StringValue.class).toString()); - line.append("|"); - // append to-node - line.append(record.getField(1, StringValue.class).toString()); - line.append("|"); - // append length - line.append(record.getField(2, IntValue.class).toString()); - line.append("|"); - // append hopCnt - line.append(record.getField(3, IntValue.class).toString()); - line.append("|"); - // append hopList - line.append(record.getField(4, StringValue.class).toString()); - line.append("|"); - line.append("\n"); - - stream.write(line.toString().getBytes()); - } - } - - /** - * Concatenates two paths where the from-node of the first path and the to-node of the second path are the same. - * The second input path becomes the first part and the first input path the second part of the output path. - * The length of the output path is the sum of both input paths. - * The output path's hops list is built from both path's hops lists and the common node. - */ - @ConstantFieldsFirst(1) - @ConstantFieldsSecond(0) - public static class ConcatPaths extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record outputRecord = new Record(); - - private final IntValue length = new IntValue(); - private final IntValue hopCnt = new IntValue(); - private final StringValue hopList = new StringValue(); - - @Override - public void join(Record rec1, Record rec2, Collector out) throws Exception { - - // rec1 has matching start, rec2 matching end - // Therefore, rec2's end node and rec1's start node are identical - // First half of new path will be rec2, second half will be rec1 - - // Get from-node and to-node of new path - final StringValue fromNode = rec2.getField(0, StringValue.class); - final StringValue toNode = rec1.getField(1, StringValue.class); - - // Check whether from-node = to-node to prevent circles! - if (fromNode.equals(toNode)) { - return; - } - - // Create new path - outputRecord.setField(0, fromNode); - outputRecord.setField(1, toNode); - - // Compute length of new path - length.setValue(rec1.getField(2, IntValue.class).getValue() + rec2.getField(2, IntValue.class).getValue()); - outputRecord.setField(2, length); - - // compute hop count - int hops = rec1.getField(3, IntValue.class).getValue() + 1 + rec2.getField(3, IntValue.class).getValue(); - hopCnt.setValue(hops); - outputRecord.setField(3, hopCnt); - - // Concatenate hops lists and insert matching node - StringBuilder sb = new StringBuilder(); - // first path - sb.append(rec2.getField(4, StringValue.class).getValue()); - sb.append(" "); - // common node - sb.append(rec1.getField(0, StringValue.class).getValue()); - // second path - sb.append(" "); - sb.append(rec1.getField(4, StringValue.class).getValue()); - - hopList.setValue(sb.toString().trim()); - outputRecord.setField(4, hopList); - - out.collect(outputRecord); - } - } - - /** - * Gets two lists of paths as input and emits for each included from-node/to-node combination the shortest path(s). - * If for a combination more than one shortest path exists, all shortest paths are emitted. - * - * - */ - @ConstantFieldsFirst({0,1}) - @ConstantFieldsSecond({0,1}) - public static class FindShortestPath extends CoGroupFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record outputRecord = new Record(); - - private final Set shortestPaths = new HashSet(); - private final Map hopCnts = new HashMap(); - private final IntValue minLength = new IntValue(); - - @Override - public void coGroup(Iterator inputRecords, Iterator concatRecords, Collector out) { - - // init minimum length and minimum path - Record pathRec = null; - StringValue path = null; - if(inputRecords.hasNext()) { - // path is in input paths - pathRec = inputRecords.next(); - } else { - // path must be in concat paths - pathRec = concatRecords.next(); - } - // get from node (common for all paths) - StringValue fromNode = pathRec.getField(0, StringValue.class); - // get to node (common for all paths) - StringValue toNode = pathRec.getField(1, StringValue.class); - // get length of path - minLength.setValue(pathRec.getField(2, IntValue.class).getValue()); - // store path and hop count - path = new StringValue(pathRec.getField(4, StringValue.class)); - shortestPaths.add(path); - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); - - // find shortest path of all input paths - while (inputRecords.hasNext()) { - pathRec = inputRecords.next(); - IntValue length = pathRec.getField(2, IntValue.class); - - if (length.getValue() == minLength.getValue()) { - // path has also minimum length add to list - path = new StringValue(pathRec.getField(4, StringValue.class)); - if(shortestPaths.add(path)) { - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); - } - } else if (length.getValue() < minLength.getValue()) { - // path has minimum length - minLength.setValue(length.getValue()); - // clear lists - hopCnts.clear(); - shortestPaths.clear(); - // get path and add path and hop count - path = new StringValue(pathRec.getField(4, StringValue.class)); - shortestPaths.add(path); - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); - } - } - - // find shortest path of all input and concatenated paths - while (concatRecords.hasNext()) { - pathRec = concatRecords.next(); - IntValue length = pathRec.getField(2, IntValue.class); - - if (length.getValue() == minLength.getValue()) { - // path has also minimum length add to list - path = new StringValue(pathRec.getField(4, StringValue.class)); - if(shortestPaths.add(path)) { - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); - } - } else if (length.getValue() < minLength.getValue()) { - // path has minimum length - minLength.setValue(length.getValue()); - // clear lists - hopCnts.clear(); - shortestPaths.clear(); - // get path and add path and hop count - path = new StringValue(pathRec.getField(4, StringValue.class)); - shortestPaths.add(path); - hopCnts.put(path, new IntValue(pathRec.getField(3, IntValue.class).getValue())); - } - } - - outputRecord.setField(0, fromNode); - outputRecord.setField(1, toNode); - outputRecord.setField(2, minLength); - - // emit all shortest paths - for(StringValue shortestPath : shortestPaths) { - outputRecord.setField(3, hopCnts.get(shortestPath)); - outputRecord.setField(4, shortestPath); - out.collect(outputRecord); - } - - hopCnts.clear(); - shortestPaths.clear(); - - } - } - - /** - * Assembles the Plan of the Pairwise Shortest Paths example Pact program. - * The program computes one iteration of the Pairwise Shortest Paths algorithm. - * - * For the first iteration, two input formats can be chosen: - * 1) RDF triples with foaf:knows predicates - * 2) Text-serialized paths (see PathInFormat and PathOutFormat) - * - * To choose 1) set the forth parameter to "true". If set to "false" 2) will be used. - * - */ - @Override - public Plan getPlan(String... args) { - - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String paths = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - boolean rdfInput = (args.length > 3 && Boolean.parseBoolean(args[3])); - - FileDataSource pathsInput; - - if(rdfInput) { - pathsInput = new FileDataSource(new RDFTripleInFormat(), paths, "RDF Triples"); - } else { - pathsInput = new FileDataSource(new PathInFormat(), paths, "Paths"); - } - pathsInput.setParallelism(numSubTasks); - - JoinOperator concatPaths = - JoinOperator.builder(new ConcatPaths(), StringValue.class, 0, 1) - .name("Concat Paths") - .build(); - - concatPaths.setParallelism(numSubTasks); - - CoGroupOperator findShortestPaths = - CoGroupOperator.builder(new FindShortestPath(), StringValue.class, 0, 0) - .keyField(StringValue.class, 1, 1) - .name("Find Shortest Paths") - .build(); - findShortestPaths.setParallelism(numSubTasks); - - FileDataSink result = new FileDataSink(new PathOutFormat(),output, "New Paths"); - result.setParallelism(numSubTasks); - - result.setInput(findShortestPaths); - findShortestPaths.setFirstInput(pathsInput); - findShortestPaths.setSecondInput(concatPaths); - concatPaths.setFirstInput(pathsInput); - concatPaths.setSecondInput(pathsInput); - - return new Plan(result, "Pairwise Shortest Paths"); - - } - - @Override - public String getDescription() { - return "Parameters: [numSubStasks], [inputPaths], [outputPaths], [RDFInputFlag]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java deleted file mode 100644 index 0dbb20a770a90..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * 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.recordJobs.graph; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.DeltaIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class WorksetConnectedComponents implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - public static final class DuplicateLongMap extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - record.setField(1, record.getField(0, LongValue.class)); - out.collect(record); - } - } - - /** - * UDF that joins a (Vertex-ID, Component-ID) pair that represents the current component that - * a vertex is associated with, with a (Source-Vertex-ID, Target-VertexID) edge. The function - * produces a (Target-vertex-ID, Component-ID) pair. - */ - public static final class NeighborWithComponentIDJoin extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record result = new Record(); - - @Override - public void join(Record vertexWithComponent, Record edge, Collector out) { - this.result.setField(0, edge.getField(1, LongValue.class)); - this.result.setField(1, vertexWithComponent.getField(1, LongValue.class)); - out.collect(this.result); - } - } - - /** - * Minimum aggregation over (Vertex-ID, Component-ID) pairs, selecting the pair with the smallest Component-ID. - */ - @Combinable - @ConstantFields(0) - public static final class MinimumComponentIDReduce extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final Record result = new Record(); - private final LongValue vertexId = new LongValue(); - private final LongValue minComponentId = new LongValue(); - - @Override - public void reduce(Iterator records, Collector out) { - - final Record first = records.next(); - final long vertexID = first.getField(0, LongValue.class).getValue(); - - long minimumComponentID = first.getField(1, LongValue.class).getValue(); - - while (records.hasNext()) { - long candidateComponentID = records.next().getField(1, LongValue.class).getValue(); - if (candidateComponentID < minimumComponentID) { - minimumComponentID = candidateComponentID; - } - } - - this.vertexId.setValue(vertexID); - this.minComponentId.setValue(minimumComponentID); - this.result.setField(0, this.vertexId); - this.result.setField(1, this.minComponentId); - out.collect(this.result); - } - } - - /** - * UDF that joins a candidate (Vertex-ID, Component-ID) pair with another (Vertex-ID, Component-ID) pair. - * Returns the candidate pair, if the candidate's Component-ID is smaller. - */ - @ConstantFieldsFirst(0) - public static final class UpdateComponentIdMatch extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void join(Record newVertexWithComponent, Record currentVertexWithComponent, Collector out){ - - long candidateComponentID = newVertexWithComponent.getField(1, LongValue.class).getValue(); - long currentComponentID = currentVertexWithComponent.getField(1, LongValue.class).getValue(); - - if (candidateComponentID < currentComponentID) { - out.collect(newVertexWithComponent); - } - } - } - - @SuppressWarnings("unchecked") - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String verticesInput = (args.length > 1 ? args[1] : ""); - final String edgeInput = (args.length > 2 ? args[2] : ""); - final String output = (args.length > 3 ? args[3] : ""); - final int maxIterations = (args.length > 4 ? Integer.parseInt(args[4]) : 1); - - // data source for initial vertices - FileDataSource initialVertices = new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices"); - - MapOperator verticesWithId = MapOperator.builder(DuplicateLongMap.class).input(initialVertices).name("Assign Vertex Ids").build(); - - // the loop takes the vertices as the solution set and changed vertices as the workset - // initially, all vertices are changed - DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration"); - iteration.setInitialSolutionSet(verticesWithId); - iteration.setInitialWorkset(verticesWithId); - iteration.setMaximumNumberOfIterations(maxIterations); - - // data source for the edges - FileDataSource edges = new FileDataSource(new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges"); - - // join workset (changed vertices) with the edges to propagate changes to neighbors - JoinOperator joinWithNeighbors = JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0) - .input1(iteration.getWorkset()) - .input2(edges) - .name("Join Candidate Id With Neighbor") - .build(); - - // find for each neighbor the smallest of all candidates - ReduceOperator minCandidateId = ReduceOperator.builder(new MinimumComponentIDReduce(), LongValue.class, 0) - .input(joinWithNeighbors) - .name("Find Minimum Candidate Id") - .build(); - - // join candidates with the solution set and update if the candidate component-id is smaller - JoinOperator updateComponentId = JoinOperator.builder(new UpdateComponentIdMatch(), LongValue.class, 0, 0) - .input1(minCandidateId) - .input2(iteration.getSolutionSet()) - .name("Update Component Id") - .build(); - - iteration.setNextWorkset(updateComponentId); - iteration.setSolutionSetDelta(updateComponentId); - - // sink is the iteration result - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, iteration, "Result"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(LongValue.class, 0) - .field(LongValue.class, 1); - - Plan plan = new Plan(result, "Workset Connected Components"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java deleted file mode 100644 index 81f4d006237f1..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.recordJobs.graph.triangleEnumUtil; - -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - - -/** - * - */ -public final class EdgeInputFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - public static final String ID_DELIMITER_CHAR = "edgeinput.delimiter"; - - private final IntValue i1 = new IntValue(); - private final IntValue i2 = new IntValue(); - - private char delimiter; - - // -------------------------------------------------------------------------------------------- - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - final int limit = offset + numBytes; - int first = 0, second = 0; - final char delimiter = this.delimiter; - - int pos = offset; - while (pos < limit && bytes[pos] != delimiter) { - first = first * 10 + (bytes[pos++] - '0'); - } - pos += 1;// skip the delimiter - while (pos < limit) { - second = second * 10 + (bytes[pos++] - '0'); - } - - if (first <= 0 || second <= 0 || first == second) { - return null; - } - - this.i1.setValue(first); - this.i2.setValue(second); - target.setField(0, this.i1); - target.setField(1, this.i2); - return target; - } - - // -------------------------------------------------------------------------------------------- - - - @Override - public void configure(Configuration parameters) { - super.configure(parameters); - this.delimiter = (char) parameters.getInteger(ID_DELIMITER_CHAR, ','); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java deleted file mode 100644 index 8441602f81d91..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.recordJobs.graph.triangleEnumUtil; - -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - - -/** - * Input format that reads edges augmented with vertex degrees. The data to be read is assumed to be in - * the format v1,d1|v2,d2\n, where v1 and v2 are the IDs of the first and - * second vertex, while d1 and d2 are the vertex degrees. - *

- * The result record holds the fields in the sequence (v1, v2, d1, d2). - *

- * The delimiters are configurable. The default delimiter between vertex ID and - * vertex degree is the comma (,). The default delimiter between the two vertices is - * the vertical bar (|). - */ -public final class EdgeWithDegreesInputFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - public static final String VERTEX_DELIMITER_CHAR = "edgeinput.vertexdelimiter"; - public static final String DEGREE_DELIMITER_CHAR = "edgeinput.degreedelimiter"; - - private final IntValue v1 = new IntValue(); - private final IntValue v2 = new IntValue(); - private final IntValue d1 = new IntValue(); - private final IntValue d2 = new IntValue(); - - private char vertexDelimiter; - private char degreeDelimiter; - - // -------------------------------------------------------------------------------------------- - - @Override - public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) { - final int limit = offset + numBytes; - int firstV = 0, secondV = 0; - int firstD = 0, secondD = 0; - - final char vertexDelimiter = this.vertexDelimiter; - final char degreeDelimiter = this.degreeDelimiter; - - int pos = offset; - - // read the first vertex ID - while (pos < limit && bytes[pos] != degreeDelimiter) { - firstV = firstV * 10 + (bytes[pos++] - '0'); - } - - pos += 1;// skip the delimiter - - // read the first vertex degree - while (pos < limit && bytes[pos] != vertexDelimiter) { - firstD = firstD * 10 + (bytes[pos++] - '0'); - } - - pos += 1;// skip the delimiter - - // read the second vertex ID - while (pos < limit && bytes[pos] != degreeDelimiter) { - secondV = secondV * 10 + (bytes[pos++] - '0'); - } - - pos += 1;// skip the delimiter - - // read the second vertex degree - while (pos < limit) { - secondD = secondD * 10 + (bytes[pos++] - '0'); - } - - if (firstV <= 0 || secondV <= 0 || firstV == secondV) { - return null; - } - - v1.setValue(firstV); - v2.setValue(secondV); - d1.setValue(firstD); - d2.setValue(secondD); - - target.setField(0, v1); - target.setField(1, v2); - target.setField(2, d1); - target.setField(3, d2); - - return target; - } - - // -------------------------------------------------------------------------------------------- - - - @Override - public void configure(Configuration parameters) { - super.configure(parameters); - this.vertexDelimiter = (char) parameters.getInteger(VERTEX_DELIMITER_CHAR, '|'); - this.degreeDelimiter = (char) parameters.getInteger(DEGREE_DELIMITER_CHAR, ','); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java deleted file mode 100644 index a72a355471406..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.recordJobs.graph.triangleEnumUtil; - -import org.apache.flink.api.java.record.io.DelimitedOutputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - - -/** - * - */ -public final class EdgeWithDegreesOutputFormat extends DelimitedOutputFormat { - private static final long serialVersionUID = 1L; - - private final StringBuilder line = new StringBuilder(); - - @Override - public int serializeRecord(Record rec, byte[] target) throws Exception { - final int e1 = rec.getField(0, IntValue.class).getValue(); - final int e2 = rec.getField(1, IntValue.class).getValue(); - final int e3 = rec.getField(2, IntValue.class).getValue(); - final int e4 = rec.getField(3, IntValue.class).getValue(); - - this.line.setLength(0); - this.line.append(e1); - this.line.append(','); - this.line.append(e3); - this.line.append('|'); - this.line.append(e2); - this.line.append(','); - this.line.append(e4); - - if (target.length >= line.length()) { - for (int i = 0; i < line.length(); i++) { - target[i] = (byte) line.charAt(i); - } - return line.length(); - } - else { - return -line.length(); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java deleted file mode 100644 index f6c27b0983708..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.recordJobs.graph.triangleEnumUtil; - -import org.apache.flink.api.java.record.io.DelimitedOutputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - - -/** - * - */ -public final class TriangleOutputFormat extends DelimitedOutputFormat { - private static final long serialVersionUID = 1L; - - private final StringBuilder line = new StringBuilder(); - - @Override - public int serializeRecord(Record rec, byte[] target) throws Exception { - final int e1 = rec.getField(0, IntValue.class).getValue(); - final int e2 = rec.getField(1, IntValue.class).getValue(); - final int e3 = rec.getField(2, IntValue.class).getValue(); - - this.line.setLength(0); - this.line.append(e1); - this.line.append(','); - this.line.append(e2); - this.line.append(','); - this.line.append(e3); - - if (target.length >= line.length()) { - for (int i = 0; i < line.length(); i++) { - target[i] = (byte) line.charAt(i); - } - return line.length(); - } else { - return -line.length(); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java deleted file mode 100644 index d528f5d8a4ae9..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java +++ /dev/null @@ -1,324 +0,0 @@ -/* - * 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.recordJobs.kmeans; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.client.LocalExecutor; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class KMeansBroadcast implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) { - // parse job parameters - int parallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataPointInput = (args.length > 1 ? args[1] : ""); - String clusterInput = (args.length > 2 ? args[2] : ""); - String output = (args.length > 3 ? args[3] : ""); - int numIterations = (args.length > 4 ? Integer.parseInt(args[4]) : 2); - - // data source data point input - @SuppressWarnings("unchecked") - FileDataSource pointsSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class), dataPointInput, "Data Points"); - - // data source for cluster center input - @SuppressWarnings("unchecked") - FileDataSource clustersSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class), clusterInput, "Centers"); - - MapOperator dataPoints = MapOperator.builder(new PointBuilder()).name("Build data points").input(pointsSource).build(); - - MapOperator clusterPoints = MapOperator.builder(new PointBuilder()).name("Build cluster points").input(clustersSource).build(); - - // ---------------------- Begin K-Means Loop --------------------- - - BulkIteration iter = new BulkIteration("k-means loop"); - iter.setInput(clusterPoints); - iter.setMaximumNumberOfIterations(numIterations); - - // compute the distances and select the closest center - MapOperator findNearestClusterCenters = MapOperator.builder(new SelectNearestCenter()) - .setBroadcastVariable("centers", iter.getPartialSolution()) - .input(dataPoints) - .name("Find Nearest Centers") - .build(); - - // computing the new cluster positions - ReduceOperator recomputeClusterCenter = ReduceOperator.builder(new RecomputeClusterCenter(), IntValue.class, 0) - .input(findNearestClusterCenters) - .name("Recompute Center Positions") - .build(); - - iter.setNextPartialSolution(recomputeClusterCenter); - - // ---------------------- End K-Means Loop --------------------- - - // create DataSinkContract for writing the new cluster positions - FileDataSink newClusterPoints = new FileDataSink(new PointOutFormat(), output, iter, "New Center Positions"); - - Plan plan = new Plan(newClusterPoints, "K-Means"); - plan.setDefaultParallelism(parallelism); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } - - // -------------------------------------------------------------------------------------------- - // Data Types and UDFs - // -------------------------------------------------------------------------------------------- - - /** - * A simple three-dimensional point. - */ - public static final class Point implements Value { - private static final long serialVersionUID = 1L; - - public double x, y, z; - - public Point() {} - - public Point(double x, double y, double z) { - this.x = x; - this.y = y; - this.z = z; - } - - public void add(Point other) { - x += other.x; - y += other.y; - z += other.z; - } - - public Point div(long val) { - x /= val; - y /= val; - z /= val; - return this; - } - - public double euclideanDistance(Point other) { - return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y) + (z-other.z)*(z-other.z)); - } - - public void clear() { - x = y = z = 0.0; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeDouble(x); - out.writeDouble(y); - out.writeDouble(z); - } - - @Override - public void read(DataInputView in) throws IOException { - x = in.readDouble(); - y = in.readDouble(); - z = in.readDouble(); - } - - @Override - public String toString() { - return "(" + x + "|" + y + "|" + z + ")"; - } - } - - public static final class PointWithId { - - public int id; - public Point point; - - public PointWithId(int id, Point p) { - this.id = id; - this.point = p; - } - } - - /** - * Determines the closest cluster center for a data point. - */ - public static final class SelectNearestCenter extends MapFunction { - private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - private final Record result = new Record(3); - - private List centers = new ArrayList(); - - /** - * Reads all the center values from the broadcast variable into a collection. - */ - @Override - public void open(Configuration parameters) throws Exception { - List clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); - - centers.clear(); - synchronized (clusterCenters) { - for (Record r : clusterCenters) { - centers.add(new PointWithId(r.getField(0, IntValue.class).getValue(), r.getField(1, Point.class))); - } - } - } - - /** - * Computes a minimum aggregation on the distance of a data point to cluster centers. - * - * Output Format: - * 0: centerID - * 1: pointVector - * 2: constant(1) (to enable combinable average computation in the following reducer) - */ - @Override - public void map(Record dataPointRecord, Collector out) { - Point p = dataPointRecord.getField(1, Point.class); - - double nearestDistance = Double.MAX_VALUE; - int centerId = -1; - - // check all cluster centers - for (PointWithId center : centers) { - // compute distance - double distance = p.euclideanDistance(center.point); - - // update nearest cluster if necessary - if (distance < nearestDistance) { - nearestDistance = distance; - centerId = center.id; - } - } - - // emit a new record with the center id and the data point. add a one to ease the - // implementation of the average function with a combiner - result.setField(0, new IntValue(centerId)); - result.setField(1, p); - result.setField(2, one); - - out.collect(result); - } - } - - @Combinable - public static final class RecomputeClusterCenter extends ReduceFunction { - private static final long serialVersionUID = 1L; - - private final Point p = new Point(); - - - /** - * Compute the new position (coordinate vector) of a cluster center. - */ - @Override - public void reduce(Iterator points, Collector out) { - Record sum = sumPointsAndCount(points); - sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); - out.collect(sum); - } - - /** - * Computes a pre-aggregated average value of a coordinate vector. - */ - @Override - public void combine(Iterator points, Collector out) { - out.collect(sumPointsAndCount(points)); - } - - private final Record sumPointsAndCount(Iterator dataPoints) { - Record next = null; - p.clear(); - int count = 0; - - // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); - p.add(next.getField(1, Point.class)); - count += next.getField(2, IntValue.class).getValue(); - } - - next.setField(1, p); - next.setField(2, new IntValue(count)); - return next; - } - } - - public static final class PointBuilder extends MapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - double x = record.getField(1, DoubleValue.class).getValue(); - double y = record.getField(2, DoubleValue.class).getValue(); - double z = record.getField(3, DoubleValue.class).getValue(); - - record.setField(1, new Point(x, y, z)); - out.collect(record); - } - } - - public static final class PointOutFormat extends FileOutputFormat { - - private static final long serialVersionUID = 1L; - - private static final String format = "%d|%.1f|%.1f|%.1f|\n"; - - @Override - public void writeRecord(Record record) throws IOException { - int id = record.getField(0, IntValue.class).getValue(); - Point p = record.getField(1, Point.class); - - byte[] bytes = String.format(format, id, p.x, p.y, p.z).getBytes(); - - this.stream.write(bytes); - } - } - - public static void main(String[] args) throws Exception { - System.out.println(LocalExecutor.optimizerPlanAsJSON(new KMeansBroadcast().getPlan("4", "/dev/random", "/dev/random", "/tmp", "20"))); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java deleted file mode 100644 index 8d75d4766f079..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.recordJobs.kmeans; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.operators.BulkIteration; -import org.apache.flink.api.java.record.operators.CrossOperator; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.client.LocalExecutor; -import org.apache.flink.test.recordJobs.kmeans.udfs.ComputeDistance; -import org.apache.flink.test.recordJobs.kmeans.udfs.FindNearestCenter; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat; -import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat; -import org.apache.flink.test.recordJobs.kmeans.udfs.RecomputeClusterCenter; -import org.apache.flink.types.IntValue; - -@SuppressWarnings("deprecation") -public class KMeansCross implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String dataPointInput = (args.length > 1 ? args[1] : ""); - final String clusterInput = (args.length > 2 ? args[2] : ""); - final String output = (args.length > 3 ? args[3] : ""); - final int numIterations = (args.length > 4 ? Integer.parseInt(args[4]) : 1); - - // create DataSourceContract for cluster center input - FileDataSource initialClusterPoints = new FileDataSource(new PointInFormat(), clusterInput, "Centers"); - initialClusterPoints.setParallelism(1); - - BulkIteration iteration = new BulkIteration("K-Means Loop"); - iteration.setInput(initialClusterPoints); - iteration.setMaximumNumberOfIterations(numIterations); - - // create DataSourceContract for data point input - FileDataSource dataPoints = new FileDataSource(new PointInFormat(), dataPointInput, "Data Points"); - - // create CrossOperator for distance computation - CrossOperator computeDistance = CrossOperator.builder(new ComputeDistance()) - .input1(dataPoints) - .input2(iteration.getPartialSolution()) - .name("Compute Distances") - .build(); - - // create ReduceOperator for finding the nearest cluster centers - ReduceOperator findNearestClusterCenters = ReduceOperator.builder(new FindNearestCenter(), IntValue.class, 0) - .input(computeDistance) - .name("Find Nearest Centers") - .build(); - - // create ReduceOperator for computing new cluster positions - ReduceOperator recomputeClusterCenter = ReduceOperator.builder(new RecomputeClusterCenter(), IntValue.class, 0) - .input(findNearestClusterCenters) - .name("Recompute Center Positions") - .build(); - iteration.setNextPartialSolution(recomputeClusterCenter); - - // create DataSourceContract for data point input - FileDataSource dataPoints2 = new FileDataSource(new PointInFormat(), dataPointInput, "Data Points 2"); - - // compute distance of points to final clusters - CrossOperator computeFinalDistance = CrossOperator.builder(new ComputeDistance()) - .input1(dataPoints2) - .input2(iteration) - .name("Compute Final Distances") - .build(); - - // find nearest final cluster for point - ReduceOperator findNearestFinalCluster = ReduceOperator.builder(new FindNearestCenter(), IntValue.class, 0) - .input(computeFinalDistance) - .name("Find Nearest Final Centers") - .build(); - - // create DataSinkContract for writing the new cluster positions - FileDataSink finalClusters = new FileDataSink(new PointOutFormat(), output+"/centers", iteration, "Cluster Positions"); - - // write assigned clusters - FileDataSink clusterAssignments = new FileDataSink(new PointOutFormat(), output+"/points", findNearestFinalCluster, "Cluster Assignments"); - - List sinks = new ArrayList(); - sinks.add(finalClusters); - sinks.add(clusterAssignments); - - // return the PACT plan - Plan plan = new Plan(sinks, "Iterative KMeans"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } - - public static void main(String[] args) throws Exception { - KMeansCross kmi = new KMeansCross(); - - if (args.length < 5) { - System.err.println(kmi.getDescription()); - System.exit(1); - } - - Plan plan = kmi.getPlan(args); - - // This will execute the kMeans clustering job embedded in a local context. - LocalExecutor.execute(plan); - - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java deleted file mode 100644 index bdf74663976ac..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java +++ /dev/null @@ -1,300 +0,0 @@ -/* - * 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.recordJobs.kmeans; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class KMeansSingleStep implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) { - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataPointInput = (args.length > 1 ? args[1] : ""); - String clusterInput = (args.length > 2 ? args[2] : ""); - String output = (args.length > 3 ? args[3] : ""); - - // create DataSourceContract for data point input - @SuppressWarnings("unchecked") - FileDataSource pointsSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class), dataPointInput, "Data Points"); - - // create DataSourceContract for cluster center input - @SuppressWarnings("unchecked") - FileDataSource clustersSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class), clusterInput, "Centers"); - - MapOperator dataPoints = MapOperator.builder(new PointBuilder()).name("Build data points").input(pointsSource).build(); - - MapOperator clusterPoints = MapOperator.builder(new PointBuilder()).name("Build cluster points").input(clustersSource).build(); - - // the mapper computes the distance to all points, which it draws from a broadcast variable - MapOperator findNearestClusterCenters = MapOperator.builder(new SelectNearestCenter()) - .setBroadcastVariable("centers", clusterPoints) - .input(dataPoints) - .name("Find Nearest Centers") - .build(); - - // create reducer recomputes the cluster centers as the average of all associated data points - ReduceOperator recomputeClusterCenter = ReduceOperator.builder(new RecomputeClusterCenter(), IntValue.class, 0) - .input(findNearestClusterCenters) - .name("Recompute Center Positions") - .build(); - - // create DataSinkContract for writing the new cluster positions - FileDataSink newClusterPoints = new FileDataSink(new PointOutFormat(), output, recomputeClusterCenter, "New Center Positions"); - - // return the plan - Plan plan = new Plan(newClusterPoints, "KMeans Iteration"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } - - public static final class Point implements Value { - private static final long serialVersionUID = 1L; - - public double x, y, z; - - public Point() {} - - public Point(double x, double y, double z) { - this.x = x; - this.y = y; - this.z = z; - } - - public void add(Point other) { - x += other.x; - y += other.y; - z += other.z; - } - - public Point div(long val) { - x /= val; - y /= val; - z /= val; - return this; - } - - public double euclideanDistance(Point other) { - return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y) + (z-other.z)*(z-other.z)); - } - - public void clear() { - x = y = z = 0.0; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeDouble(x); - out.writeDouble(y); - out.writeDouble(z); - } - - @Override - public void read(DataInputView in) throws IOException { - x = in.readDouble(); - y = in.readDouble(); - z = in.readDouble(); - } - - @Override - public String toString() { - return "(" + x + "|" + y + "|" + z + ")"; - } - } - - public static final class PointWithId { - - public int id; - public Point point; - - public PointWithId(int id, Point p) { - this.id = id; - this.point = p; - } - } - - /** - * Determines the closest cluster center for a data point. - */ - public static final class SelectNearestCenter extends MapFunction { - private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - private final Record result = new Record(3); - - private List centers = new ArrayList(); - - /** - * Reads all the center values from the broadcast variable into a collection. - */ - @Override - public void open(Configuration parameters) throws Exception { - Collection clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); - - centers.clear(); - for (Record r : clusterCenters) { - centers.add(new PointWithId(r.getField(0, IntValue.class).getValue(), r.getField(1, Point.class))); - } - } - - /** - * Computes a minimum aggregation on the distance of a data point to cluster centers. - * - * Output Format: - * 0: centerID - * 1: pointVector - * 2: constant(1) (to enable combinable average computation in the following reducer) - */ - @Override - public void map(Record dataPointRecord, Collector out) { - Point p = dataPointRecord.getField(1, Point.class); - - double nearestDistance = Double.MAX_VALUE; - int centerId = -1; - - // check all cluster centers - for (PointWithId center : centers) { - // compute distance - double distance = p.euclideanDistance(center.point); - - // update nearest cluster if necessary - if (distance < nearestDistance) { - nearestDistance = distance; - centerId = center.id; - } - } - - // emit a new record with the center id and the data point. add a one to ease the - // implementation of the average function with a combiner - result.setField(0, new IntValue(centerId)); - result.setField(1, p); - result.setField(2, one); - - out.collect(result); - } - } - - @Combinable - public static final class RecomputeClusterCenter extends ReduceFunction { - private static final long serialVersionUID = 1L; - - private final Point p = new Point(); - - - /** - * Compute the new position (coordinate vector) of a cluster center. - */ - @Override - public void reduce(Iterator points, Collector out) { - Record sum = sumPointsAndCount(points); - sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); - out.collect(sum); - } - - /** - * Computes a pre-aggregated average value of a coordinate vector. - */ - @Override - public void combine(Iterator points, Collector out) { - out.collect(sumPointsAndCount(points)); - } - - private final Record sumPointsAndCount(Iterator dataPoints) { - Record next = null; - p.clear(); - int count = 0; - - // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); - p.add(next.getField(1, Point.class)); - count += next.getField(2, IntValue.class).getValue(); - } - - next.setField(1, p); - next.setField(2, new IntValue(count)); - return next; - } - } - - public static final class PointBuilder extends MapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - double x = record.getField(1, DoubleValue.class).getValue(); - double y = record.getField(2, DoubleValue.class).getValue(); - double z = record.getField(3, DoubleValue.class).getValue(); - - record.setField(1, new Point(x, y, z)); - out.collect(record); - } - } - - public static final class PointOutFormat extends FileOutputFormat { - - private static final long serialVersionUID = 1L; - - private static final String format = "%d|%.1f|%.1f|%.1f|\n"; - - @Override - public void writeRecord(Record record) throws IOException { - int id = record.getField(0, IntValue.class).getValue(); - Point p = record.getField(1, Point.class); - - byte[] bytes = String.format(format, id, p.x, p.y, p.z).getBytes(); - - this.stream.write(bytes); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java deleted file mode 100644 index ee33113d046b5..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.recordJobs.kmeans.udfs; - -import java.io.Serializable; - -import org.apache.flink.api.java.record.functions.CrossFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -/** - * Cross PACT computes the distance of all data points to all cluster - * centers. - */ -@SuppressWarnings("deprecation") -@ConstantFieldsFirst({0,1}) -public class ComputeDistance extends CrossFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final DoubleValue distance = new DoubleValue(); - - /** - * Computes the distance of one data point to one cluster center. - * - * Output Format: - * 0: pointID - * 1: pointVector - * 2: clusterID - * 3: distance - */ - @Override - public Record cross(Record dataPointRecord, Record clusterCenterRecord) throws Exception { - - CoordVector dataPoint = dataPointRecord.getField(1, CoordVector.class); - - IntValue clusterCenterId = clusterCenterRecord.getField(0, IntValue.class); - CoordVector clusterPoint = clusterCenterRecord.getField(1, CoordVector.class); - - this.distance.setValue(dataPoint.computeEuclidianDistance(clusterPoint)); - - // add cluster center id and distance to the data point record - dataPointRecord.setField(2, clusterCenterId); - dataPointRecord.setField(3, this.distance); - - return dataPointRecord; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java deleted file mode 100644 index 78b60efcba87d..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.recordJobs.kmeans.udfs; - -import java.io.Serializable; -import java.util.Collection; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -/** - * Cross PACT computes the distance of all data points to all cluster - * centers. - */ -@SuppressWarnings("deprecation") -@ConstantFieldsFirst({0,1}) -public class ComputeDistanceParameterized extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final DoubleValue distance = new DoubleValue(); - - private Collection clusterCenters; - - @Override - public void open(Configuration parameters) throws Exception { - this.clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); - } - - /** - * Computes the distance of one data point to one cluster center. - * - * Output Format: - * 0: pointID - * 1: pointVector - * 2: clusterID - * 3: distance - */ - @Override - public void map(Record dataPointRecord, Collector out) { - - CoordVector dataPoint = dataPointRecord.getField(1, CoordVector.class); - - for (Record clusterCenterRecord : this.clusterCenters) { - IntValue clusterCenterId = clusterCenterRecord.getField(0, IntValue.class); - CoordVector clusterPoint = clusterCenterRecord.getField(1, CoordVector.class); - - this.distance.setValue(dataPoint.computeEuclidianDistance(clusterPoint)); - - // add cluster center id and distance to the data point record - dataPointRecord.setField(2, clusterCenterId); - dataPointRecord.setField(3, this.distance); - - out.collect(dataPointRecord); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java deleted file mode 100644 index 1e893ce80f9a5..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.recordJobs.kmeans.udfs; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -/** - * Reduce PACT determines the closes cluster center for a data point. This - * is a minimum aggregation. Hence, a Combiner can be easily implemented. - */ -@SuppressWarnings("deprecation") -@Combinable -@ConstantFields(1) -public class FindNearestCenter extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue centerId = new IntValue(); - private final CoordVector position = new CoordVector(); - private final IntValue one = new IntValue(1); - - private final Record result = new Record(3); - - /** - * Computes a minimum aggregation on the distance of a data point to - * cluster centers. - * - * Output Format: - * 0: centerID - * 1: pointVector - * 2: constant(1) (to enable combinable average computation in the following reducer) - */ - @Override - public void reduce(Iterator pointsWithDistance, Collector out) { - double nearestDistance = Double.MAX_VALUE; - int nearestClusterId = 0; - - // check all cluster centers - while (pointsWithDistance.hasNext()) { - Record res = pointsWithDistance.next(); - - double distance = res.getField(3, DoubleValue.class).getValue(); - - // compare distances - if (distance < nearestDistance) { - // if distance is smaller than smallest till now, update nearest cluster - nearestDistance = distance; - nearestClusterId = res.getField(2, IntValue.class).getValue(); - res.getFieldInto(1, this.position); - } - } - - // emit a new record with the center id and the data point. add a one to ease the - // implementation of the average function with a combiner - this.centerId.setValue(nearestClusterId); - this.result.setField(0, this.centerId); - this.result.setField(1, this.position); - this.result.setField(2, this.one); - - out.collect(this.result); - } - - // ---------------------------------------------------------------------------------------- - - private final Record nearest = new Record(); - - /** - * Computes a minimum aggregation on the distance of a data point to - * cluster centers. - */ - @Override - public void combine(Iterator pointsWithDistance, Collector out) { - double nearestDistance = Double.MAX_VALUE; - - // check all cluster centers - while (pointsWithDistance.hasNext()) { - Record res = pointsWithDistance.next(); - double distance = res.getField(3, DoubleValue.class).getValue(); - - // compare distances - if (distance < nearestDistance) { - nearestDistance = distance; - res.copyTo(this.nearest); - } - } - - // emit nearest one - out.collect(this.nearest); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointOutFormat.java deleted file mode 100644 index 410397ef53b36..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointOutFormat.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.recordJobs.kmeans.udfs; - -import java.text.DecimalFormat; -import java.text.DecimalFormatSymbols; - -import org.apache.flink.api.java.record.io.DelimitedOutputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -/** - * Writes records that contain an id and a CoordVector. - * The output format is line-based, i.e. one record is written to - * a line and terminated with '\n'. Within a line the first '|' character - * separates the id from the CoordVector. The vector consists of a vector of - * decimals. The decimals are separated by '|'. The is is the id of a data - * point or cluster center and the vector the corresponding position - * (coordinate vector) of the data point or cluster center. Example line: - * "42|23.23|52.57|74.43| Id: 42 Coordinate vector: (23.23, 52.57, 74.43) - */ -public class PointOutFormat extends DelimitedOutputFormat { - private static final long serialVersionUID = 1L; - - private final DecimalFormat df = new DecimalFormat("####0.00"); - private final StringBuilder line = new StringBuilder(); - - - public PointOutFormat() { - DecimalFormatSymbols dfSymbols = new DecimalFormatSymbols(); - dfSymbols.setDecimalSeparator('.'); - this.df.setDecimalFormatSymbols(dfSymbols); - } - - @Override - public int serializeRecord(Record record, byte[] target) { - - line.setLength(0); - - IntValue centerId = record.getField(0, IntValue.class); - CoordVector centerPos = record.getField(1, CoordVector.class); - - - line.append(centerId.getValue()); - - for (double coord : centerPos.getCoordinates()) { - line.append('|'); - line.append(df.format(coord)); - } - line.append('|'); - - byte[] byteString = line.toString().getBytes(); - - if (byteString.length <= target.length) { - System.arraycopy(byteString, 0, target, 0, byteString.length); - return byteString.length; - } - else { - return -byteString.length; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java deleted file mode 100644 index 89e222b64c93d..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.recordJobs.kmeans.udfs; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -/** - * Reduce PACT computes the new position (coordinate vector) of a cluster - * center. This is an average computation. Hence, Combinable is annotated - * and the combine method implemented. - * - * Output Format: - * 0: clusterID - * 1: clusterVector - */ -@SuppressWarnings("deprecation") -@Combinable -@ConstantFields(0) -public class RecomputeClusterCenter extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue count = new IntValue(); - - /** - * Compute the new position (coordinate vector) of a cluster center. - */ - @Override - public void reduce(Iterator dataPoints, Collector out) { - Record next = null; - - // initialize coordinate vector sum and count - CoordVector coordinates = new CoordVector(); - double[] coordinateSum = null; - int count = 0; - - // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); - - // get the coordinates and the count from the record - double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); - int thisCount = next.getField(2, IntValue.class).getValue(); - - if (coordinateSum == null) { - if (coordinates.getCoordinates() != null) { - coordinateSum = coordinates.getCoordinates(); - } - else { - coordinateSum = new double[thisCoords.length]; - } - } - - addToCoordVector(coordinateSum, thisCoords); - count += thisCount; - } - - // compute new coordinate vector (position) of cluster center - for (int i = 0; i < coordinateSum.length; i++) { - coordinateSum[i] /= count; - } - - coordinates.setCoordinates(coordinateSum); - next.setField(1, coordinates); - next.setNull(2); - - // emit new position of cluster center - out.collect(next); - } - - /** - * Computes a pre-aggregated average value of a coordinate vector. - */ - @Override - public void combine(Iterator dataPoints, Collector out) { - - Record next = null; - - // initialize coordinate vector sum and count - CoordVector coordinates = new CoordVector(); - double[] coordinateSum = null; - int count = 0; - - // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); - - // get the coordinates and the count from the record - double[] thisCoords = next.getField(1, CoordVector.class).getCoordinates(); - int thisCount = next.getField(2, IntValue.class).getValue(); - - if (coordinateSum == null) { - if (coordinates.getCoordinates() != null) { - coordinateSum = coordinates.getCoordinates(); - } - else { - coordinateSum = new double[thisCoords.length]; - } - } - - addToCoordVector(coordinateSum, thisCoords); - count += thisCount; - } - - coordinates.setCoordinates(coordinateSum); - this.count.setValue(count); - next.setField(1, coordinates); - next.setField(2, this.count); - - // emit partial sum and partial count for average computation - out.collect(next); - } - - /** - * Adds two coordinate vectors by summing up each of their coordinates. - * - * @param cvToAddTo - * The coordinate vector to which the other vector is added. - * This vector is returned. - * @param cvToBeAdded - * The coordinate vector which is added to the other vector. - * This vector is not modified. - */ - private void addToCoordVector(double[] cvToAddTo, double[] cvToBeAdded) { - // check if both vectors have same length - if (cvToAddTo.length != cvToBeAdded.length) { - throw new IllegalArgumentException("The given coordinate vectors are not of equal length."); - } - - // sum coordinate vectors coordinate-wise - for (int i = 0; i < cvToAddTo.length; i++) { - cvToAddTo[i] += cvToBeAdded[i]; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java deleted file mode 100644 index b94880487f236..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirstExcept; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class MergeOnlyJoin implements Program { - - private static final long serialVersionUID = 1L; - - @ConstantFieldsFirstExcept(2) - public static class JoinInputs extends JoinFunction { - private static final long serialVersionUID = 1L; - - @Override - public void join(Record input1, Record input2, Collector out) { - input1.setField(2, input2.getField(1, IntValue.class)); - out.collect(input1); - } - } - - @ConstantFieldsExcept({}) - public static class DummyReduce extends ReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator values, Collector out) { - while (values.hasNext()) { - out.collect(values.next()); - } - } - } - - - @Override - public Plan getPlan(final String... args) { - // parse program parameters - int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String input1Path = (args.length > 1 ? args[1] : ""); - String input2Path = (args.length > 2 ? args[2] : ""); - String output = (args.length > 3 ? args[3] : ""); - int numSubtasksInput2 = (args.length > 4 ? Integer.parseInt(args[4]) : 1); - - // create DataSourceContract for Orders input - @SuppressWarnings("unchecked") - CsvInputFormat format1 = new CsvInputFormat('|', IntValue.class, IntValue.class); - FileDataSource input1 = new FileDataSource(format1, input1Path, "Input 1"); - - ReduceOperator aggInput1 = ReduceOperator.builder(DummyReduce.class, IntValue.class, 0) - .input(input1) - .name("AggOrders") - .build(); - - - // create DataSourceContract for Orders input - @SuppressWarnings("unchecked") - CsvInputFormat format2 = new CsvInputFormat('|', IntValue.class, IntValue.class); - FileDataSource input2 = new FileDataSource(format2, input2Path, "Input 2"); - input2.setParallelism(numSubtasksInput2); - - ReduceOperator aggInput2 = ReduceOperator.builder(DummyReduce.class, IntValue.class, 0) - .input(input2) - .name("AggLines") - .build(); - aggInput2.setParallelism(numSubtasksInput2); - - // create JoinOperator for joining Orders and LineItems - JoinOperator joinLiO = JoinOperator.builder(JoinInputs.class, IntValue.class, 0, 0) - .input1(aggInput1) - .input2(aggInput2) - .name("JoinLiO") - .build(); - - // create DataSinkContract for writing the result - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, joinLiO, "Output"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter('|') - .lenient(true) - .field(IntValue.class, 0) - .field(IntValue.class, 1) - .field(IntValue.class, 2); - - // assemble the PACT plan - Plan plan = new Plan(result, "Merge Only Join"); - plan.setDefaultParallelism(numSubtasks); - return plan; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java deleted file mode 100644 index d805b92b4cf2a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.recordJobs.relational; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.relational.query1Util.GroupByReturnFlag; -import org.apache.flink.test.recordJobs.relational.query1Util.LineItemFilter; -import org.apache.flink.test.recordJobs.util.IntTupleDataInFormat; -import org.apache.flink.test.recordJobs.util.StringTupleDataOutFormat; -import org.apache.flink.types.StringValue; - -@SuppressWarnings("deprecation") -public class TPCHQuery1 implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - private int parallelism = 1; - private String lineItemInputPath; - private String outputPath; - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - - - if (args.length != 3) { - this.parallelism = 1; - this.lineItemInputPath = ""; - this.outputPath = ""; - } else { - this.parallelism = Integer.parseInt(args[0]); - this.lineItemInputPath = args[1]; - this.outputPath = args[2]; - } - - FileDataSource lineItems = - new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems"); - lineItems.setParallelism(this.parallelism); - - FileDataSink result = - new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output"); - result.setParallelism(this.parallelism); - - MapOperator lineItemFilter = - MapOperator.builder(new LineItemFilter()) - .name("LineItem Filter") - .build(); - lineItemFilter.setParallelism(this.parallelism); - - ReduceOperator groupByReturnFlag = - ReduceOperator.builder(new GroupByReturnFlag(), StringValue.class, 0) - .name("groupyBy") - .build(); - - lineItemFilter.setInput(lineItems); - groupByReturnFlag.setInput(lineItemFilter); - result.setInput(groupByReturnFlag); - - return new Plan(result, "TPC-H 1"); - } - - @Override - public String getDescription() { - return "Parameters: [parallelism] [lineitem-input] [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java deleted file mode 100644 index 4bb0cdf117b7f..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java +++ /dev/null @@ -1,365 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.io.IOException; -import java.text.DecimalFormat; -import java.text.DecimalFormatSymbols; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.util.IntTupleDataInFormat; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({"serial", "deprecation"}) -public class TPCHQuery10 implements Program, ProgramDescription { - - // -------------------------------------------------------------------------------------------- - // Local Filters and Projections - // -------------------------------------------------------------------------------------------- - - /** - * Forwards (0 = orderkey, 1 = custkey). - */ - public static class FilterO extends MapFunction - { - private static final int YEAR_FILTER = 1990; - - private final IntValue custKey = new IntValue(); - - @Override - public void map(Record record, Collector out) throws Exception { - - Tuple t = record.getField(1, Tuple.class); - if (Integer.parseInt(t.getStringValueAt(4).substring(0, 4)) > FilterO.YEAR_FILTER) { - // project - this.custKey.setValue((int) t.getLongValueAt(1)); - record.setField(1, this.custKey); - out.collect(record); - } - - } - } - - /** - * Forwards (0 = lineitem, 1 = tuple (extendedprice, discount) ) - */ - public static class FilterLI extends MapFunction - { - private final Tuple tuple = new Tuple(); - - @Override - public void map(Record record, Collector out) throws Exception - { - Tuple t = record.getField(1, this.tuple); - if (t.getStringValueAt(8).equals("R")) { - t.project(0x60); // l_extendedprice, l_discount - record.setField(1, t); - out.collect(record); - } - } - } - - /** - * Returns (0 = custkey, 1 = custName, 2 = NULL, 3 = balance, 4 = nationkey, 5 = address, 6 = phone, 7 = comment) - */ - public static class ProjectC extends MapFunction { - - private final Tuple tuple = new Tuple(); - - private final StringValue custName = new StringValue(); - - private final StringValue balance = new StringValue(); - private final IntValue nationKey = new IntValue(); - private final StringValue address = new StringValue(); - private final StringValue phone = new StringValue(); - private final StringValue comment = new StringValue(); - - @Override - public void map(Record record, Collector out) throws Exception - { - final Tuple t = record.getField(1, this.tuple); - - this.custName.setValue(t.getStringValueAt(1)); - this.address.setValue(t.getStringValueAt(2)); - this.nationKey.setValue((int) t.getLongValueAt(3)); - this.phone.setValue(t.getStringValueAt(4)); - this.balance.setValue(t.getStringValueAt(5)); - this.comment.setValue(t.getStringValueAt(7)); - - record.setField(1, this.custName); - record.setField(3, this.balance); - record.setField(4, this.nationKey); - record.setField(5, this.address); - record.setField(6, this.phone); - record.setField(7, this.comment); - - out.collect(record); - } - } - - /** - * Returns (0 = nationkey, 1 = nation_name) - */ - public static class ProjectN extends MapFunction - { - private final Tuple tuple = new Tuple(); - private final StringValue nationName = new StringValue(); - - @Override - public void map(Record record, Collector out) throws Exception - { - final Tuple t = record.getField(1, this.tuple); - - this.nationName.setValue(t.getStringValueAt(1)); - record.setField(1, this.nationName); - out.collect(record); - } - } - - // -------------------------------------------------------------------------------------------- - // Joins - // -------------------------------------------------------------------------------------------- - - /** - * Returns (0 = custKey, 1 = tuple (extendedprice, discount) ) - */ - public static class JoinOL extends JoinFunction - { - @Override - public void join(Record order, Record lineitem, Collector out) throws Exception { - lineitem.setField(0, order.getField(1, IntValue.class)); - out.collect(lineitem); - } - } - - /** - * Returns (0 = custkey, 1 = custName, 2 = extPrice * (1-discount), 3 = balance, 4 = nationkey, 5 = address, 6 = phone, 7 = comment) - */ - public static class JoinCOL extends JoinFunction - { - private final DoubleValue d = new DoubleValue(); - - @Override - public void join(Record custRecord, Record olRecord, Collector out) throws Exception - { - final Tuple t = olRecord.getField(1, Tuple.class); - final double extPrice = Double.parseDouble(t.getStringValueAt(0)); - final double discount = Double.parseDouble(t.getStringValueAt(1)); - - this.d.setValue(extPrice * (1 - discount)); - custRecord.setField(2, this.d); - out.collect(custRecord); - } - - } - - /** - * Returns (0 = custkey, 1 = custName, 2 = extPrice * (1-discount), 3 = balance, 4 = nationName, 5 = address, 6 = phone, 7 = comment) - */ - public static class JoinNCOL extends JoinFunction - { - @Override - public void join(Record colRecord, Record nation, Collector out) throws Exception { - colRecord.setField(4, nation.getField(1, StringValue.class)); - out.collect(colRecord); - } - } - - @ReduceOperator.Combinable - public static class Sum extends ReduceFunction - { - private final DoubleValue d = new DoubleValue(); - - @Override - public void reduce(Iterator records, Collector out) throws Exception - { - Record record = null; - double sum = 0; - while (records.hasNext()) { - record = records.next(); - sum += record.getField(2, DoubleValue.class).getValue(); - } - - this.d.setValue(sum); - record.setField(2, this.d); - out.collect(record); - } - - @Override - public void combine(Iterator records, Collector out) throws Exception { - reduce(records,out); - } - } - - public static class TupleOutputFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - private final DecimalFormat formatter; - private final StringBuilder buffer = new StringBuilder(); - - public TupleOutputFormat() { - DecimalFormatSymbols decimalFormatSymbol = new DecimalFormatSymbols(); - decimalFormatSymbol.setDecimalSeparator('.'); - - this.formatter = new DecimalFormat("#.####"); - this.formatter.setDecimalFormatSymbols(decimalFormatSymbol); - } - - @Override - public void writeRecord(Record record) throws IOException - { - this.buffer.setLength(0); - this.buffer.append(record.getField(0, IntValue.class).toString()).append('|'); - this.buffer.append(record.getField(1, StringValue.class).toString()).append('|'); - - this.buffer.append(this.formatter.format(record.getField(2, DoubleValue.class).getValue())).append('|'); - - this.buffer.append(record.getField(3, StringValue.class).toString()).append('|'); - this.buffer.append(record.getField(4, StringValue.class).toString()).append('|'); - this.buffer.append(record.getField(5, StringValue.class).toString()).append('|'); - this.buffer.append(record.getField(6, StringValue.class).toString()).append('|'); - this.buffer.append(record.getField(7, StringValue.class).toString()).append('|'); - - this.buffer.append('\n'); - - final byte[] bytes = this.buffer.toString().getBytes(); - this.stream.write(bytes); - } - } - - @Override - public String getDescription() { - return "TPC-H Query 10"; - } - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - final String ordersPath; - final String lineitemsPath; - final String customersPath; - final String nationsPath; - final String resultPath; - - final int parallelism; - - if (args.length < 6) { - throw new IllegalArgumentException("Invalid number of parameters"); - } else { - parallelism = Integer.parseInt(args[0]); - ordersPath = args[1]; - lineitemsPath = args[2]; - customersPath = args[3]; - nationsPath = args[4]; - resultPath = args[5]; - } - - FileDataSource orders = new FileDataSource(new IntTupleDataInFormat(), ordersPath, "Orders"); - // orders.setOutputContract(UniqueKey.class); - // orders.getCompilerHints().setAvgNumValuesPerKey(1); - - FileDataSource lineitems = new FileDataSource(new IntTupleDataInFormat(), lineitemsPath, "LineItems"); - // lineitems.getCompilerHints().setAvgNumValuesPerKey(4); - - FileDataSource customers = new FileDataSource(new IntTupleDataInFormat(), customersPath, "Customers"); - - FileDataSource nations = new FileDataSource(new IntTupleDataInFormat(), nationsPath, "Nations"); - - - MapOperator mapO = MapOperator.builder(FilterO.class) - .name("FilterO") - .build(); - - MapOperator mapLi = MapOperator.builder(FilterLI.class) - .name("FilterLi") - .build(); - - MapOperator projectC = MapOperator.builder(ProjectC.class) - .name("ProjectC") - .build(); - - MapOperator projectN = MapOperator.builder(ProjectN.class) - .name("ProjectN") - .build(); - - JoinOperator joinOL = JoinOperator.builder(JoinOL.class, IntValue.class, 0, 0) - .name("JoinOL") - .build(); - - JoinOperator joinCOL = JoinOperator.builder(JoinCOL.class, IntValue.class, 0, 0) - .name("JoinCOL") - .build(); - - JoinOperator joinNCOL = JoinOperator.builder(JoinNCOL.class, IntValue.class, 4, 0) - .name("JoinNCOL") - .build(); - - ReduceOperator reduce = ReduceOperator.builder(Sum.class) - .keyField(IntValue.class, 0) - .keyField(StringValue.class, 1) - .keyField(StringValue.class, 3) - .keyField(StringValue.class, 4) - .keyField(StringValue.class, 5) - .keyField(StringValue.class, 6) - .keyField(StringValue.class, 7) - .name("Reduce") - .build(); - - FileDataSink result = new FileDataSink(new TupleOutputFormat(), resultPath, "Output"); - - result.setInput(reduce); - - reduce.setInput(joinNCOL); - - joinNCOL.setFirstInput(joinCOL); - joinNCOL.setSecondInput(projectN); - - joinCOL.setFirstInput(projectC); - joinCOL.setSecondInput(joinOL); - - joinOL.setFirstInput(mapO); - joinOL.setSecondInput(mapLi); - - projectC.setInput(customers); - projectN.setInput(nations); - mapLi.setInput(lineitems); - mapO.setInput(orders); - - // return the PACT plan - Plan p = new Plan(result, "TPCH Q10"); - p.setDefaultParallelism(parallelism); - return p; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java deleted file mode 100644 index cebe6f94e15d5..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java +++ /dev/null @@ -1,277 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -/** - * The TPC-H is a decision support benchmark on relational data. - * Its documentation and the data generator (DBGEN) can be found - * on http://www.tpc.org/tpch/ .This implementation is tested with - * the DB2 data format. - * - * This program implements a modified version of the query 3 of - * the TPC-H benchmark including one join, some filtering and an - * aggregation. - * - * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue - * FROM orders, lineitem - * WHERE l_orderkey = o_orderkey - * AND o_orderstatus = "X" - * AND YEAR(o_orderdate) > Y - * AND o_orderpriority LIKE "Z%" - * GROUP BY l_orderkey, o_shippriority; - */ -@SuppressWarnings("deprecation") -public class TPCHQuery3 implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - public static final String YEAR_FILTER = "parameter.YEAR_FILTER"; - public static final String PRIO_FILTER = "parameter.PRIO_FILTER"; - - /** - * Map PACT implements the selection and projection on the orders table. - */ - @ConstantFields({0,1}) - public static class FilterO extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private String prioFilter; // filter literal for the order priority - private int yearFilter; // filter literal for the year - - // reusable objects for the fields touched in the mapper - private StringValue orderStatus; - private StringValue orderDate; - private StringValue orderPrio; - - /** - * Reads the filter literals from the configuration. - * - * @see org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration) - */ - @Override - public void open(Configuration parameters) { - this.yearFilter = parameters.getInteger(YEAR_FILTER, 1990); - this.prioFilter = parameters.getString(PRIO_FILTER, "0"); - } - - /** - * Filters the orders table by year, order status and order priority. - * - * o_orderstatus = "X" - * AND YEAR(o_orderdate) > Y - * AND o_orderpriority LIKE "Z" - * - * Output Schema: - * 0:ORDERKEY, - * 1:SHIPPRIORITY - */ - @Override - public void map(final Record record, final Collector out) { - orderStatus = record.getField(2, StringValue.class); - if (!orderStatus.getValue().equals("F")) { - return; - } - - orderPrio = record.getField(4, StringValue.class); - if(!orderPrio.getValue().startsWith(this.prioFilter)) { - return; - } - - orderDate = record.getField(3, StringValue.class); - if (!(Integer.parseInt(orderDate.getValue().substring(0, 4)) > this.yearFilter)) { - return; - } - - record.setNumFields(2); - out.collect(record); - } - } - - /** - * Match PACT realizes the join between LineItem and Order table. - * - */ - @ConstantFieldsFirst({0,1}) - public static class JoinLiO extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - /** - * Implements the join between LineItem and Order table on the order key. - * - * Output Schema: - * 0:ORDERKEY - * 1:SHIPPRIORITY - * 2:EXTENDEDPRICE - */ - @Override - public void join(Record order, Record lineitem, Collector out) { - order.setField(2, lineitem.getField(1, DoubleValue.class)); - out.collect(order); - } - } - - /** - * Reduce PACT implements the sum aggregation. - * The Combinable annotation is set as the partial sums can be calculated - * already in the combiner - * - */ - @Combinable - @ConstantFields({0,1}) - public static class AggLiO extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final DoubleValue extendedPrice = new DoubleValue(); - - /** - * Implements the sum aggregation. - * - * Output Schema: - * 0:ORDERKEY - * 1:SHIPPRIORITY - * 2:SUM(EXTENDEDPRICE) - */ - @Override - public void reduce(Iterator values, Collector out) { - Record rec = null; - double partExtendedPriceSum = 0; - - while (values.hasNext()) { - rec = values.next(); - partExtendedPriceSum += rec.getField(2, DoubleValue.class).getValue(); - } - - this.extendedPrice.setValue(partExtendedPriceSum); - rec.setField(2, this.extendedPrice); - out.collect(rec); - } - - /** - * Creates partial sums on the price attribute for each data batch. - */ - @Override - public void combine(Iterator values, Collector out) { - reduce(values, out); - } - } - - - @Override - public Plan getPlan(final String... args) { - // parse program parameters - final int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String ordersPath = (args.length > 1 ? args[1] : ""); - final String lineitemsPath = (args.length > 2 ? args[2] : ""); - final String output = (args.length > 3 ? args[3] : ""); - - // create DataSourceContract for Orders input - FileDataSource orders = new FileDataSource(new CsvInputFormat(), ordersPath, "Orders"); - CsvInputFormat.configureRecordFormat(orders) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) // order id - .field(IntValue.class, 7) // ship prio - .field(StringValue.class, 2, 2) // order status - .field(StringValue.class, 4, 10) // order date - .field(StringValue.class, 5, 8); // order prio - - // create DataSourceContract for LineItems input - FileDataSource lineitems = new FileDataSource(new CsvInputFormat(), lineitemsPath, "LineItems"); - CsvInputFormat.configureRecordFormat(lineitems) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) // order id - .field(DoubleValue.class, 5); // extended price - - // create MapOperator for filtering Orders tuples - MapOperator filterO = MapOperator.builder(new FilterO()) - .input(orders) - .name("FilterO") - .build(); - // filter configuration - filterO.setParameter(YEAR_FILTER, 1993); - filterO.setParameter(PRIO_FILTER, "5"); - // compiler hints - filterO.getCompilerHints().setFilterFactor(0.05f); - - // create JoinOperator for joining Orders and LineItems - JoinOperator joinLiO = JoinOperator.builder(new JoinLiO(), LongValue.class, 0, 0) - .input1(filterO) - .input2(lineitems) - .name("JoinLiO") - .build(); - - // create ReduceOperator for aggregating the result - // the reducer has a composite key, consisting of the fields 0 and 1 - ReduceOperator aggLiO = ReduceOperator.builder(new AggLiO()) - .keyField(LongValue.class, 0) - .keyField(StringValue.class, 1) - .input(joinLiO) - .name("AggLio") - .build(); - - // create DataSinkContract for writing the result - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, aggLiO, "Output"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter('|') - .lenient(true) - .field(LongValue.class, 0) - .field(IntValue.class, 1) - .field(DoubleValue.class, 2); - - // assemble the PACT plan - Plan plan = new Plan(result, "TPCH Q3"); - plan.setDefaultParallelism(numSubtasks); - return plan; - } - - - @Override - public String getDescription() { - return "Parameters: [numSubStasks], [orders], [lineitem], [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3Unioned.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3Unioned.java deleted file mode 100644 index 157e3cf95e8fe..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3Unioned.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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.recordJobs.relational; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3.AggLiO; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3.FilterO; -import org.apache.flink.test.recordJobs.relational.TPCHQuery3.JoinLiO; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -/** - * The TPC-H is a decision support benchmark on relational data. - * Its documentation and the data generator (DBGEN) can be found - * on http://www.tpc.org/tpch/ .This implementation is tested with - * the DB2 data format. - * THe PACT program implements a modified version of the query 3 of - * the TPC-H benchmark including one join, some filtering and an - * aggregation. - * - * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue - * FROM orders, lineitem - * WHERE l_orderkey = o_orderkey - * AND o_orderstatus = "X" - * AND YEAR(o_orderdate) > Y - * AND o_orderpriority LIKE "Z%" - * GROUP BY l_orderkey, o_shippriority; - */ -@SuppressWarnings("deprecation") -public class TPCHQuery3Unioned implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(final String... args) { - // parse program parameters - final int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String orders1Path = (args.length > 1 ? args[1] : ""); - String orders2Path = (args.length > 2 ? args[2] : ""); - String partJoin1Path = (args.length > 3 ? args[3] : ""); - String partJoin2Path = (args.length > 4 ? args[4] : ""); - - String lineitemsPath = (args.length > 5 ? args[5] : ""); - String output = (args.length > 6 ? args[6] : ""); - - // create DataSourceContract for Orders input - FileDataSource orders1 = new FileDataSource(new CsvInputFormat(), orders1Path, "Orders 1"); - CsvInputFormat.configureRecordFormat(orders1) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) // order id - .field(IntValue.class, 7) // ship prio - .field(StringValue.class, 2, 2) // order status - .field(StringValue.class, 4, 10) // order date - .field(StringValue.class, 5, 8); // order prio - - FileDataSource orders2 = new FileDataSource(new CsvInputFormat(), orders2Path, "Orders 2"); - CsvInputFormat.configureRecordFormat(orders2) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) // order id - .field(IntValue.class, 7) // ship prio - .field(StringValue.class, 2, 2) // order status - .field(StringValue.class, 4, 10) // order date - .field(StringValue.class, 5, 8); // order prio - - // create DataSourceContract for LineItems input - FileDataSource lineitems = new FileDataSource(new CsvInputFormat(), lineitemsPath, "LineItems"); - CsvInputFormat.configureRecordFormat(lineitems) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) - .field(DoubleValue.class, 5); - - // create MapOperator for filtering Orders tuples - MapOperator filterO1 = MapOperator.builder(new FilterO()) - .name("FilterO") - .input(orders1) - .build(); - // filter configuration - filterO1.setParameter(TPCHQuery3.YEAR_FILTER, 1993); - filterO1.setParameter(TPCHQuery3.PRIO_FILTER, "5"); - filterO1.getCompilerHints().setFilterFactor(0.05f); - - // create MapOperator for filtering Orders tuples - MapOperator filterO2 = MapOperator.builder(new FilterO()) - .name("FilterO") - .input(orders2) - .build(); - // filter configuration - filterO2.setParameter(TPCHQuery3.YEAR_FILTER, 1993); - filterO2.setParameter(TPCHQuery3.PRIO_FILTER, "5"); - - // create JoinOperator for joining Orders and LineItems - @SuppressWarnings("unchecked") - JoinOperator joinLiO = JoinOperator.builder(new JoinLiO(), LongValue.class, 0, 0) - .input1(filterO2, filterO1) - .input2(lineitems) - .name("JoinLiO") - .build(); - - FileDataSource partJoin1 = new FileDataSource(new CsvInputFormat(), partJoin1Path, "Part Join 1"); - CsvInputFormat.configureRecordFormat(partJoin1) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) - .field(IntValue.class, 1) - .field(DoubleValue.class, 2); - - FileDataSource partJoin2 = new FileDataSource(new CsvInputFormat(), partJoin2Path, "Part Join 2"); - CsvInputFormat.configureRecordFormat(partJoin2) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(LongValue.class, 0) - .field(IntValue.class, 1) - .field(DoubleValue.class, 2); - - // create ReduceOperator for aggregating the result - // the reducer has a composite key, consisting of the fields 0 and 1 - @SuppressWarnings("unchecked") - ReduceOperator aggLiO = ReduceOperator.builder(new AggLiO()) - .keyField(LongValue.class, 0) - .keyField(StringValue.class, 1) - .input(joinLiO, partJoin2, partJoin1) - .name("AggLio") - .build(); - - // create DataSinkContract for writing the result - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, aggLiO, "Output"); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter('|') - .lenient(true) - .field(LongValue.class, 0) - .field(IntValue.class, 1) - .field(DoubleValue.class, 2); - - // assemble the PACT plan - Plan plan = new Plan(result, "TPCH Q3 Unioned"); - plan.setDefaultParallelism(numSubtasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: [numSubStasks], [orders1], [orders2], [partJoin1], [partJoin2], [lineitem], [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java deleted file mode 100644 index ec3c5b4f2d7f1..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java +++ /dev/null @@ -1,284 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Calendar; -import java.util.Date; -import java.util.GregorianCalendar; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.recordJobs.util.IntTupleDataInFormat; -import org.apache.flink.test.recordJobs.util.StringTupleDataOutFormat; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Implementation of the TPC-H Query 4 as a Flink program. - */ - -@SuppressWarnings({"serial", "deprecation"}) -public class TPCHQuery4 implements Program, ProgramDescription { - - private static Logger LOG = LoggerFactory.getLogger(TPCHQuery4.class); - - private int parallelism = 1; - private String ordersInputPath; - private String lineItemInputPath; - private String outputPath; - - - /** - * Small {@link MapFunction} to filer out the irrelevant orders. - * - */ - //@SameKey - public static class OFilter extends MapFunction { - - private final String dateParamString = "1995-01-01"; - private final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd"); - private final GregorianCalendar gregCal = new GregorianCalendar(); - - private Date paramDate; - private Date plusThreeMonths; - - @Override - public void open(Configuration parameters) { - try { - this.paramDate = sdf.parse(this.dateParamString); - this.plusThreeMonths = getPlusThreeMonths(paramDate); - - } catch (ParseException e) { - throw new RuntimeException(e); - } - } - - @Override - public void map(Record record, Collector out) throws Exception { - Tuple tuple = record.getField(1, Tuple.class); - Date orderDate; - - String orderStringDate = tuple.getStringValueAt(4); - - try { - orderDate = sdf.parse(orderStringDate); - } catch (ParseException e) { - throw new RuntimeException(e); - } - - if(paramDate.before(orderDate) && plusThreeMonths.after(orderDate)) { - out.collect(record); - } - - } - - /** - * Calculates the {@link Date} which is three months after the given one. - * @param paramDate of type {@link Date}. - * @return a {@link Date} three month later. - */ - private Date getPlusThreeMonths(Date paramDate) { - - gregCal.setTime(paramDate); - gregCal.add(Calendar.MONTH, 3); - Date plusThreeMonths = gregCal.getTime(); - return plusThreeMonths; - } - } - - /** - * Simple filter for the line item selection. It filters all teh tuples that do - * not satisfy the "l_commitdate < l_receiptdate" condition. - * - */ - //@SameKey - public static class LiFilter extends MapFunction { - - private final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd"); - - @Override - public void map(Record record, Collector out) throws Exception { - Tuple tuple = record.getField(1, Tuple.class); - String commitString = tuple.getStringValueAt(11); - String receiptString = tuple.getStringValueAt(12); - - Date commitDate; - Date receiptDate; - - try { - commitDate = sdf.parse(commitString); - receiptDate = sdf.parse(receiptString); - } catch (ParseException e) { - throw new RuntimeException(e); - } - - if (commitDate.before(receiptDate)) { - out.collect(record); - } - - } - } - - /** - * Implements the equijoin on the orderkey and performs the projection on - * the order priority as well. - * - */ - public static class JoinLiO extends JoinFunction { - - @Override - public void join(Record order, Record line, Collector out) - throws Exception { - Tuple orderTuple = order.getField(1, Tuple.class); - - orderTuple.project(32); - String newOrderKey = orderTuple.getStringValueAt(0); - - order.setField(0, new StringValue(newOrderKey)); - out.collect(order); - } - } - - /** - * Implements the count(*) part. - * - */ - //@SameKey - public static class CountAgg extends ReduceFunction { - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - long count = 0; - Record rec = null; - - while(records.hasNext()) { - rec = records.next(); - count++; - } - - if(rec != null) - { - Tuple tuple = new Tuple(); - tuple.addAttribute("" + count); - rec.setField(1, tuple); - } - - out.collect(rec); - } - } - - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - - if(args == null || args.length != 4) - { - LOG.warn("number of arguments do not match!"); - this.ordersInputPath = ""; - this.lineItemInputPath = ""; - this.outputPath = ""; - }else - { - setArgs(args); - } - - FileDataSource orders = - new FileDataSource(new IntTupleDataInFormat(), this.ordersInputPath, "Orders"); - orders.setParallelism(this.parallelism); - //orders.setOutputContract(UniqueKey.class); - - FileDataSource lineItems = - new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems"); - lineItems.setParallelism(this.parallelism); - - FileDataSink result = - new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output"); - result.setParallelism(parallelism); - - MapOperator lineFilter = - MapOperator.builder(LiFilter.class) - .name("LineItemFilter") - .build(); - lineFilter.setParallelism(parallelism); - - MapOperator ordersFilter = - MapOperator.builder(OFilter.class) - .name("OrdersFilter") - .build(); - ordersFilter.setParallelism(parallelism); - - JoinOperator join = - JoinOperator.builder(JoinLiO.class, IntValue.class, 0, 0) - .name("OrdersLineitemsJoin") - .build(); - join.setParallelism(parallelism); - - ReduceOperator aggregation = - ReduceOperator.builder(CountAgg.class, StringValue.class, 0) - .name("AggregateGroupBy") - .build(); - aggregation.setParallelism(this.parallelism); - - lineFilter.setInput(lineItems); - ordersFilter.setInput(orders); - join.setFirstInput(ordersFilter); - join.setSecondInput(lineFilter); - aggregation.setInput(join); - result.setInput(aggregation); - - - return new Plan(result, "TPC-H 4"); - } - - /** - * Get the args into the members. - * @param args - */ - private void setArgs(String[] args) { - this.parallelism = Integer.parseInt(args[0]); - this.ordersInputPath = args[1]; - this.lineItemInputPath = args[2]; - this.outputPath = args[3]; - } - - - @Override - public String getDescription() { - return "Parameters: [parallelism] [orders-input] [lineitem-input] [output]"; - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java deleted file mode 100644 index c00d231740c39..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java +++ /dev/null @@ -1,251 +0,0 @@ -/* - * 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.recordJobs.relational; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.test.recordJobs.relational.query9Util.AmountAggregate; -import org.apache.flink.test.recordJobs.relational.query9Util.FilteredPartsJoin; -import org.apache.flink.test.recordJobs.relational.query9Util.IntPair; -import org.apache.flink.test.recordJobs.relational.query9Util.LineItemMap; -import org.apache.flink.test.recordJobs.relational.query9Util.OrderMap; -import org.apache.flink.test.recordJobs.relational.query9Util.OrderedPartsJoin; -import org.apache.flink.test.recordJobs.relational.query9Util.PartFilter; -import org.apache.flink.test.recordJobs.relational.query9Util.PartJoin; -import org.apache.flink.test.recordJobs.relational.query9Util.PartListJoin; -import org.apache.flink.test.recordJobs.relational.query9Util.PartsuppMap; -import org.apache.flink.test.recordJobs.relational.query9Util.StringIntPair; -import org.apache.flink.test.recordJobs.relational.query9Util.StringIntPairStringDataOutFormat; -import org.apache.flink.test.recordJobs.relational.query9Util.SupplierMap; -import org.apache.flink.test.recordJobs.relational.query9Util.SuppliersJoin; -import org.apache.flink.test.recordJobs.util.IntTupleDataInFormat; -import org.apache.flink.types.IntValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Quote from the TPC-H homepage: - * "The TPC-H is a decision support benchmark on relational data. - * Its documentation and the data generator (DBGEN) can be found - * on http://www.tpc.org/tpch/ .This implementation is tested with - * the DB2 data format." - * This PACT program implements the query 9 of the TPC-H benchmark: - * - *

- * select nation, o_year, sum(amount) as sum_profit
- * from (
- *   select n_name as nation, extract(year from o_orderdate) as o_year,
- *          l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount
- *   from part, supplier, lineitem, partsupp, orders, nation
- *   where
- *     s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey
- *     and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey
- *     and p_name like '%[COLOR]%'
- * ) as profit
- * group by nation, o_year
- * order by nation, o_year desc;
- * 
- * - * Plan:
- * Match "part" and "partsupp" on "partkey" -> "parts" with (partkey, suppkey) as key - * Match "orders" and "lineitem" on "orderkey" -> "ordered_parts" with (partkey, suppkey) as key - * Match "parts" and "ordered_parts" on (partkey, suppkey) -> "filtered_parts" with "suppkey" as key - * Match "supplier" and "nation" on "nationkey" -> "suppliers" with "suppkey" as key - * Match "filtered_parts" and "suppliers" on" suppkey" -> "partlist" with (nation, o_year) as key - * Group "partlist" by (nation, o_year), calculate sum(amount) - * - * Attention: The "order by" part is not implemented! - * - */ -@SuppressWarnings({"serial", "deprecation"}) -public class TPCHQuery9 implements Program, ProgramDescription { - public final String ARGUMENTS = "parallelism partInputPath partSuppInputPath ordersInputPath lineItemInputPath supplierInputPath nationInputPath outputPath"; - - private static Logger LOG = LoggerFactory.getLogger(TPCHQuery9.class); - - private int parallelism = 1; - - private String partInputPath, partSuppInputPath, ordersInputPath, lineItemInputPath, supplierInputPath, - nationInputPath; - - private String outputPath; - - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - - if (args.length != 8) - { - LOG.warn("number of arguments do not match!"); - - this.parallelism = 1; - this.partInputPath = ""; - this.partSuppInputPath = ""; - this.ordersInputPath = ""; - this.lineItemInputPath = ""; - this.supplierInputPath = ""; - this.nationInputPath = ""; - this.outputPath = ""; - }else - { - this.parallelism = Integer.parseInt(args[0]); - this.partInputPath = args[1]; - this.partSuppInputPath = args[2]; - this.ordersInputPath = args[3]; - this.lineItemInputPath = args[4]; - this.supplierInputPath = args[5]; - this.nationInputPath = args[6]; - this.outputPath = args[7]; - } - - /* Create the 6 data sources: */ - /* part: (partkey | name, mfgr, brand, type, size, container, retailprice, comment) */ - FileDataSource partInput = new FileDataSource( - new IntTupleDataInFormat(), this.partInputPath, "\"part\" source"); - //partInput.setOutputContract(UniqueKey.class); -// partInput.getCompilerHints().setAvgNumValuesPerKey(1); - - /* partsupp: (partkey | suppkey, availqty, supplycost, comment) */ - FileDataSource partSuppInput = new FileDataSource( - new IntTupleDataInFormat(), this.partSuppInputPath, "\"partsupp\" source"); - - /* orders: (orderkey | custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment) */ - FileDataSource ordersInput = new FileDataSource( - new IntTupleDataInFormat(), this.ordersInputPath, "\"orders\" source"); - //ordersInput.setOutputContract(UniqueKey.class); -// ordersInput.getCompilerHints().setAvgNumValuesPerKey(1); - - /* lineitem: (orderkey | partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, ...) */ - FileDataSource lineItemInput = new FileDataSource( - new IntTupleDataInFormat(), this.lineItemInputPath, "\"lineitem\" source"); - - /* supplier: (suppkey | name, address, nationkey, phone, acctbal, comment) */ - FileDataSource supplierInput = new FileDataSource( - new IntTupleDataInFormat(), this.supplierInputPath, "\"supplier\" source"); - //supplierInput.setOutputContract(UniqueKey.class); -// supplierInput.getCompilerHints().setAvgNumValuesPerKey(1); - - /* nation: (nationkey | name, regionkey, comment) */ - FileDataSource nationInput = new FileDataSource( - new IntTupleDataInFormat(), this.nationInputPath, "\"nation\" source"); - //nationInput.setOutputContract(UniqueKey.class); -// nationInput.getCompilerHints().setAvgNumValuesPerKey(1); - - /* Filter on part's name, project values to NULL: */ - MapOperator filterPart = MapOperator.builder(PartFilter.class) - .name("filterParts") - .build(); - - /* Map to change the key element of partsupp, project value to (supplycost, suppkey): */ - MapOperator mapPartsupp = MapOperator.builder(PartsuppMap.class) - .name("mapPartsupp") - .build(); - - /* Map to extract the year from order: */ - MapOperator mapOrder = MapOperator.builder(OrderMap.class) - .name("mapOrder") - .build(); - - /* Project value to (partkey, suppkey, quantity, price = extendedprice*(1-discount)): */ - MapOperator mapLineItem = MapOperator.builder(LineItemMap.class) - .name("proj.Partsupp") - .build(); - - /* - change the key of supplier to nationkey, project value to suppkey */ - MapOperator mapSupplier = MapOperator.builder(SupplierMap.class) - .name("proj.Partsupp") - .build(); - - /* Equijoin on partkey of part and partsupp: */ - JoinOperator partsJoin = JoinOperator.builder(PartJoin.class, IntValue.class, 0, 0) - .name("partsJoin") - .build(); - - /* Equijoin on orderkey of orders and lineitem: */ - JoinOperator orderedPartsJoin = - JoinOperator.builder(OrderedPartsJoin.class, IntValue.class, 0, 0) - .name("orderedPartsJoin") - .build(); - - /* Equijoin on nationkey of supplier and nation: */ - JoinOperator suppliersJoin = - JoinOperator.builder(SuppliersJoin.class, IntValue.class, 0, 0) - .name("suppliersJoin") - .build(); - - /* Equijoin on (partkey,suppkey) of parts and orderedParts: */ - JoinOperator filteredPartsJoin = - JoinOperator.builder(FilteredPartsJoin.class, IntPair.class, 0, 0) - .name("filteredPartsJoin") - .build(); - - /* Equijoin on suppkey of filteredParts and suppliers: */ - JoinOperator partListJoin = - JoinOperator.builder(PartListJoin.class, IntValue.class , 0, 0) - .name("partlistJoin") - .build(); - - /* Aggregate sum(amount) by (nation,year): */ - ReduceOperator sumAmountAggregate = - ReduceOperator.builder(AmountAggregate.class, StringIntPair.class, 0) - .name("groupyBy") - .build(); - - /* Connect input filters: */ - filterPart.setInput(partInput); - mapPartsupp.setInput(partSuppInput); - mapOrder.setInput(ordersInput); - mapLineItem.setInput(lineItemInput); - mapSupplier.setInput(supplierInput); - - /* Connect equijoins: */ - partsJoin.setFirstInput(filterPart); - partsJoin.setSecondInput(mapPartsupp); - orderedPartsJoin.setFirstInput(mapOrder); - orderedPartsJoin.setSecondInput(mapLineItem); - suppliersJoin.setFirstInput(mapSupplier); - suppliersJoin.setSecondInput(nationInput); - filteredPartsJoin.setFirstInput(partsJoin); - filteredPartsJoin.setSecondInput(orderedPartsJoin); - partListJoin.setFirstInput(filteredPartsJoin); - partListJoin.setSecondInput(suppliersJoin); - - /* Connect aggregate: */ - sumAmountAggregate.setInput(partListJoin); - - /* Connect sink: */ - FileDataSink result = new FileDataSink(new StringIntPairStringDataOutFormat(), this.outputPath, "Results sink"); - result.setInput(sumAmountAggregate); - - Plan p = new Plan(result, "TPC-H query 9"); - p.setDefaultParallelism(this.parallelism); - return p; - } - - @Override - public String getDescription() { - return "TPC-H query 9, parameters: " + this.ARGUMENTS; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java deleted file mode 100644 index a681f64c71c81..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecondExcept; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -/** - * The TPC-H is a decision support benchmark on relational data. - * Its documentation and the data generator (DBGEN) can be found - * on http://www.tpc.org/tpch/ .This implementation is tested with - * the DB2 data format. - * - * This program implements a query on the TPC-H schema - * including one join and an aggregation. - * This query is used as example in the Asterix project (http://asterix.ics.uci.edu/). - * - * SELECT c_mktsegment, COUNT(o_orderkey) - * FROM orders, customer - * WHERE c_custkey = o_custkey - * GROUP BY c_mktsegment; - * - */ -@SuppressWarnings("deprecation") -public class TPCHQueryAsterix implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - - /** - * Realizes the join between Customers and Order table. - */ - @ConstantFieldsSecondExcept(0) - public static class JoinCO extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - - /** - * Output Schema: - * 0: PARTIAL_COUNT=1 - * 1: C_MKTSEGMENT - */ - @Override - public void join(Record order, Record cust, Collector out) - throws Exception { - cust.setField(0, one); - out.collect(cust); - } - } - - /** - * Reduce implements the aggregation of the results. The - * Combinable annotation is set as the partial counts can be calculated - * already in the combiner - * - */ - @Combinable - @ConstantFields(1) - public static class AggCO extends ReduceFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private final IntValue integer = new IntValue(); - private Record record = new Record(); - - /** - * Output Schema: - * 0: COUNT - * 1: C_MKTSEGMENT - * - */ - @Override - public void reduce(Iterator records, Collector out) - throws Exception { - - int count = 0; - - while (records.hasNext()) { - record = records.next(); - count+=record.getField(0, integer).getValue(); - } - - integer.setValue(count); - record.setField(0, integer); - out.collect(record); - } - - /** - * Computes partial counts - */ - public void combine(Iterator records, Collector out) - throws Exception { - reduce(records, out); - } - - } - - - @Override - public Plan getPlan(final String... args) { - - // parse program parameters - int numSubtasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String ordersPath = (args.length > 1 ? args[1] : ""); - String customerPath = (args.length > 2 ? args[2] : ""); - String output = (args.length > 3 ? args[3] : ""); - - /* - * Output Schema: - * 0: CUSTOMER_ID - */ - // create DataSourceContract for Orders input - FileDataSource orders = new FileDataSource(new CsvInputFormat(), ordersPath, "Orders"); - orders.setParallelism(numSubtasks); - CsvInputFormat.configureRecordFormat(orders) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(IntValue.class, 1); - - /* - * Output Schema: - * 0: CUSTOMER_ID - * 1: MKT_SEGMENT - */ - // create DataSourceContract for Customer input - FileDataSource customers = new FileDataSource(new CsvInputFormat(), customerPath, "Customers"); - customers.setParallelism(numSubtasks); - CsvInputFormat.configureRecordFormat(customers) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(IntValue.class, 0) - .field(StringValue.class, 6); - - // create JoinOperator for joining Orders and LineItems - JoinOperator joinCO = JoinOperator.builder(new JoinCO(), IntValue.class, 0, 0) - .name("JoinCO") - .build(); - joinCO.setParallelism(numSubtasks); - - // create ReduceOperator for aggregating the result - ReduceOperator aggCO = ReduceOperator.builder(new AggCO(), StringValue.class, 1) - .name("AggCo") - .build(); - aggCO.setParallelism(numSubtasks); - - // create DataSinkContract for writing the result - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, "Output"); - result.setParallelism(numSubtasks); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(IntValue.class, 0) - .field(StringValue.class, 1); - - // assemble the plan - result.setInput(aggCO); - aggCO.setInput(joinCO); - joinCO.setFirstInput(orders); - joinCO.setSecondInput(customers); - - return new Plan(result, "TPCH Asterix"); - } - - - @Override - public String getDescription() { - return "Parameters: [numSubStasks], [orders], [customer], [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java deleted file mode 100644 index 377c7eec79f91..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java +++ /dev/null @@ -1,341 +0,0 @@ -/* - * 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.recordJobs.relational; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.CoGroupFunction; -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirstExcept; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecondExcept; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.CoGroupOperator; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -/** - * Implements the following relational OLAP query as PACT program: - * - *
- * SELECT r.pageURL, r.pageRank, r.avgDuration
- * FROM Documents d JOIN Rankings r
- * 	ON d.url = r.url
- * WHERE CONTAINS(d.text, [keywords])
- * 	AND r.rank > [rank]
- * 	AND NOT EXISTS (
- * 		SELECT * FROM Visits v
- * 		WHERE v.destUrl = d.url
- * 			AND v.visitDate < [date]); 
- *  * 
- * - * Table Schemas:
- * CREATE TABLE Documents (
- * 					url VARCHAR(100) PRIMARY KEY,
- * 					contents TEXT );
- * 
- * CREATE TABLE Rankings (
- * 					pageRank INT,
- * 					pageURL VARCHAR(100) PRIMARY KEY,     
- * 					avgDuration INT );       
- * 
- * CREATE TABLE Visits (
- * 					sourceIP VARCHAR(16),
- * 					destURL VARCHAR(100),
- * 					visitDate DATE,
- * 					adRevenue FLOAT,
- * 					userAgent VARCHAR(64),
- * 					countryCode VARCHAR(3),
- * 					languageCode VARCHAR(6),
- * 					searchWord VARCHAR(32),
- * 					duration INT );
- * 
- * - */ -@SuppressWarnings("deprecation") -public class WebLogAnalysis implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - - /** - * MapFunction that filters for documents that contain a certain set of - * keywords. - */ - @ConstantFieldsExcept(1) - public static class FilterDocs extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private static final String[] KEYWORDS = { " editors ", " oscillations ", " convection " }; - - /** - * Filters for documents that contain all of the given keywords and projects the records on the URL field. - * - * Output Format: - * 0: URL - */ - @Override - public void map(Record record, Collector out) throws Exception { - // FILTER - // Only collect the document if all keywords are contained - String docText = record.getField(1, StringValue.class).toString(); - boolean allContained = true; - for (String kw : KEYWORDS) { - if (!docText.contains(kw)) { - allContained = false; - break; - } - } - - if (allContained) { - record.setNull(1); - out.collect(record); - } - } - } - - /** - * MapFunction that filters for records where the rank exceeds a certain threshold. - */ - @ConstantFieldsExcept({}) - public static class FilterRanks extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private static final int RANKFILTER = 50; - - /** - * Filters for records of the rank relation where the rank is greater - * than the given threshold. - * - * Output Format: - * 0: URL - * 1: RANK - * 2: AVG_DURATION - */ - @Override - public void map(Record record, Collector out) throws Exception { - - if (record.getField(1, IntValue.class).getValue() > RANKFILTER) { - out.collect(record); - } - } - } - - /** - * MapFunction that filters for records of the visits relation where the year - * (from the date string) is equal to a certain value. - */ - @ConstantFieldsExcept(1) - public static class FilterVisits extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - private static final int YEARFILTER = 2010; - - /** - * Filters for records of the visits relation where the year of visit is equal to a - * specified value. The URL of all visit records passing the filter is emitted. - * - * Output Format: - * 0: URL - */ - @Override - public void map(Record record, Collector out) throws Exception { - // Parse date string with the format YYYY-MM-DD and extract the year - String dateString = record.getField(1, StringValue.class).getValue(); - int year = Integer.parseInt(dateString.substring(0,4)); - - if (year == YEARFILTER) { - record.setNull(1); - out.collect(record); - - } - } - } - - /** - * JoinFunction that joins the filtered entries from the documents and the - * ranks relation. - */ - @ConstantFieldsSecondExcept({}) - public static class JoinDocRanks extends JoinFunction implements Serializable { - private static final long serialVersionUID = 1L; - - /** - * Joins entries from the documents and ranks relation on their URL. - * - * Output Format: - * 0: URL - * 1: RANK - * 2: AVG_DURATION - */ - @Override - public void join(Record document, Record rank, Collector out) throws Exception { - out.collect(rank); - } - } - - /** - * CoGroupFunction that realizes an anti-join. - * If the first input does not provide any pairs, all pairs of the second input are emitted. - * Otherwise, no pair is emitted. - */ - @ConstantFieldsFirstExcept({}) - public static class AntiJoinVisits extends CoGroupFunction implements Serializable { - private static final long serialVersionUID = 1L; - - /** - * If the visit iterator is empty, all pairs of the rank iterator are emitted. - * Otherwise, no pair is emitted. - * - * Output Format: - * 0: URL - * 1: RANK - * 2: AVG_DURATION - */ - @Override - public void coGroup(Iterator ranks, Iterator visits, Collector out) { - // Check if there is a entry in the visits relation - if (!visits.hasNext()) { - while (ranks.hasNext()) { - // Emit all rank pairs - out.collect(ranks.next()); - } - } - } - } - - - @Override - public Plan getPlan(String... args) { - - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String docsInput = (args.length > 1 ? args[1] : ""); - String ranksInput = (args.length > 2 ? args[2] : ""); - String visitsInput = (args.length > 3 ? args[3] : ""); - String output = (args.length > 4 ? args[4] : ""); - - /* - * Output Format: - * 0: URL - * 1: DOCUMENT_TEXT - */ - // Create DataSourceContract for documents relation - @SuppressWarnings("unchecked") - CsvInputFormat docsFormat = new CsvInputFormat('|', StringValue.class, StringValue.class); - FileDataSource docs = new FileDataSource(docsFormat, docsInput, "Docs Input"); - - /* - * Output Format: - * 0: URL - * 1: RANK - * 2: AVG_DURATION - */ - // Create DataSourceContract for ranks relation - FileDataSource ranks = new FileDataSource(new CsvInputFormat(), ranksInput, "Ranks input"); - CsvInputFormat.configureRecordFormat(ranks) - .recordDelimiter('\n') - .fieldDelimiter('|') - .field(StringValue.class, 1) - .field(IntValue.class, 0) - .field(IntValue.class, 2); - - /* - * Output Format: - * 0: URL - * 1: DATE - */ - // Create DataSourceContract for visits relation - @SuppressWarnings("unchecked") - CsvInputFormat visitsFormat = new CsvInputFormat('|', null, StringValue.class, StringValue.class); - FileDataSource visits = new FileDataSource(visitsFormat, visitsInput, "Visits input:q"); - - // Create MapOperator for filtering the entries from the documents - // relation - MapOperator filterDocs = MapOperator.builder(new FilterDocs()) - .input(docs) - .name("Filter Docs") - .build(); - filterDocs.getCompilerHints().setFilterFactor(0.15f); - - // Create MapOperator for filtering the entries from the ranks relation - MapOperator filterRanks = MapOperator.builder(new FilterRanks()) - .input(ranks) - .name("Filter Ranks") - .build(); - filterRanks.getCompilerHints().setFilterFactor(0.25f); - - // Create MapOperator for filtering the entries from the visits relation - MapOperator filterVisits = MapOperator.builder(new FilterVisits()) - .input(visits) - .name("Filter Visits") - .build(); - filterVisits.getCompilerHints().setFilterFactor(0.2f); - - // Create JoinOperator to join the filtered documents and ranks - // relation - JoinOperator joinDocsRanks = JoinOperator.builder(new JoinDocRanks(), StringValue.class, 0, 0) - .input1(filterDocs) - .input2(filterRanks) - .name("Join Docs Ranks") - .build(); - - // Create CoGroupOperator to realize a anti join between the joined - // documents and ranks relation and the filtered visits relation - CoGroupOperator antiJoinVisits = CoGroupOperator.builder(new AntiJoinVisits(), StringValue.class, 0, 0) - .input1(joinDocsRanks) - .input2(filterVisits) - .name("Antijoin DocsVisits") - .build(); - - // Create DataSinkContract for writing the result of the OLAP query - FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, antiJoinVisits, "Result"); - result.setParallelism(numSubTasks); - CsvOutputFormat.configureRecordFormat(result) - .recordDelimiter('\n') - .fieldDelimiter('|') - .lenient(true) - .field(IntValue.class, 1) - .field(StringValue.class, 0) - .field(IntValue.class, 2); - - // Return the PACT plan - Plan p = new Plan(result, "Weblog Analysis"); - p.setDefaultParallelism(numSubTasks); - return p; - } - - - @Override - public String getDescription() { - return "Parameters: [numSubTasks], [docs], [ranks], [visits], [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java deleted file mode 100644 index d7aab4baf04cb..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.recordJobs.relational.query1Util; - -import java.util.Iterator; - -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings("deprecation") -public class GroupByReturnFlag extends ReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - Record outRecord = new Record(); - Tuple returnTuple = new Tuple(); - - long quantity = 0; - double extendedPriceSum = 0.0; - - boolean first = true; - while(records.hasNext()) { - Record rec = records.next(); - Tuple t = rec.getField(1, Tuple.class); - - if(first) { - first = false; - rec.copyTo(outRecord); - returnTuple.addAttribute(rec.getField(0, StringValue.class).toString()); - } - - long tupleQuantity = Long.parseLong(t.getStringValueAt(4)); - quantity += tupleQuantity; - - double extendedPricePerTuple = Double.parseDouble(t.getStringValueAt(5)); - extendedPriceSum += extendedPricePerTuple; - } - - LongValue pactQuantity = new LongValue(quantity); - returnTuple.addAttribute("" + pactQuantity); - returnTuple.addAttribute("" + extendedPriceSum); - - outRecord.setField(1, returnTuple); - out.collect(outRecord); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilter.java deleted file mode 100644 index be5452427b6dc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilter.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.recordJobs.relational.query1Util; - -import java.text.DateFormat; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Date; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Filters the line item tuples according to the filter condition - * l_shipdate <= date '1998-12-01' - interval '[DELTA]' day (3) - * TODO: add parametrisation; first version uses a static interval = 90 - * - * In prepration of the following reduce step (see {@link GroupByReturnFlag}) the key has to be set to "return flag" - */ -@SuppressWarnings("deprecation") -public class LineItemFilter extends MapFunction { - - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(LineItemFilter.class); - private static final String DATE_CONSTANT = "1998-09-03"; - - private static final DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - - - private final Date constantDate; - - - public LineItemFilter() { - try { - this.constantDate = format.parse(DATE_CONSTANT); - } - catch (ParseException e) { - LOG.error("Date constant could not be parsed.", e); - throw new RuntimeException("Date constant could not be parsed."); - } - } - - @Override - public void map(Record record, Collector out) throws Exception { - Tuple value = record.getField(1, Tuple.class); - - if (value != null && value.getNumberOfColumns() >= 11) { - String shipDateString = value.getStringValueAt(10); - - try { - Date shipDate = format.parse(shipDateString); - - if (shipDate.before(constantDate)) { - String returnFlag = value.getStringValueAt(8); - - record.setField(0, new StringValue(returnFlag)); - out.collect(record); - } - } - catch (ParseException e) { - LOG.warn("ParseException while parsing the shipping date.", e); - } - - } - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilterTest.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilterTest.java deleted file mode 100644 index b03878bf8ab78..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilterTest.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * 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.recordJobs.relational.query1Util; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.MockitoAnnotations.initMocks; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.runtime.io.network.api.writer.RecordWriter; -import org.apache.flink.runtime.operators.shipping.RecordOutputCollector; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; -import org.junit.Before; -import org.junit.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mock; - -public class LineItemFilterTest { - - private static final String RETURN_FLAG = "N"; - - @Mock - RecordWriter recordWriterMock; - - private List> writerList = new ArrayList>(); - - @Before - public void setUp() - { - initMocks(this); - writerList.add(recordWriterMock); - } - - @Test - public void shouldNotFilterTuple() throws Exception, InterruptedException - { - LineItemFilter out = new LineItemFilter(); - - String shipDate = "1996-03-13"; - Tuple input = createInputTuple(shipDate); - IntValue inputKey = new IntValue(); - Record rec = new Record(); - rec.setField(0, inputKey); - rec.setField(1, input); - - Collector collector = new RecordOutputCollector(writerList); - - StringValue returnFlag = new StringValue(RETURN_FLAG); - - out.map(rec, collector); - - ArgumentCaptor argument = ArgumentCaptor.forClass(Record.class); - verify(recordWriterMock).emit(argument.capture()); - assertEquals(returnFlag, argument.getValue().getField(0, StringValue.class)); - assertEquals(input, argument.getValue().getField(1, Record.class)); - } - - - @Test - public void shouldFilterTuple() throws Exception, InterruptedException - { - LineItemFilter out = new LineItemFilter(); - - String shipDate = "1999-03-13"; - - Tuple input = createInputTuple(shipDate); - IntValue inputKey = new IntValue(); - Record rec = new Record(); - rec.setField(0, inputKey); - rec.setField(1, input); - - Collector collector = new RecordOutputCollector(writerList); - - out.map(rec, collector); - verifyNoMoreInteractions(recordWriterMock); - } - - @Test - public void shouldNotThrowExceptionWhenNullTuple() throws Exception - { - LineItemFilter out = new LineItemFilter(); - - Tuple input = null; - IntValue inputKey = new IntValue(); - Record rec = new Record(); - rec.setField(0, inputKey); - rec.setField(1, input); - - - Collector collector = new RecordOutputCollector(writerList); - - out.map(rec, collector); - verifyNoMoreInteractions(recordWriterMock); - } - - @Test - public void shouldNoThrowExceptionOnMalformedDate() throws Exception, InterruptedException - { - LineItemFilter out = new LineItemFilter(); - - String shipDate = "foobarDate"; - - Tuple input = createInputTuple(shipDate); - IntValue inputKey = new IntValue(); - Record rec = new Record(); - rec.setField(0, inputKey); - rec.setField(1, input); - - Collector collector = new RecordOutputCollector(writerList); - - out.map(rec, collector); - verifyNoMoreInteractions(recordWriterMock); - } - - @Test - public void shouldNoThrowExceptionOnTooShortTuple() throws Exception, InterruptedException - { - LineItemFilter out = new LineItemFilter(); - - Tuple input = new Tuple(); - input.addAttribute("" +1); - input.addAttribute("" + 155190); - input.addAttribute("" + 7706); - input.addAttribute("" + 1); - input.addAttribute("" + 17); - input.addAttribute("" + 21168.23); - input.addAttribute("" + 0.04); - input.addAttribute("" + 0.02); - input.addAttribute(RETURN_FLAG); - input.addAttribute("0"); - //the relevant column is missing now - - IntValue inputKey = new IntValue(); - Record rec = new Record(); - rec.setField(0, inputKey); - rec.setField(1, input); - - Collector collector = new RecordOutputCollector(writerList); - - out.map(rec, collector); - verifyNoMoreInteractions(recordWriterMock); - } - - - - /** - * Creates a subtuple of the lineitem relation. - * - * 1155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the| - * @param shipDate the date the {@link LineItemFilter} filters for. - */ - private Tuple createInputTuple(String shipDate) { - Tuple input = new Tuple(); - input.addAttribute("" +1); - input.addAttribute("" + 155190); - input.addAttribute("" + 7706); - input.addAttribute("" + 1); - input.addAttribute("" + 17); - input.addAttribute("" + 21168.23); - input.addAttribute("" + 0.04); - input.addAttribute("" + 0.02); - input.addAttribute(RETURN_FLAG); - input.addAttribute("0"); - input.addAttribute(shipDate); - return input; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java deleted file mode 100644 index 638877f326681..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import java.util.Iterator; - -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({"serial", "deprecation"}) -public class AmountAggregate extends ReduceFunction { - - private StringValue value = new StringValue(); - - /** - * Aggregate "amount": - * - * sum(amount) - * GROUP BY nation, year - * - * Output Schema: - * Key: (nation, year) - * Value: amount - * - */ - - @Override - public void reduce(Iterator records, Collector out) throws Exception - { - Record record = null; - float amount = 0; - - while (records.hasNext()) { - record = records.next(); - StringValue value = record.getField(1, StringValue.class); - amount += Float.parseFloat(value.toString()); - } - - value.setValue(String.valueOf(amount)); - record.setField(1, value); - out.collect(record); - } - - - /** - * Creates partial sums of "amount" for each data batch: - */ - @Override - public void combine(Iterator records, Collector out) throws Exception - { - reduce(records, out); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/FilteredPartsJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/FilteredPartsJoin.java deleted file mode 100644 index a391684322c5a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/FilteredPartsJoin.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "serial", "deprecation" }) -public class FilteredPartsJoin extends JoinFunction { - - private final IntPair partAndSupplierKey = new IntPair(); - private final StringValue supplyCostStr = new StringValue(); - private final Tuple ordersValue = new Tuple(); - - /** - * Join together parts and orderedParts by matching partkey and suppkey. - * - * Output Schema: - * Key: suppkey - * Value: (amount, year) - * - */ - @Override - public void join(Record value1, Record value2, Collector out) - throws Exception { - IntPair partAndSupplierKey = value1.getField(0, this.partAndSupplierKey); - StringValue supplyCostStr = value1.getField(1, this.supplyCostStr); - Tuple ordersValue = value2.getField(1, this.ordersValue); - - IntValue year = new IntValue(Integer.parseInt(ordersValue.getStringValueAt(0))); - float quantity = Float.parseFloat(ordersValue.getStringValueAt(1)); - float price = Float.parseFloat(ordersValue.getStringValueAt(2)); - float supplyCost = Float.parseFloat(supplyCostStr.toString()); - float amount = price - supplyCost * quantity; - - /* Push (supplierKey, (amount, year)): */ - value1.setField(0, partAndSupplierKey.getSecond()); - value1.setField(1, new StringIntPair(new StringValue("" + amount), year)); - out.collect(value1); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/IntPair.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/IntPair.java deleted file mode 100644 index 64b21bdd220b3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/IntPair.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Pair; - -public class IntPair extends Pair { - private static final long serialVersionUID = 1L; - - public IntPair() { - } - - public IntPair(IntValue first, IntValue second) { - super(first, second); - } - - public IntPair(int first, int second) { - super(new IntValue(first), new IntValue(second)); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/LineItemMap.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/LineItemMap.java deleted file mode 100644 index 6a862b80eebd3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/LineItemMap.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class LineItemMap extends MapFunction { - - /** - * Filter "lineitem". - * - * Output Schema: - * Key: orderkey - * Value: (partkey, suppkey, quantity, price) - * - */ - @Override - public void map(Record record, Collector out) throws Exception - { - Tuple inputTuple = record.getField(1, Tuple.class); - - /* Extract the year from the date element of the order relation: */ - - /* pice = extendedprice * (1 - discount): */ - float price = Float.parseFloat(inputTuple.getStringValueAt(5)) * (1 - Float.parseFloat(inputTuple.getStringValueAt(6))); - /* Project (orderkey | partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, ...) to (partkey, suppkey, quantity): */ - inputTuple.project((0 << 0) | (1 << 1) | (1 << 2) | (0 << 3) | (1 << 4)); - inputTuple.addAttribute("" + price); - record.setField(1, inputTuple); - out.collect(record); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderMap.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderMap.java deleted file mode 100644 index ee10333816983..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderMap.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class OrderMap extends MapFunction { - - private final Tuple inputTuple = new Tuple(); - - /** - * Project "orders" - * - * Output Schema: - * Key: orderkey - * Value: year (from date) - * - */ - @Override - public void map(Record record, Collector out) throws Exception { - Tuple inputTuple = record.getField(1, this.inputTuple); - - int year = Integer.parseInt(inputTuple.getStringValueAt(4).substring(0, 4)); - record.setField(1, new IntValue(year)); - out.collect(record); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderedPartsJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderedPartsJoin.java deleted file mode 100644 index 2e6453c411aa8..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderedPartsJoin.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class OrderedPartsJoin extends JoinFunction { - - /** - * Join "orders" and "lineitem" by "orderkey". - * - * Output Schema: - * Key: (partkey, suppkey) - * Value: (year, quantity, price) - * - */ - @Override - public void join(Record value1, Record value2, Collector out) - throws Exception { - - IntValue year = value1.getField(1, IntValue.class); - Tuple lineItem = value2.getField(1, Tuple.class); - - /* (partkey, suppkey) from lineItem: */ - IntPair newKey = new IntPair(new IntValue(Integer.parseInt(lineItem.getStringValueAt(0))), new IntValue(Integer.parseInt(lineItem.getStringValueAt(1)))); - Tuple newValue = new Tuple(); - newValue.addAttribute(year.toString()); // year - newValue.addAttribute(lineItem.getStringValueAt(2)); // quantity - newValue.addAttribute(lineItem.getStringValueAt(3)); // price - - value1.setField(0, newKey); - value1.setField(1, newValue); - out.collect(value1); - - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartFilter.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartFilter.java deleted file mode 100644 index 1e9a9208b99b3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartFilter.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.NullValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class PartFilter extends MapFunction { - - private final Tuple inputTuple = new Tuple(); - - private static String COLOR = "green"; - - /** - * Filter and project "part". - * The parts are filtered by "name LIKE %green%". - * - * Output Schema: - * Key: partkey - * Value: (empty) - * - */ - @Override - public void map(Record record, Collector out) throws Exception - { - Tuple inputTuple = record.getField(1, this.inputTuple); - if (inputTuple.getStringValueAt(1).indexOf(COLOR) != -1) { - record.setField(1, NullValue.getInstance()); - out.collect(record); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartJoin.java deleted file mode 100644 index 67f2f6a4812b4..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartJoin.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class PartJoin extends JoinFunction { - - private final Tuple partSuppValue = new Tuple(); - private final IntValue partKey = new IntValue(); - - /** - * Join "part" and "partsupp" by "partkey". - * - * Output Schema: - * Key: (partkey, suppkey) - * Value: supplycost - * - */ - @Override - public void join(Record value1, Record value2, Collector out) - throws Exception { - - IntValue partKey = value1.getField(0, this.partKey); - Tuple partSuppValue = value2.getField(1, this.partSuppValue); - - IntPair newKey = new IntPair(partKey, new IntValue(Integer.parseInt(partSuppValue.getStringValueAt(0)))); - String supplyCost = partSuppValue.getStringValueAt(1); - - value1.setField(0, newKey); - value1.setField(1, new StringValue(supplyCost)); - out.collect(value1); - - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartListJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartListJoin.java deleted file mode 100644 index 66860cec17746..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartListJoin.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class PartListJoin extends JoinFunction { - - private final StringIntPair amountYearPair = new StringIntPair(); - private final StringValue nationName = new StringValue(); - - /** - * Join "filteredParts" and "suppliers" by "suppkey". - * - * Output Schema: - * Key: (nation, year) - * Value: amount - * - */ - @Override - public void join(Record value1, Record value2, Collector out) throws Exception - { - StringIntPair amountYearPair = value1.getField(1, this.amountYearPair); - StringValue nationName = value2.getField(1, this.nationName); - - IntValue year = amountYearPair.getSecond(); - StringValue amount = amountYearPair.getFirst(); - StringIntPair key = new StringIntPair(nationName, year); - value1.setField(0, key); - value1.setField(1, amount); - out.collect(value1); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartsuppMap.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartsuppMap.java deleted file mode 100644 index 32c1d6433f860..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartsuppMap.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class PartsuppMap extends MapFunction { - - private Tuple inputTuple = new Tuple(); - - /** - * Project "partsupp". - * - * Output Schema: - * Key: partkey - * Value: (suppkey, supplycost) - * - */ - @Override - public void map(Record record, Collector out) throws Exception { - inputTuple = record.getField(1, inputTuple); - inputTuple.project((0 << 0) | (1 << 1) | (0 << 2) | (1 << 3) | (0 << 4)); - record.setField(1, inputTuple); - out.collect(record); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPair.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPair.java deleted file mode 100644 index 37f47b18b4af0..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPair.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Pair; -import org.apache.flink.types.StringValue; - -public class StringIntPair extends Pair { - private static final long serialVersionUID = 1L; - - public StringIntPair() { - } - - public StringIntPair(StringValue first, IntValue second) { - super(first, second); - } - - public StringIntPair(String first, int second) { - super(new StringValue(first), new IntValue(second)); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java deleted file mode 100644 index 42ddf6e8adac1..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import java.io.IOException; - -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; - -public class StringIntPairStringDataOutFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - private final StringBuilder buffer = new StringBuilder(); - private StringIntPair key = new StringIntPair(); - private StringValue value = new StringValue(); - - @Override - public void writeRecord(Record record) throws IOException { - key = record.getField(0, key); - value = record.getField(1, value); - - this.buffer.setLength(0); - this.buffer.append(key.getFirst().toString()); - this.buffer.append('|'); - this.buffer.append(key.getSecond().toString()); - this.buffer.append('|'); - this.buffer.append(value.toString()); - this.buffer.append('\n'); - - byte[] bytes = this.buffer.toString().getBytes(); - - this.stream.write(bytes); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SupplierMap.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SupplierMap.java deleted file mode 100644 index 2b435315fcf84..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SupplierMap.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class SupplierMap extends MapFunction { - - private IntValue suppKey = new IntValue(); - private Tuple inputTuple = new Tuple(); - - /** - * Project "supplier". - * - * Output Schema: - * Key: nationkey - * Value: suppkey - * - */ - @Override - public void map(Record record, Collector out) throws Exception { - suppKey = record.getField(0, suppKey); - inputTuple = record.getField(1, inputTuple); - - /* Project (suppkey | name, address, nationkey, phone, acctbal, comment): */ - IntValue nationKey = new IntValue(Integer.parseInt(inputTuple.getStringValueAt(3))); - - record.setField(0, nationKey); - record.setField(1, suppKey); - - out.collect(record); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SuppliersJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SuppliersJoin.java deleted file mode 100644 index f1252373cc4fe..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SuppliersJoin.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.recordJobs.relational.query9Util; - -import org.apache.flink.api.java.record.functions.JoinFunction; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -@SuppressWarnings({ "deprecation", "serial" }) -public class SuppliersJoin extends JoinFunction { - - private IntValue suppKey = new IntValue(); - private Tuple nationVal = new Tuple(); - - /** - * Join "nation" and "supplier" by "nationkey". - * - * Output Schema: - * Key: suppkey - * Value: "nation" (name of the nation) - * - */ - @Override - public void join(Record value1, Record value2, Collector out) - throws Exception { - suppKey = value1.getField(1, suppKey); - nationVal = value2.getField(1, nationVal); - - StringValue nationName = new StringValue(nationVal.getStringValueAt(1)); - - value1.setField(0, suppKey); - value1.setField(1, nationName); - - out.collect(value1); - - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java deleted file mode 100644 index b4dc1b494f669..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.recordJobs.sort; - -import java.io.Serializable; -import java.util.Iterator; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept; -import org.apache.flink.api.java.record.io.CsvInputFormat; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.util.Collector; - -/** - * This job shows how to define ordered input for a Reduce contract. - * The inputs for CoGroups can be (individually) ordered as well. - */ -@SuppressWarnings("deprecation") -public class ReduceGroupSort implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - /** - * Increments the first field of the first record of the reduce group by 100 and emits it. - * Then all remaining records of the group are emitted. * - */ - @ConstantFieldsExcept(0) - public static class IdentityReducer extends ReduceFunction implements Serializable { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator records, Collector out) { - - Record next = records.next(); - - // Increments the first field of the first record of the reduce group by 100 and emit it - IntValue incrVal = next.getField(0, IntValue.class); - incrVal.setValue(incrVal.getValue() + 100); - next.setField(0, incrVal); - out.collect(next); - - // emit all remaining records - while (records.hasNext()) { - out.collect(records.next()); - } - } - } - - - @Override - public Plan getPlan(String... args) { - - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataInput = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - @SuppressWarnings("unchecked") - CsvInputFormat format = new CsvInputFormat(' ', IntValue.class, IntValue.class); - FileDataSource input = new FileDataSource(format, dataInput, "Input"); - - // create the reduce contract and sets the key to the first field - ReduceOperator sorter = ReduceOperator.builder(new IdentityReducer(), IntValue.class, 0) - .input(input) - .name("Reducer") - .build(); - // sets the group sorting to the second field - sorter.setGroupOrder(new Ordering(1, IntValue.class, Order.ASCENDING)); - - // create and configure the output format - FileDataSink out = new FileDataSink(new CsvOutputFormat(), output, sorter, "Sorted Output"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(IntValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "SecondarySort Example"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: [numSubStasks] [input] [output]"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java deleted file mode 100644 index 4563c074bb39f..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.recordJobs.sort; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.test.recordJobs.sort.tsUtil.TeraDistribution; -import org.apache.flink.test.recordJobs.sort.tsUtil.TeraInputFormat; -import org.apache.flink.test.recordJobs.sort.tsUtil.TeraKey; -import org.apache.flink.test.recordJobs.sort.tsUtil.TeraOutputFormat; - -/** - * This is an example implementation of the TeraSort benchmark using the Flink system. The benchmark - * requires the input data to be generated according to the rules of Jim Gray's sort benchmark. A possible way to such - * input data is the Hadoop TeraGen program. For more details see
- * http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/examples/terasort/TeraGen.html. - */ -@SuppressWarnings("deprecation") -public final class TeraSort implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - - @Override - public String getDescription() { - return "Parameters: [numSubStasks] [input] [output]"; - } - - - @Override - public Plan getPlan(String... args) throws IllegalArgumentException { - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String input = (args.length > 1 ? args[1] : ""); - final String output = (args.length > 2 ? args[2] : ""); - - // This task will read the input data and generate the key/value pairs - final FileDataSource source = - new FileDataSource(new TeraInputFormat(), input, "Data Source"); - source.setParallelism(numSubTasks); - - // This task writes the sorted data back to disk - final FileDataSink sink = - new FileDataSink(new TeraOutputFormat(), output, "Data Sink"); - sink.setParallelism(numSubTasks); - sink.setGlobalOrder(new Ordering(0, TeraKey.class, Order.ASCENDING), new TeraDistribution()); - - sink.setInput(source); - - return new Plan(sink, "TeraSort"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraDistribution.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraDistribution.java deleted file mode 100644 index 6d670c84d93e4..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraDistribution.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.recordJobs.sort.tsUtil; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Key; - -/** - * This class implements the uniform data distribution of the TeraSort benchmark. - */ -public class TeraDistribution implements DataDistribution { - - private static final long serialVersionUID = 1L; - private static final int ALPHABETH_SIZE = 95; - - @Override - public Key[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - final byte[] buf = new byte[TeraKey.KEY_SIZE]; - double threshold = (double) ALPHABETH_SIZE / (double) (totalNumBuckets + 1) * (double) (bucketNum + 1); - - for (int i = 0; i < buf.length; ++i) { - final int ch = (int) Math.floor(threshold) % ALPHABETH_SIZE; - buf[i] = (byte) (' ' + ch); - - threshold = threshold - (double) ch; - threshold = threshold * ALPHABETH_SIZE; - } - - return new TeraKey[] { new TeraKey(buf, 0) }; - } - - @Override - public int getNumberOfFields() { - return 1; - } - - @Override - public void write(DataOutputView out) {} - - @Override - public void read(DataInputView in) {} -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraInputFormat.java deleted file mode 100644 index 7af5e1045cfe4..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraInputFormat.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.recordJobs.sort.tsUtil; - -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.types.Record; - -/** - * This class is responsible for converting a line from the input file to a two field record. - * Lines which do not match the expected length are skipped. - */ -public final class TeraInputFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - private final TeraKey key = new TeraKey(); - private final TeraValue value = new TeraValue(); - - - @Override - public Record readRecord(Record target, byte[] record, int offset, int numBytes) throws RuntimeException { - if (numBytes != (TeraKey.KEY_SIZE + TeraValue.VALUE_SIZE)) { - throw new RuntimeException("Malformatted input!"); - } - - this.key.setValue(record, offset); - this.value.setValue(record, offset + TeraKey.KEY_SIZE); - - target.setField(0, this.key); - target.setField(1, this.value); - return target; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraKey.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraKey.java deleted file mode 100644 index 511e094feb175..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraKey.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.recordJobs.sort.tsUtil; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Key; - -/** - * An integer number according to the rules of Jim Gray's sorting benchmark has a length of 100 bytes. The first 10 - * bytes represent the key for the sort process, the rest is the value. The final byte of a 100 byte integer is always - * the newline character. - *

- * This class is a wrapper for the key part of the integer number. - * - */ -public final class TeraKey implements Key { - private static final long serialVersionUID = 1L; - - /** - * The size of the key in bytes. - */ - public static final int KEY_SIZE = 10; - - /** - * The buffer to store the key. - */ - private byte[] key; - - /** - * The offset to the key byte sequence. - */ - private int offset; - - /** - * Constructs a new key object. The key points to the subsequence in the given array, i.e. it - * is sharing the byte array. - * - * @param srcBuf The source buffer to read the key from. - * @param offset The offset in the byte array where the key subsequence starts. - */ - public TeraKey(final byte[] srcBuf, int offset) { - this.key = srcBuf; - this.offset = offset; - } - - /** - * Default constructor required for serialization/deserialization. - */ - public TeraKey() { - this.key = new byte[KEY_SIZE]; - } - - /** - * Sets the value of this key object. This key will point to the subsequence in the given array, i.e. it - * is sharing the byte array. - * - * @param data The source buffer to read the key from. - * @param offset The offset in the byte array where the key subsequence starts. - */ - public void setValue(final byte[] data, int offset) { - this.key = data; - this.offset = offset; - } - - - @Override - public void write(DataOutputView out) throws IOException - { - out.write(this.key, this.offset, KEY_SIZE); - } - - - @Override - public void read(DataInputView in) throws IOException { - in.readFully(this.key, 0, KEY_SIZE); - this.offset = 0; - } - - - @Override - public int compareTo(TeraKey tsk) { - int diff = 0; - for (int i = 0; i < KEY_SIZE; ++i) { - - diff = (this.key[i + this.offset] - tsk.key[i + tsk.offset]); - if (diff != 0) { - break; - } - } - - return diff; - } - - - @Override - public int hashCode() { - int result = 1; - for (int i = 0; i < KEY_SIZE; i++) { - result = 31 * result + this.key[i + this.offset]; - } - return result; - } - - - @Override - public boolean equals(Object obj) { - if (getClass() != obj.getClass()) { - return false; - } - - final TeraKey other = (TeraKey) obj; - for (int i = 0, tx = this.offset, ox = other.offset; i < KEY_SIZE; i++, tx++, ox++) { - if (this.key[tx] != other.key[ox]) { - return false; - } - } - return true; - } - - /** - * Copies the key to the given byte buffer. - * - * @param buf - * the buffer to copy the key to - */ - public void copyToBuffer(final byte[] buf) { - - System.arraycopy(this.key, this.offset, buf, 0, KEY_SIZE); - } - - - @Override - public String toString() { - - return new String(this.key, this.offset, KEY_SIZE); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraOutputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraOutputFormat.java deleted file mode 100644 index da64cd2dfa2b0..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraOutputFormat.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.recordJobs.sort.tsUtil; - -import java.io.IOException; - -import org.apache.flink.api.java.record.io.FileOutputFormat; -import org.apache.flink.types.Record; - - -/** - * The class is responsible for converting a two field record back into a line which is afterward written back to disk. - * Each line ends with a newline character. - * - */ -public final class TeraOutputFormat extends FileOutputFormat { - private static final long serialVersionUID = 1L; - - /** - * A buffer to store the line which is about to be written back to disk. - */ - private final byte[] buffer = new byte[TeraKey.KEY_SIZE + TeraValue.VALUE_SIZE + 1]; - - - @Override - public void writeRecord(Record record) throws IOException { - record.getField(0, TeraKey.class).copyToBuffer(this.buffer); - record.getField(1, TeraValue.class).copyToBuffer(this.buffer); - - this.buffer[TeraKey.KEY_SIZE + TeraValue.VALUE_SIZE] = '\n'; - - this.stream.write(buffer, 0, buffer.length); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraValue.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraValue.java deleted file mode 100644 index 1658c5d94bc49..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraValue.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.recordJobs.sort.tsUtil; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Value; - -/** - * An integer number according to the rules of Jim Gray's sorting benchmark has a length of 100 bytes. The first 10 - * bytes represent the key for the sort process, the rest is the value. The final byte of a 100 byte integer is always - * the newline character. - *

- * This class is a wrapper for the value part of the integer number. - */ -public final class TeraValue implements Value { - private static final long serialVersionUID = 1L; - - /** - * The size of the value in bytes. - */ - public static final int VALUE_SIZE = 89; - - /** - * The buffer to store the value. - */ - private byte[] value; - - /** - * The offset to the value byte sequence. - */ - private int offset; - - /** - * Constructs a new value object. The value points to the subsequence in the given array, i.e. it - * is sharing the byte array. - * - * @param srcBuf The source buffer to read the value from. - * @param offset The offset in the byte array where the value subsequence starts. - */ - public TeraValue(final byte[] srcBuf, int offset) { - this.value = srcBuf; - this.offset = offset; - } - - /** - * Default constructor required for serialization/deserialization. - */ - public TeraValue() { - this.value = new byte[VALUE_SIZE]; - } - - /** - * Sets the value of this value object. This value will point to the subsequence in the given array, i.e. it - * is sharing the byte array. - * - * @param data The source buffer to read the value from. - * @param offset The offset in the byte array where the value subsequence starts. - */ - public void setValue(final byte[] data, int offset) { - this.value = data; - this.offset = offset; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.write(this.value, this.offset, VALUE_SIZE); - } - - @Override - public void read(DataInputView in) throws IOException { - in.readFully(this.value, 0, VALUE_SIZE); - this.offset = 0; - } - - @Override - public String toString() { - return new String(this.value, this.offset, VALUE_SIZE); - } - - /** - * Copies the value to the given byte buffer. - * - * @param buf - * the buffer to copy the value to - */ - public void copyToBuffer(final byte[] buf) { - System.arraycopy(this.value, this.offset, buf, 0, VALUE_SIZE); - } -} - diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/DiscardingOutputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/DiscardingOutputFormat.java deleted file mode 100644 index 545e1836532b8..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/DiscardingOutputFormat.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.recordJobs.util; - -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.Record; - -/** - * A simple output format that discards all data by doing nothing. - */ -public class DiscardingOutputFormat implements OutputFormat { - - private static final long serialVersionUID = 1L; - - @Override - public void configure(Configuration parameters) {} - - - @Override - public void open(int taskNumber, int numTasks) {} - - @Override - public void writeRecord(Record record) {} - - - @Override - public void close() {} -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/IntTupleDataInFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/IntTupleDataInFormat.java deleted file mode 100644 index e5d32c613bd5b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/IntTupleDataInFormat.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.recordJobs.util; - -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -public class IntTupleDataInFormat extends DelimitedInputFormat { - private static final long serialVersionUID = 1L; - - public static final int MAX_COLUMNS = 20; - - public static final int DELIMITER = '|'; - - private final IntValue key = new IntValue(); - private final int[] offsets = new int[MAX_COLUMNS]; - - @Override - public Record readRecord(Record target, byte[] line, int offset, int numBytes) { - final int limit = offset + numBytes; - int readPos = offset; - - // allocate the offsets array - final int[] offsets = this.offsets; - offsets[0] = offset; - - int col = 1; // the column we are in - - while (readPos < limit) { - if (line[readPos++] == DELIMITER) { - offsets[col++] = readPos; - } - } - - final Tuple value = new Tuple(line, offsets, col - 1); - this.key.setValue((int) value.getLongValueAt(0)); - - target.setField(0, this.key); - target.setField(1, value); - return target; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/StringTupleDataOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/StringTupleDataOutFormat.java deleted file mode 100644 index e6b85481458c3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/StringTupleDataOutFormat.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.recordJobs.util; - -import org.apache.flink.api.java.record.io.DelimitedOutputFormat; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; - -public class StringTupleDataOutFormat extends DelimitedOutputFormat { - private static final long serialVersionUID = 1L; - - @Override - public int serializeRecord(Record rec, byte[] target) throws Exception { - String string = rec.getField(0, StringValue.class).toString(); - byte[] stringBytes = string.getBytes(); - Tuple tuple = rec.getField(1, Tuple.class); - String tupleStr = tuple.toString(); - byte[] tupleBytes = tupleStr.getBytes(); - int totalLength = stringBytes.length + 1 + tupleBytes.length; - if(target.length >= totalLength) { - System.arraycopy(stringBytes, 0, target, 0, stringBytes.length); - target[stringBytes.length] = '|'; - System.arraycopy(tupleBytes, 0, target, stringBytes.length + 1, tupleBytes.length); - return totalLength; - } else { - return -1 * totalLength; - } - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/Tuple.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/Tuple.java deleted file mode 100644 index 064f15e442247..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/Tuple.java +++ /dev/null @@ -1,615 +0,0 @@ -/* - * 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.recordJobs.util; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Value; - -public class Tuple implements Value { - private static final long serialVersionUID = 1L; - - private byte[] bytes; - - private int[] offsets; - - private int numCols; - - /** - * Instantiates an empty tuple. - */ - public Tuple() { - numCols = 0; - } - - /** - * Creates a new tuple with a given set of attributes. - * - * @param bytes - * The bytes array. Attributes are separated by a single character. The last attribute - * is also terminated with a single character. - * @param offsets - * The offsets of the columns in the byte array. The last entry gives the offset of the terminating - * character + 1 (if the byte array exactly holds all attributes and delimiting characters this is - * the length of the array). - * @param cols - * The number of columns. - */ - public Tuple(byte[] bytes, int[] offsets, int cols) { - this.bytes = bytes; - this.offsets = offsets; - this.numCols = cols; - } - - // ------------------------------------------------------------------------ - // Accessors - // ------------------------------------------------------------------------ - - /** - * Returns the number of attributes / columns of the tuple. - * - * @return The number of columns of the tuple. - */ - public int getNumberOfColumns() { - return numCols; - } - - /** - * Returns the internal byte array of the tuple. - * - * @return The internal byte array of the tuple. - */ - public byte[] getBytes() { - return bytes; - } - - /** - * Returns the length of the column with the specified index. Column indices start at 0. - * - * @param colNumber Index of the column. Indices start at 0. - * @return The length of the specified column. - */ - public int getColumnLength(int colNumber) { - if(offsets == null) return -1; - if(colNumber < 0) return -1; - if(colNumber >= offsets.length) return -1; - return offsets[colNumber + 1] - offsets[colNumber] - 1; - } - - // ------------------------------------------------------------------------ - // Modification - // ------------------------------------------------------------------------ - - /** - * Appends all columns of the specified tuple to this tuple. - * - * @param other The tuple whose columns are appended to this tuple. - */ - public void concatenate(Tuple other) { - - if(other.getBytes() == null) return; - - if (bytes == null) { - bytes = (byte[]) other.bytes.clone(); - offsets = (int[]) other.offsets.clone(); - numCols = other.numCols; - } else { - int len = offsets[numCols]; - int otherLen = other.offsets[other.numCols]; - int totalLen = len + otherLen; - - // bytes: - // our content - if (bytes.length < totalLen) { - byte[] tmp = new byte[totalLen]; - System.arraycopy(bytes, 0, tmp, 0, len); - bytes = tmp; - } - - // the other's content - System.arraycopy(other.bytes, 0, bytes, len, otherLen); - - // offsets - if (offsets.length < numCols + other.numCols + 1) { - int[] tmp = new int[numCols + other.numCols + 1]; - System.arraycopy(offsets, 0, tmp, 0, numCols + 1); - offsets = tmp; - } - - // other offsets - for (int i = 1; i < other.numCols + 1; i++) { - offsets[numCols + i] = other.offsets[i] + len; - } - - numCols += other.numCols; - } - } - - /** - * Performs a projection on the tuple. - * The int parameter is interpreted as a bitmap on the columns. - * I.e. a bitmap value of 1 projects to the first column, 2 to the second, 3 to the first two columns, and so on. - * - * @param bitmap the projection bitmap. - */ - public void project(int bitmap) { - int[] lengths = new int[numCols]; - int lenCount = 0; - - if(bytes == null || offsets == null) return; - - // go through the bitmap and find the indexes of the columns to retain - int k = 0; - for (int i = 0; bitmap != 0 && i < numCols; i++, bitmap >>>= 1) { - if ((bitmap & 0x1) != 0) { - int len = offsets[i + 1] - offsets[i]; - lengths[k] = len; - lenCount += len; - offsets[k] = offsets[i]; - k++; - } - } - numCols = k; - - // allocate the new (smaller) array - byte[] tmp = new byte[lenCount]; - lenCount = 0; - - // copy the columns to the beginning and adjust the offsets to the new array - for (int i = 0; i < k; i++) { - System.arraycopy(bytes, offsets[i], tmp, lenCount, lengths[i]); - offsets[i] = lenCount; - lenCount += lengths[i]; - } - - bytes = tmp; - offsets[numCols] = tmp.length; - } - - /** - * Compares a String attribute of this tuple with a String attribute of another tuple. - * The strings are compared lexicographic. - * - * @param other The other tuple. - * @param thisColumn The index of this tuple's String attribute. - * @param otherColumn The index of the other tuple's String attribute. - * @return 1 if this tuple's attribute is greater, 0 if both attributes have the same value, - * -1 if this tuple's attribute is smaller. - * @throws IndexOutOfBoundsException Thrown if one of the column indices is invalid (smaller than 0 or bigger - * than the attribute count). - */ - public int compareStringAttribute(Tuple other, int thisColumn, int otherColumn) { - - if(thisColumn < 0) throw new IndexOutOfBoundsException(); - if(otherColumn < 0) throw new IndexOutOfBoundsException(); - if(thisColumn >= numCols) throw new IndexOutOfBoundsException(); - if(otherColumn >= other.numCols) throw new IndexOutOfBoundsException(); - - int len = getColumnLength(thisColumn); - int otherLen = other.getColumnLength(otherColumn); - int min = Math.min(len, otherLen); - - int startPos = offsets[thisColumn]; - int otherStartPos = other.offsets[otherColumn]; - - for (int i = 0; i < min; i++) { - if (bytes[startPos + i] < other.bytes[otherStartPos + i]) { - return -1; - } else if (bytes[startPos + i] > other.bytes[otherStartPos + i]) { - return 1; - } - } - - if (len < otherLen) { - return -1; - } else if (len > otherLen) { - return 1; - } else { - return 0; - } - } - - /** - * Compares an Integer attribute of this tuple with an Integer attribute of another tuple. - * - * @param other The other tuple. - * @param thisColumn The index of this tuple's int attribute. - * @param otherColumn The index of the other tuple's int attribute. - * @return 1 if this tuple's attribute is greater, 0 if both attributes have the same value, - * -1 if this tuple's attribute is smaller. - * @throws IndexOutOfBoundsException Thrown if one of the column indices is invalid (smaller than 0 or bigger - * than the attribute count). - */ - public int compareIntAttribute(Tuple other, int thisColumn, int otherColumn) { - int len = getColumnLength(thisColumn); - int otherLen = other.getColumnLength(otherColumn); - - if(thisColumn < 0) throw new IndexOutOfBoundsException(); - if(otherColumn < 0) throw new IndexOutOfBoundsException(); - if(thisColumn >= numCols) throw new IndexOutOfBoundsException(); - if(otherColumn >= other.numCols) throw new IndexOutOfBoundsException(); - - short thisNegative = 1; - short otherNegative = 1; - - if(this.bytes[offsets[thisColumn]] == '-') { - thisNegative = -1; - } - - if(other.getBytes()[other.offsets[otherColumn]] == '-') { - otherNegative = -1; - } - - // check one int is negative - if(thisNegative != otherNegative) { - return thisNegative; - } - - // check if they vary in length - if (len < otherLen) { - return -1 * thisNegative; - } else if (len > otherLen) { - return 1 * thisNegative; - } - - // both have the same orientation and length, check digit-wise - int myStartPos = offsets[thisColumn]; - int compStartPos = other.offsets[otherColumn]; - - for (int i = 0; i < len; i++) { - if (bytes[myStartPos + i] < other.bytes[compStartPos + i]) { - return -1 * thisNegative; - } else if (bytes[myStartPos + i] > other.bytes[compStartPos + i]) { - return 1 * thisNegative; - } - } - return 0; - - } - - /** - * Returns the String value of the attribute with the specified index. - * - * @param column The index of the attribute whose String value is returned. - * @return The String value of the specified attribute. - * @throws IndexOutOfBoundsException Thrown if the index of the column is invalid (smaller than 0 or bigger - * than the attribute count). - */ - public String getStringValueAt(int column) throws IndexOutOfBoundsException { - // check for validity of column index - if(column < 0) throw new IndexOutOfBoundsException(); - if(column >= numCols) throw new IndexOutOfBoundsException(); - - int off = offsets[column]; - int len = getColumnLength(column); - - char[] chars = new char[len]; - for (int i = 0; i < len; i++) { - chars[i] = (char) (bytes[off + i] & 0xff); - } - - return new String(chars); - } - - /** - * Returns the Long value of the attribute with the specified index. - * The value must be represented in the decimal system. - * - * @param column The index of the attribute whose value is returned as long. - * @return The long value of the specified attribute. - * @throws IndexOutOfBoundsException Thrown if the index of the column is invalid (smaller than 0 or bigger - * than the attribute count). - * @throws NumberFormatException Thrown if the attribute is not a valid long value - * (contains any other character than digits or '-'.) - */ - public long getLongValueAt(int column) throws IndexOutOfBoundsException, NumberFormatException { - - if(column < 0) throw new IndexOutOfBoundsException(); - if(column >= numCols) throw new IndexOutOfBoundsException(); - - int off = offsets[column]; - int len = getColumnLength(column); - - boolean isNegative = false; - - if(bytes[off] == '-') { - isNegative = true; - off++; - len--; - } - - long value = 0; - for (int i = off; i < off + len; i++) { - - if(bytes[i] < '0' || bytes[i] > '9') throw new NumberFormatException(); - - value *= 10; - value += (bytes[i] - 48); - } - - if(isNegative) { - value *= -1; - } - - return value; - } - - /** - * Returns an attribute which is specified by an index as byte array. - * - * @param column The index of the attribute which is returned as byte array. - * @return The value of the specified attribute as byte array value. - * @throws IndexOutOfBoundsException Thrown if the index of the column is invalid (smaller than 0 or bigger - * than the attribute count). - */ - public byte[] getByteArrayValueAt(int column) throws IndexOutOfBoundsException { - - if(column < 0) throw new IndexOutOfBoundsException(); - if(column >= numCols) throw new IndexOutOfBoundsException(); - - int len = getColumnLength(column); - byte[] buffer = new byte[len]; - System.arraycopy(bytes, offsets[column], buffer, 0, len); - return buffer; - } - - /** - * Sets the size of the internal byte array of the tuple to the minimum capacity. - * If the minimum capacity is smaller than the current size of the tuple's byte array, - * nothing is done. Otherwise a new byte array is allocated and the content of the old one copied. - * - * @param minCapacity The new size of the internal byte array. - */ - public void reserveSpace(int minCapacity) { - if (bytes.length < minCapacity) { - byte[] tmp = new byte[minCapacity]; - System.arraycopy(bytes, 0, tmp, 0, offsets[numCols]); - bytes = tmp; - } - } - - /** - * Reduces the size of the internal byte and offset arrays to the currently used size. - */ - public void compact() { - int len = offsets[numCols]; - - if (bytes.length > len) { - byte[] tmp = new byte[len]; - System.arraycopy(bytes, 0, tmp, 0, len); - bytes = tmp; - } - - if (offsets.length > numCols + 1) { - int[] tmp = new int[numCols + 1]; - System.arraycopy(offsets, 0, tmp, 0, numCols + 1); - offsets = tmp; - } - } - - /** - * Appends an attribute at the end of the tuple. - * - * @param attValue The attribute to append. - */ - public void addAttribute(byte[] attValue) { - int end; - - if (numCols == 0) { - offsets = new int[5]; - bytes = new byte[Math.max(256, attValue.length + 1)]; - end = 0; - } else { - end = offsets[numCols]; - - // increase offset array, if necessary - if (numCols + 1 >= offsets.length) { - int[] tmp = new int[offsets.length * 2]; - System.arraycopy(offsets, 0, tmp, 0, numCols + 1); - offsets = tmp; - } - - // increase byte buffer, if necessary - if (bytes.length < end + attValue.length + 1) { - byte[] tmp = new byte[bytes.length + attValue.length + 1]; - System.arraycopy(bytes, 0, tmp, 0, end); - bytes = tmp; - } - } - - // copy bytes, offsets and increase columns - System.arraycopy(attValue, 0, bytes, end, attValue.length); - end += attValue.length; - bytes[end++] = '|'; - numCols++; - offsets[numCols] = end; - } - - /** - * Appends an attribute at the end of the tuple. - * - * @param attValue The attribute to append. - */ - public void addAttribute(String attValue) { - int end; - - if (numCols == 0) { - offsets = new int[5]; - bytes = new byte[Math.max(256, attValue.length() + 1)]; - end = 0; - } else { - end = offsets[numCols]; - - // increase offset array, if necessary - if (numCols + 1 >= offsets.length) { - int[] tmp = new int[offsets.length * 2]; - System.arraycopy(offsets, 0, tmp, 0, numCols + 1); - offsets = tmp; - } - - // increase byte buffer, if necessary - if (bytes.length < end + attValue.length() + 1) { - byte[] tmp = new byte[bytes.length + attValue.length() + 1]; - System.arraycopy(bytes, 0, tmp, 0, end); - bytes = tmp; - } - } - - // copy bytes, offsets and increase columns - for (int i = 0; i < attValue.length(); i++, end++) { - bytes[end] = (byte) (attValue.charAt(i) & 0xff); - } - bytes[end++] = '|'; - numCols++; - offsets[numCols] = end; - } - - /** - * Appends an attribute by copying it from another tuple. - * - * @param other The other tuple to copy from. - * @param column The index of the attribute to copy within the other tuple. - */ - public void addAttributeFromKVRecord(Tuple other, int column) { - int len = other.getColumnLength(column) + 1; - int end; - - if (numCols == 0) { - offsets = new int[5]; - bytes = new byte[Math.max(256, len)]; - end = 0; - } else { - end = offsets[numCols]; - - // increase offset array, if necessary - if (numCols + 1 >= offsets.length) { - int[] tmp = new int[offsets.length * 2]; - System.arraycopy(offsets, 0, tmp, 0, numCols + 1); - offsets = tmp; - } - - // increase byte buffer, if necessary - if (bytes.length < end + len) { - byte[] tmp = new byte[end + len]; - System.arraycopy(bytes, 0, tmp, 0, end); - bytes = tmp; - } - } - - System.arraycopy(other.bytes, other.offsets[column], bytes, end, len); - numCols++; - offsets[numCols] = end + len; - } - - public void setContents(byte[] bytes, int offset, int len, char delimiter) - { - // make space - if (this.bytes == null || this.bytes.length < len) { - this.bytes = new byte[len]; - } - - // copy the binary data - System.arraycopy(bytes, offset, this.bytes, 0, len); - - int readPos = offset; - - // allocate the offsets array - if (this.offsets == null) { - this.offsets = new int[4]; - } - - int col = 1; // the column we are in - - int startPos = readPos; - - while (readPos < offset + len) { - if (bytes[readPos++] == delimiter) { - if (offsets.length <= col) { - int newOffsets[] = new int[this.offsets.length * 2]; - System.arraycopy(this.offsets, 0, newOffsets, 0, this.offsets.length); - this.offsets = newOffsets; - } - this.offsets[col++] = readPos - startPos; - } - } - - this.numCols = col - 1; - } - - - // ------------------------------------------------------------------------ - // Serialization - // ------------------------------------------------------------------------ - - @Override - public void read(DataInputView in) throws IOException { - // read the bytes - int numBytes = in.readInt(); - if (numBytes > 0) { - bytes = new byte[numBytes]; - in.readFully(bytes); - - // read the offsets - numCols = in.readInt() + 1; - offsets = new int[numCols + 1]; - for (int i = 1; i < numCols; i++) { - offsets[i] = in.readInt(); - } - // set last offset - offsets[numCols] = numBytes; - } else { - numCols = 0; - } - } - - @Override - public void write(DataOutputView out) throws IOException { - // write the bytes - int numBytes = (numCols > 0 ? offsets[numCols] : 0); - out.writeInt(numBytes); - if (numBytes > 0) { - out.write(bytes, 0, numBytes); - - // write the offsets - // exclude first and last - out.writeInt(numCols - 1); - for (int i = 1; i < numCols; i++) { - out.writeInt(offsets[i]); - } - } - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(); - - for (int i = 0; i < numCols; i++) { - for (int k = 0; k < getColumnLength(i); k++) { - bld.append((char) (bytes[offsets[i] + k] & 0xff)); - } - bld.append('|'); - } - - return bld.toString(); - } - -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/UniformIntInput.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/UniformIntInput.java deleted file mode 100644 index 5e880cb1226f8..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/UniformIntInput.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.recordJobs.util; - -import org.apache.flink.api.java.record.io.GenericInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -/** - * - */ -public class UniformIntInput extends GenericInputFormat { - private static final long serialVersionUID = 1L; - - public static final String NUM_KEYS_KEY = "testfomat.numkeys"; - public static final String NUM_VALUES_KEY = "testfomat.numvalues"; - - private static final int DEFAULT_NUM_KEYS = 1000; - private static final int DEFAULT_NUM_VALUES = 1000; - - private final IntValue key = new IntValue(); - private final IntValue value = new IntValue(); - - private int numKeys; - private int numValues; - - private int keyInt; - private int valueInt; - - public UniformIntInput() { - this(DEFAULT_NUM_KEYS, DEFAULT_NUM_VALUES); - } - - public UniformIntInput(final int numKeys, final int numValues) { - this.numKeys = numKeys; - this.numValues = numValues; - } - - - - - @Override - public void configure(Configuration parameters) { - super.configure(parameters); - - this.numKeys = parameters.getInteger(NUM_KEYS_KEY, this.numKeys); - this.numValues = parameters.getInteger(NUM_VALUES_KEY, this.numValues); - } - - - @Override - public boolean reachedEnd() { - return this.valueInt >= this.numValues; - } - - - @Override - public Record nextRecord(Record record) { - if (this.keyInt == this.numKeys) { - this.keyInt = 0; - this.valueInt++; - } - - this.key.setValue(this.keyInt); - this.value.setValue(this.valueInt); - - record.setField(0, this.key); - record.setField(1, this.value); - record.updateBinaryRepresenation(); - - this.keyInt++; - - return record; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java deleted file mode 100644 index 96eb1fc539a07..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.recordJobs.wordcount; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.client.LocalExecutor; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.Collector; - -import java.util.Iterator; -import java.util.StringTokenizer; -import java.util.concurrent.TimeUnit; - -/** - * Implements a word count which takes the input file and counts the number of - * the occurrences of each word in the file. - */ -@SuppressWarnings("deprecation") -public class WordCount implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - - /** - * Converts a Record containing one string in to multiple string/integer pairs. - * The string is tokenized by whitespaces. For each token a new record is emitted, - * where the token is the first field and an Integer(1) is the second field. - */ - public static class TokenizeLine extends MapFunction { - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector collector) { - // get the first field (as type StringValue) from the record - String line = record.getField(0, StringValue.class).getValue(); - - // normalize the line - line = line.replaceAll("\\W+", " ").toLowerCase(); - - // tokenize the line - StringTokenizer tokenizer = new StringTokenizer(line); - while (tokenizer.hasMoreTokens()) { - String word = tokenizer.nextToken(); - - // we emit a (word, 1) pair - collector.collect(new Record(new StringValue(word), new IntValue(1))); - } - } - } - - /** - * Sums up the counts for a certain given key. The counts are assumed to be at position 1 - * in the record. The other fields are not modified. - */ - @Combinable - @ConstantFields(0) - public static class CountWords extends ReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - Record element = null; - int sum = 0; - while (records.hasNext()) { - element = records.next(); - int cnt = element.getField(1, IntValue.class).getValue(); - sum += cnt; - } - - element.setField(1, new IntValue(sum)); - out.collect(element); - } - - @Override - public void combine(Iterator records, Collector out) throws Exception { - // the logic is the same as in the reduce function, so simply call the reduce method - reduce(records, out); - } - } - - - @Override - public Plan getPlan(String... args) { - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataInput = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - FileDataSource source = new FileDataSource(new TextInputFormat(), dataInput, "Input Lines"); - MapOperator mapper = MapOperator.builder(new TokenizeLine()) - .input(source) - .name("Tokenize Lines") - .build(); - ReduceOperator reducer = ReduceOperator.builder(CountWords.class, StringValue.class, 0) - .input(mapper) - .name("Count Words") - .build(); - - @SuppressWarnings("unchecked") - FileDataSink out = new FileDataSink(new CsvOutputFormat("\n", " ", StringValue.class, IntValue.class), output, reducer, "Word Counts"); - - Plan plan = new Plan(out, "WordCount Example"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } - - - public static void main(String[] args) throws Exception { - WordCount wc = new WordCount(); - - if (args.length < 3) { - System.err.println(wc.getDescription()); - System.exit(1); - } - - Plan plan = wc.getPlan(args); - - // This will execute the word-count embedded in a local context. replace this line by the commented - // succeeding line to send the job to a local installation or to a cluster for execution - JobExecutionResult result = LocalExecutor.execute(plan); - System.err.println("Total runtime: " + result.getNetRuntime(TimeUnit.MILLISECONDS) + " ms"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java deleted file mode 100644 index 780db582b38f7..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.recordJobs.wordcount; - -import java.io.Serializable; -import java.util.HashSet; -import java.util.Iterator; -import java.util.StringTokenizer; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.accumulators.Histogram; -import org.apache.flink.api.common.accumulators.LongCounter; -import org.apache.flink.api.java.record.functions.MapFunction; -import org.apache.flink.api.java.record.functions.ReduceFunction; -import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.record.io.CsvOutputFormat; -import org.apache.flink.api.java.record.io.TextInputFormat; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable; -import org.apache.flink.client.LocalExecutor; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; - -/** - * This is similar to the WordCount example and additionally demonstrates how to - * use custom accumulators (built-in or custom). - */ -@SuppressWarnings("deprecation") -public class WordCountAccumulators implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - public static class TokenizeLine extends MapFunction implements Serializable { - private static final long serialVersionUID = 1L; - - // For efficiency it is recommended to have member variables for the accumulators - public static final String ACCUM_NUM_LINES = "accumulator.num-lines"; - private LongCounter numLines = new LongCounter(); - - // This histogram accumulator collects the distribution of number of words per line - public static final String ACCUM_WORDS_PER_LINE = "accumulator.words-per-line"; - private Histogram wordsPerLine = new Histogram(); - - public static final String ACCUM_DISTINCT_WORDS = "accumulator.distinct-words"; - private SetAccumulator distinctWords = new SetAccumulator(); - - - @Override - public void open(Configuration parameters) throws Exception { - - // Accumulators have to be registered to the system - getRuntimeContext().addAccumulator(ACCUM_NUM_LINES, this.numLines); - getRuntimeContext().addAccumulator(ACCUM_WORDS_PER_LINE, this.wordsPerLine); - getRuntimeContext().addAccumulator(ACCUM_DISTINCT_WORDS, this.distinctWords); - - // You could also write to accumulators in open() or close() - } - - @Override - public void map(Record record, Collector collector) { - - // Increment counter - numLines.add(1L); - - // get the first field (as type StringValue) from the record - String line = record.getField(0, StringValue.class).getValue(); - - // normalize the line - line = line.replaceAll("\\W+", " ").toLowerCase(); - - // tokenize the line - StringTokenizer tokenizer = new StringTokenizer(line); - int numWords = 0; - - while (tokenizer.hasMoreTokens()) { - String word = tokenizer.nextToken(); - - distinctWords.add(new StringValue(word)); - ++numWords; - - // we emit a (word, 1) pair - collector.collect(new Record(new StringValue(word), new IntValue(1))); - } - - // Add a value to the histogram accumulator - this.wordsPerLine.add(numWords); - } - } - - @Combinable - @ConstantFields(0) - public static class CountWords extends ReduceFunction implements Serializable { - - private static final long serialVersionUID = 1L; - - private final IntValue cnt = new IntValue(); - - @Override - public void reduce(Iterator records, Collector out) { - Record element = null; - int sum = 0; - while (records.hasNext()) { - element = records.next(); - IntValue i = element.getField(1, IntValue.class); - sum += i.getValue(); - } - - this.cnt.setValue(sum); - element.setField(1, this.cnt); - out.collect(element); - } - } - - @Override - public Plan getPlan(String... args) { - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataInput = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - FileDataSource source = new FileDataSource(new TextInputFormat(), dataInput, "Input Lines"); - - MapOperator mapper = MapOperator.builder(new TokenizeLine()).input(source).name("Tokenize Lines").build(); - - ReduceOperator reducer = ReduceOperator.builder(CountWords.class, StringValue.class, 0).input(mapper) - .name("Count Words").build(); - - FileDataSink out = new FileDataSink(new CsvOutputFormat(), output, reducer, "Word Counts"); - - CsvOutputFormat.configureRecordFormat(out).recordDelimiter('\n') - .fieldDelimiter(' ').field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "WordCount Example"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: [numSubStasks] [input] [output]"; - } - - public static void main(String[] args) throws Exception { - WordCountAccumulators wc = new WordCountAccumulators(); - - if (args.length < 3) { - System.err.println(wc.getDescription()); - System.exit(1); - } - - Plan plan = wc.getPlan(args); - - JobExecutionResult result = LocalExecutor.execute(plan); - - // Accumulators can be accessed by their name. - System.out.println("Number of lines counter: "+ result.getAccumulatorResult(TokenizeLine.ACCUM_NUM_LINES)); - System.out.println("Words per line histogram: " + result.getAccumulatorResult(TokenizeLine.ACCUM_WORDS_PER_LINE)); - System.out.println("Distinct words: " + result.getAccumulatorResult(TokenizeLine.ACCUM_DISTINCT_WORDS)); - } - - /** - * Custom accumulator - */ - public static class SetAccumulator implements Accumulator> { - - private static final long serialVersionUID = 1L; - - private HashSet set = new HashSet(); - - @Override - public void add(T value) { - this.set.add(value); - } - - @Override - public HashSet getLocalValue() { - return this.set; - } - - @Override - public void resetLocal() { - this.set.clear(); - } - - @Override - public void merge(Accumulator> other) { - // build union - this.set.addAll(other.getLocalValue()); - } - - @Override - public Accumulator> clone() { - SetAccumulator result = new SetAccumulator(); - result.set.addAll(set); - return result; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java index 310ded8244f9f..ec2dbb75fa110 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java @@ -29,8 +29,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.test.util.RecordAPITestBase; -import org.junit.After; import org.junit.Ignore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +36,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.test.util.JavaProgramTestBase; @Ignore public class NetworkStackThroughputITCase { @@ -62,8 +62,8 @@ public class NetworkStackThroughputITCase { // ------------------------------------------------------------------------ - // wrapper to reuse RecordAPITestBase code in runs via main() - private static class TestBaseWrapper extends RecordAPITestBase { + // wrapper to reuse JavaProgramTestBase code in runs via main() + private static class TestBaseWrapper extends JavaProgramTestBase { private int dataVolumeGb; private boolean useForwarder; @@ -90,7 +90,6 @@ public TestBaseWrapper(Configuration config) { setTaskManagerNumSlots(numSlots); } - @Override protected JobGraph getJobGraph() throws Exception { return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, parallelism); } @@ -138,19 +137,19 @@ private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean return jobGraph; } - @After - public void calculateThroughput() { - if (getJobExecutionResult() != null) { - int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1); - long dataVolumeMbit = dataVolumeGb * 8192; - long runtimeSecs = getJobExecutionResult().getNetRuntime(TimeUnit.SECONDS); + @Override + protected void testProgram() throws Exception { + JobExecutionResult jer = executor.submitJobAndWait(getJobGraph(), false); + int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1); + + long dataVolumeMbit = dataVolumeGb * 8192; + long runtimeSecs = jer.getNetRuntime(TimeUnit.SECONDS); - int mbitPerSecond = (int) (((double) dataVolumeMbit) / runtimeSecs); + int mbitPerSecond = (int) (((double) dataVolumeMbit) / runtimeSecs); - LOG.info(String.format("Test finished with throughput of %d MBit/s (runtime [secs]: %d, " + - "data volume [gb/mbits]: %d/%d)", mbitPerSecond, runtimeSecs, dataVolumeGb, dataVolumeMbit)); - } + LOG.info(String.format("Test finished with throughput of %d MBit/s (runtime [secs]: %d, " + + "data volume [gb/mbits]: %d/%d)", mbitPerSecond, runtimeSecs, dataVolumeGb, dataVolumeMbit)); } } @@ -289,8 +288,7 @@ public void testThroughput() throws Exception { TestBaseWrapper test = new TestBaseWrapper(config); System.out.println(Arrays.toString(p)); - test.testJob(); - test.calculateThroughput(); + test.testProgram(); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/IntTupleDataInFormatTest.java b/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/IntTupleDataInFormatTest.java deleted file mode 100644 index 0cff9b6905466..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/IntTupleDataInFormatTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.testPrograms.util.tests; - -import org.apache.flink.test.recordJobs.util.IntTupleDataInFormat; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.junit.Assert; -import org.junit.Test; - -public class IntTupleDataInFormatTest -{ - @Test - public void testReadLineKeyValuePairOfIntValueTupleByteArray() { - - String[] testTuples = { - "1|attribute1|attribute2|3|attribute4|5|", - "2|3|", - "3|attribute1|attribute2|", - "-1|attr1|attr2|", - "-2|attribute1|attribute2|", - Integer.MAX_VALUE+"|attr1|attr2|attr3|attr4|", - Integer.MIN_VALUE+"|attr1|attr2|attr3|attr4|" - }; - - int[] expectedKeys = { - 1,2,3,-1,-2,Integer.MAX_VALUE,Integer.MIN_VALUE - }; - - int[] expectedAttrCnt = {6,2,3,3,3,5,5}; - - IntTupleDataInFormat inFormat = new IntTupleDataInFormat(); - Record rec = new Record(); - - for(int i = 0; i < testTuples.length; i++) { - - byte[] tupleBytes = testTuples[i].getBytes(); - - inFormat.readRecord(rec, tupleBytes, 0, tupleBytes.length); - - Assert.assertTrue("Expected Key: "+expectedKeys[i]+" != Returned Key: "+rec.getField(0, IntValue.class), rec.getField(0, IntValue.class).equals(new IntValue(expectedKeys[i]))); - Assert.assertTrue("Expected Attr Cnt: "+expectedAttrCnt[i]+" != Returned Attr Cnt: "+rec.getField(1, Tuple.class), rec.getField(1, Tuple.class).getNumberOfColumns() == expectedAttrCnt[i]); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/TupleTest.java b/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/TupleTest.java deleted file mode 100644 index 41ec6b983abd6..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/testPrograms/util/tests/TupleTest.java +++ /dev/null @@ -1,751 +0,0 @@ -/* - * 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.testPrograms.util.tests; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import org.apache.flink.core.memory.InputViewDataInputStreamWrapper; -import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper; -import org.apache.flink.test.recordJobs.util.Tuple; -import org.junit.Assert; -import org.junit.Test; - -public class TupleTest { - - @Test - public void testTupleByteArrayShortArrayInt() { - - String tupleStr = "attr1|attr2|3|4|attr5|"; - int[] offsets = {0,6,12,14,16,22}; - Tuple t1 = new Tuple(tupleStr.getBytes(),offsets,5); - - Assert.assertTrue(t1.getBytes().length == tupleStr.getBytes().length); - for(int i=0;i 0); - Assert.assertTrue(t2.compareStringAttribute(t1, 1, 1) > 0); - Assert.assertTrue(t2.compareStringAttribute(t1, 2, 2) == 0); - Assert.assertTrue(t2.compareStringAttribute(t1, 3, 2) > 0); - - Assert.assertTrue(t1.compareStringAttribute(t1, 0, 0) == 0); - Assert.assertTrue(t1.compareStringAttribute(t1, 1, 1) == 0); - Assert.assertTrue(t1.compareStringAttribute(t1, 2, 2) == 0); - Assert.assertTrue(t1.compareStringAttribute(t1, 0, 1) < 0); - Assert.assertTrue(t1.compareStringAttribute(t1, 2, 1) > 0); - - // check for out-of-bounds values - boolean exceptionThrown = false; - try { - t1.compareStringAttribute(t1, 0, 3); - } catch(IndexOutOfBoundsException ioobe) { - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); - - exceptionThrown = false; - try { - t1.compareStringAttribute(t1, 4, 0); - } catch(IndexOutOfBoundsException ioobe) { - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); - } - - @Test - public void testCompareIntAttribute() { - Tuple t1 = new Tuple(); - - t1.addAttribute("1"); - t1.addAttribute("2"); - t1.addAttribute("112315412"); - t1.addAttribute(Integer.MAX_VALUE+""); - t1.addAttribute("-1"); - t1.addAttribute(Integer.MIN_VALUE+""); - - // check identical values - Assert.assertTrue(t1.compareIntAttribute(t1, 0, 0) == 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 1, 1) == 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 2, 2) == 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 3, 3) == 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 4, 4) == 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 5, 5) == 0); - - // check unequal values - Assert.assertTrue(t1.compareIntAttribute(t1, 0, 1) < 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 1, 0) > 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 1, 2) < 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 2, 1) > 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 2, 3) < 0); - - // check negative values - Assert.assertTrue(t1.compareIntAttribute(t1, 0, 4) > 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 4, 0) < 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 4, 5) > 0); - Assert.assertTrue(t1.compareIntAttribute(t1, 5, 4) < 0); - - // check for non-existing attributes - boolean exceptionThrown = false; - try { - t1.compareIntAttribute(t1, 0, 6); - } catch(IndexOutOfBoundsException ioobe) { - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); - - exceptionThrown = false; - try { - t1.compareIntAttribute(t1, 7, 0); - } catch(IndexOutOfBoundsException ioobe) { - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); - } - - @Test - public void testGetStringValueAt() { - - Tuple t = new Tuple(); - - String[] testStrings = {"a","b","123123","Hello world!","Test with p|pe","!ยง$%&/()=*'.:,;-_#+'`}][{"}; - - for(String testString : testStrings) { - t.addAttribute(testString); - } - - // check for same value - for(int i=0;i { private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - + private static final int DELAY = 20; + private final boolean delay; + + public InfiniteIntegerInputFormat(boolean delay) { + this.delay = delay; + } @Override public boolean reachedEnd() throws IOException { return false; } - @Override - public Record nextRecord(Record record) throws IOException{ - record.setField(0, this.one); - return record; + public Integer nextRecord(Integer reuse) throws IOException { + if (delay) { + try { + Thread.sleep(DELAY); + } catch (InterruptedException iex) { + // do nothing + } + } + return 1; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java similarity index 63% rename from flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java rename to flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java index f1de48d90135b..ad6edd4a3edec 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java @@ -15,40 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - -package org.apache.flink.test.recordJobs.util; +package org.apache.flink.test.util; import java.io.IOException; +import org.apache.flink.api.common.io.GenericInputFormat; +import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.record.io.GenericInputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -public class InfiniteIntegerInputFormatWithDelay extends GenericInputFormat { +public class InfiniteIntegerTupleInputFormat extends GenericInputFormat> { private static final long serialVersionUID = 1L; - private static final int DELAY = 20; - - private final IntValue one = new IntValue(1); - + private final boolean delay; + + public InfiniteIntegerTupleInputFormat(boolean delay) { + this.delay = delay; + } @Override public boolean reachedEnd() throws IOException { return false; } - @Override - public Record nextRecord(Record record) throws IOException { - record.setField(0, this.one); - - try { - Thread.sleep(DELAY); - } catch (InterruptedException iex) { - // do nothing + public Tuple2 nextRecord(Tuple2 reuse) throws IOException { + if (delay) { + try { + Thread.sleep(DELAY); + } catch (InterruptedException iex) { + // do nothing + } } - - return record; + return new Tuple2<>(1, 1); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java new file mode 100644 index 0000000000000..b0dd78ef18d82 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java @@ -0,0 +1,58 @@ +/* + * 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.util; + +import java.text.DecimalFormat; +import java.text.DecimalFormatSymbols; +import org.apache.flink.api.java.io.TextOutputFormat.TextFormatter; +import org.apache.flink.api.java.tuple.Tuple2; + +/** + * Writes records that contain an id and a CoordVector. The output format is line-based, i.e. one record is written to a + * line and terminated with '\n'. Within a line the first '|' character separates the id from the CoordVector. The + * vector consists of a vector of decimals. The decimals are separated by '|'. The is is the id of a data point or + * cluster center and the vector the corresponding position (coordinate vector) of the data point or cluster center. + * Example line: "42|23.23|52.57|74.43| Id: 42 Coordinate vector: (23.23, 52.57, 74.43) + */ +public class PointFormatter implements TextFormatter> { + private static final long serialVersionUID = 1L; + + private final DecimalFormat df = new DecimalFormat("####0.00"); + private final StringBuilder line = new StringBuilder(); + + public PointFormatter() { + DecimalFormatSymbols dfSymbols = new DecimalFormatSymbols(); + dfSymbols.setDecimalSeparator('.'); + this.df.setDecimalFormatSymbols(dfSymbols); + } + + @Override + public String format(Tuple2 value) { + line.setLength(0); + + line.append(value.f0); + + for (double coord : value.f1.getCoordinates()) { + line.append('|'); + line.append(df.format(coord)); + } + line.append('|'); + + return line.toString(); + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java similarity index 59% rename from flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java rename to flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java index c5dd8ec5a39da..424b7813dc64b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java @@ -15,55 +15,46 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.flink.test.util; -package org.apache.flink.test.recordJobs.kmeans.udfs; - +import java.io.IOException; import java.util.ArrayList; import java.util.List; - -import org.apache.flink.api.java.record.io.DelimitedInputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; +import org.apache.flink.api.common.io.DelimitedInputFormat; +import org.apache.flink.api.java.tuple.Tuple2; /** - * Generates records with an id and a and CoordVector. - * The input format is line-based, i.e. one record is read from one line - * which is terminated by '\n'. Within a line the first '|' character separates - * the id from the CoordVector. The vector consists of a vector of decimals. - * The decimals are separated by '|' as well. The id is the id of a data point or - * cluster center and the CoordVector the corresponding position (coordinate - * vector) of the data point or cluster center. Example line: - * "42|23.23|52.57|74.43| Id: 42 Coordinate vector: (23.23, 52.57, 74.43) + * Generates records with an id and a and CoordVector. The input format is line-based, i.e. one record is read from one + * line which is terminated by '\n'. Within a line the first '|' character separates the id from the CoordVector. The + * vector consists of a vector of decimals. The decimals are separated by '|' as well. The id is the id of a data point + * or cluster center and the CoordVector the corresponding position (coordinate vector) of the data point or cluster + * center. Example line: "42|23.23|52.57|74.43| Id: 42 Coordinate vector: (23.23, 52.57, 74.43) */ -public class PointInFormat extends DelimitedInputFormat { +public class PointInFormat extends DelimitedInputFormat> { private static final long serialVersionUID = 1L; - - private final IntValue idInteger = new IntValue(); - private final CoordVector point = new CoordVector(); - + private final List dimensionValues = new ArrayList(); private double[] pointValues = new double[0]; - + @Override - public Record readRecord(Record record, byte[] line, int offset, int numBytes) { - + public Tuple2 readRecord(Tuple2 reuse, byte[] bytes, int offset, int numBytes) throws IOException { + final int limit = offset + numBytes; - + int id = -1; int value = 0; int fractionValue = 0; int fractionChars = 0; boolean negative = false; - + this.dimensionValues.clear(); for (int pos = offset; pos < limit; pos++) { - if (line[pos] == '|') { + if (bytes[pos] == '|') { // check if id was already set if (id == -1) { id = value; - } - else { + } else { double v = value + ((double) fractionValue) * Math.pow(10, (-1 * (fractionChars - 1))); this.dimensionValues.add(negative ? -v : v); } @@ -72,26 +63,25 @@ public Record readRecord(Record record, byte[] line, int offset, int numBytes) { fractionValue = 0; fractionChars = 0; negative = false; - } else if (line[pos] == '.') { + } else if (bytes[pos] == '.') { fractionChars = 1; - } else if (line[pos] == '-') { + } else if (bytes[pos] == '-') { negative = true; } else { if (fractionChars == 0) { value *= 10; - value += line[pos] - '0'; + value += bytes[pos] - '0'; } else { fractionValue *= 10; - fractionValue += line[pos] - '0'; + fractionValue += bytes[pos] - '0'; fractionChars++; } } } // set the ID - this.idInteger.setValue(id); - record.setField(0, this.idInteger); - + reuse.f0 = id; + // set the data points if (this.pointValues.length != this.dimensionValues.size()) { this.pointValues = new double[this.dimensionValues.size()]; @@ -100,8 +90,7 @@ public Record readRecord(Record record, byte[] line, int offset, int numBytes) { this.pointValues[i] = this.dimensionValues.get(i); } - this.point.setCoordinates(this.pointValues); - record.setField(1, this.point); - return record; + reuse.f1 = new CoordVector(this.pointValues); + return reuse; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java new file mode 100644 index 0000000000000..c779275cfb34a --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java @@ -0,0 +1,54 @@ +/* + * 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.util; + +import java.io.IOException; +import org.apache.flink.api.common.io.GenericInputFormat; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; + +public class UniformIntTupleGeneratorInputFormat extends GenericInputFormat> { + private final int keyTotal; + private final int valueTotal; + private int valueCount = 0; + private UniformIntTupleGenerator generator; + + public UniformIntTupleGeneratorInputFormat(int numKeys, int numVals) { + keyTotal = numKeys; + valueTotal = numVals; + } + + @Override + public void open(GenericInputSplit split) throws IOException { + super.open(split); + this.generator = new UniformIntTupleGenerator(keyTotal, valueTotal, false); + + } + + @Override + public boolean reachedEnd() throws IOException { + return valueCount >= valueTotal; + } + + @Override + public Tuple2 nextRecord(Tuple2 reuse) throws IOException { + valueCount += 1; + return generator.next(); + } +}