From ebba20dfa87fda3719eccf5006fffa4c030caf2b Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Oct 2015 21:22:45 +0100 Subject: [PATCH] [FLINK-2901] Remove Record API dependencies from flink-tests #2 This closes #1306 --- .../AccumulatorIterativeITCase.java | 77 +++ ...ngTestBase.java => CancelingTestBase.java} | 4 +- ...ngITCase.java => JoinCancelingITCase.java} | 5 +- .../test/cancelling/MapCancelingITCase.java | 2 +- .../IterationTerminationWithTwoTails.java | 74 +++ .../KMeansWithBroadcastSetITCase.java | 98 +++ .../recordJobs/graph/ComputeEdgeDegrees.java | 231 ------- .../graph/ConnectedComponentsWithCoGroup.java | 134 ---- .../graph/DeltaPageRankWithInitialDeltas.java | 187 ------ .../EnumTrianglesOnEdgesWithDegrees.java | 186 ------ .../graph/EnumTrianglesRdfFoaf.java | 293 --------- .../graph/EnumTrianglesWithDegrees.java | 104 --- .../test/recordJobs/graph/PairwiseSP.java | 435 ------------- .../graph/WorksetConnectedComponents.java | 198 ------ .../triangleEnumUtil/EdgeInputFormat.java | 77 --- .../EdgeWithDegreesInputFormat.java | 118 ---- .../EdgeWithDegreesOutputFormat.java | 61 -- .../TriangleOutputFormat.java | 57 -- .../recordJobs/kmeans/KMeansBroadcast.java | 324 --------- .../test/recordJobs/kmeans/KMeansCross.java | 136 ---- .../recordJobs/kmeans/KMeansSingleStep.java | 300 --------- .../kmeans/udfs/ComputeDistance.java | 65 -- .../udfs/ComputeDistanceParameterized.java | 77 --- .../recordJobs/kmeans/udfs/CoordVector.java | 158 ----- .../kmeans/udfs/FindNearestCenter.java | 114 ---- .../recordJobs/kmeans/udfs/PointInFormat.java | 107 --- .../kmeans/udfs/PointOutFormat.java | 78 --- .../kmeans/udfs/RecomputeClusterCenter.java | 158 ----- .../recordJobs/relational/MergeOnlyJoin.java | 122 ---- .../recordJobs/relational/TPCHQuery1.java | 87 --- .../recordJobs/relational/TPCHQuery10.java | 365 ----------- .../recordJobs/relational/TPCHQuery3.java | 277 -------- .../relational/TPCHQuery3Unioned.java | 175 ----- .../recordJobs/relational/TPCHQuery4.java | 284 -------- .../recordJobs/relational/TPCHQuery9.java | 251 ------- .../relational/TPCHQueryAsterix.java | 205 ------ .../recordJobs/relational/WebLogAnalysis.java | 341 ---------- .../query1Util/GroupByReturnFlag.java | 68 -- .../relational/query1Util/LineItemFilter.java | 89 --- .../query1Util/LineItemFilterTest.java | 191 ------ .../query9Util/AmountAggregate.java | 71 -- .../query9Util/FilteredPartsJoin.java | 62 -- .../relational/query9Util/IntPair.java | 38 -- .../relational/query9Util/LineItemMap.java | 53 -- .../relational/query9Util/OrderMap.java | 49 -- .../query9Util/OrderedPartsJoin.java | 58 -- .../relational/query9Util/PartFilter.java | 52 -- .../relational/query9Util/PartJoin.java | 58 -- .../relational/query9Util/PartListJoin.java | 55 -- .../relational/query9Util/PartsuppMap.java | 47 -- .../relational/query9Util/StringIntPair.java | 39 -- .../StringIntPairStringDataOutFormat.java | 54 -- .../relational/query9Util/SupplierMap.java | 55 -- .../relational/query9Util/SuppliersJoin.java | 57 -- .../test/recordJobs/sort/ReduceGroupSort.java | 114 ---- .../flink/test/recordJobs/sort/TeraSort.java | 74 --- .../sort/tsUtil/TeraDistribution.java | 61 -- .../sort/tsUtil/TeraInputFormat.java | 49 -- .../test/recordJobs/sort/tsUtil/TeraKey.java | 157 ----- .../sort/tsUtil/TeraOutputFormat.java | 52 -- .../recordJobs/sort/tsUtil/TeraValue.java | 110 ---- .../util/DiscardingOutputFormat.java | 46 -- .../util/InfiniteIntegerInputFormat.java | 48 -- .../InfiniteIntegerInputFormatWithDelay.java | 54 -- .../recordJobs/util/IntTupleDataInFormat.java | 60 -- .../util/StringTupleDataOutFormat.java | 47 -- .../flink/test/recordJobs/util/Tuple.java | 615 ------------------ .../test/recordJobs/util/UniformIntInput.java | 93 --- .../test/recordJobs/wordcount/WordCount.java | 166 ----- .../wordcount/WordCountAccumulators.java | 225 ------- 70 files changed, 254 insertions(+), 8778 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java rename flink-tests/src/test/java/org/apache/flink/test/cancelling/{CancellingTestBase.java => CancelingTestBase.java} (98%) rename flink-tests/src/test/java/org/apache/flink/test/cancelling/{MatchJoinCancelingITCase.java => JoinCancelingITCase.java} (98%) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ComputeEdgeDegrees.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesRdfFoaf.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/EnumTrianglesWithDegrees.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/WorksetConnectedComponents.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansSingleStep.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistance.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/CoordVector.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/FindNearestCenter.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointOutFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery3Unioned.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/GroupByReturnFlag.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilter.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query1Util/LineItemFilterTest.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/AmountAggregate.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/FilteredPartsJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/IntPair.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/LineItemMap.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderMap.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/OrderedPartsJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartFilter.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartListJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/PartsuppMap.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPair.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SupplierMap.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/query9Util/SuppliersJoin.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/ReduceGroupSort.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraDistribution.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraInputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraKey.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraOutputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/tsUtil/TeraValue.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/DiscardingOutputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/IntTupleDataInFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/StringTupleDataOutFormat.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/Tuple.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/UniformIntInput.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCountAccumulators.java 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 new file mode 100644 index 00000000000..6dc0a0bb3b1 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java @@ -0,0 +1,77 @@ +/* + * 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.accumulators; + +import org.apache.flink.api.common.accumulators.IntCounter; +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.JavaProgramTestBase; +import org.apache.flink.util.Collector; +import org.junit.Assert; + +public class AccumulatorIterativeITCase extends JavaProgramTestBase { + private static final int NUM_ITERATIONS = 3; + private static final int NUM_SUBTASKS = 1; + private static final String ACC_NAME = "test"; + + @Override + protected boolean skipCollectionExecution() { + return true; + } + + @Override + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(NUM_SUBTASKS); + + IterativeDataSet iteration = env.fromElements(1, 2, 3).iterate(NUM_ITERATIONS); + + 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 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(Iterable values, Collector out) { + // Compute the sum + int sum = 0; + + for (Integer value : values) { + sum += value; + testCounter.add(value); + } + out.collect(sum); + } + } +} 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/CancelingTestBase.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java rename to flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index 14c396da18e..f6361cafce3 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/CancelingTestBase.java @@ -49,9 +49,9 @@ import scala.concurrent.duration.FiniteDuration; /** * */ -public abstract class CancellingTestBase extends TestLogger { +public abstract class CancelingTestBase extends TestLogger { - private static final Logger LOG = LoggerFactory.getLogger(CancellingTestBase.class); + private static final Logger LOG = LoggerFactory.getLogger(CancelingTestBase.class); private static final int MINIMUM_HEAP_SIZE_MB = 192; 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/JoinCancelingITCase.java similarity index 98% rename from flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java index b6f82d49b2e..d797e47a6dc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java @@ -30,12 +30,11 @@ 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 MatchJoinCancelingITCase extends CancellingTestBase { +public class JoinCancelingITCase extends CancelingTestBase { private static final int parallelism = 4; - public MatchJoinCancelingITCase() { + public JoinCancelingITCase() { setTaskManagerNumSlots(parallelism); } 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 55b19111e68..03b1a24e9fa 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 @@ -26,7 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.InfiniteIntegerInputFormat; import org.junit.Test; -public class MapCancelingITCase extends CancellingTestBase { +public class MapCancelingITCase extends CancelingTestBase { private static final int parallelism = 4; public MapCancelingITCase() { 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 new file mode 100644 index 00000000000..5a2df3f5b16 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java @@ -0,0 +1,74 @@ +/* + * 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.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; + +public class IterationTerminationWithTwoTails extends JavaProgramTestBase { + private static final String EXPECTED = "22\n"; + + @Override + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); + + DataSet initialInput = env.fromElements("1", "2", "3", "4", "5").name("input"); + + IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); + + DataSet sumReduce = iteration.reduceGroup(new SumReducer()).name("Compute sum (GroupReduce"); + + DataSet terminationFilter = iteration.filter(new TerminationFilter()).name("Compute termination criterion (Map)"); + + List result = iteration.closeWith(sumReduce, terminationFilter).collect(); + + containsResultAsText(result, EXPECTED); + } + + public static final class SumReducer implements GroupReduceFunction { + private static final long serialVersionUID = 1L; + + @Override + public void reduce(Iterable values, Collector out) throws Exception { + int sum = 0; + for (String value : values) { + sum += Integer.parseInt(value) + 1; + } + out.collect("" + sum); + } + } + + public static class TerminationFilter implements FilterFunction { + private static final long serialVersionUID = 1L; + + @Override + 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/KMeansWithBroadcastSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java new file mode 100644 index 00000000000..8edc2aacb24 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java @@ -0,0 +1,98 @@ +/* + * 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.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.examples.java.clustering.KMeans; +import org.apache.flink.examples.java.clustering.KMeans.Point; +import org.apache.flink.examples.java.clustering.KMeans.Centroid; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.test.testdata.KMeansData; + +import java.util.List; + +public class KMeansWithBroadcastSetITCase extends JavaProgramTestBase { + + @SuppressWarnings("serial") + @Override + protected void testProgram() throws Exception { + + String[] points = KMeansData.DATAPOINTS_2D.split("\n"); + String[] centers = KMeansData.INITIAL_CENTERS_2D.split("\n"); + + // set up execution environment + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // get input data + DataSet pointsSet = env.fromElements(points) + .map(new MapFunction() { + public Point map(String p) { + String[] fields = p.split("\\|"); + return new Point( + Double.parseDouble(fields[1]), + Double.parseDouble(fields[2])); + } + }); + + DataSet centroidsSet = env.fromElements(centers) + .map(new MapFunction() { + public Centroid map(String c) { + String[] fields = c.split("\\|"); + return new Centroid( + Integer.parseInt(fields[0]), + Double.parseDouble(fields[1]), + Double.parseDouble(fields[2])); + } + }); + + // set number of bulk iterations for KMeans algorithm + IterativeDataSet loop = centroidsSet.iterate(20); + + DataSet newCentroids = pointsSet + // compute closest centroid for each point + .map(new KMeans.SelectNearestCenter()).withBroadcastSet(loop, "centroids") + // count and sum point coordinates for each centroid + .map(new KMeans.CountAppender()) + .groupBy(0).reduce(new KMeans.CentroidAccumulator()) + // compute new centroids from point counts and coordinate sums + .map(new KMeans.CentroidAverager()); + + // feed new centroids back into next iteration + DataSet finalCentroids = loop.closeWith(newCentroids); + + DataSet stringCentroids = finalCentroids + .map(new MapFunction() { + @Override + public String map(Centroid c) throws Exception { + return String.format("%d|%.2f|%.2f|", c.id, c.x, c.y); + } + }); + + List result = stringCentroids.collect(); + + KMeansData.checkResultsWithDelta( + KMeansData.CENTERS_2D_AFTER_20_ITERATIONS_DOUBLE_DIGIT, + result, + 0.01); + } + +} 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 9ec8c7366c3..00000000000 --- 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 cc6fd01ce8e..00000000000 --- 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 81adbf56558..00000000000 --- 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 b2328bae953..00000000000 --- 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 ba8b54a8fb2..00000000000 --- 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 dc521584236..00000000000 --- 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 34d4b60775e..00000000000 --- 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 0dbb20a770a..00000000000 --- 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 81f4d006237..00000000000 --- 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 8441602f81d..00000000000 --- 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 a72a3554714..00000000000 --- 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 f6c27b09837..00000000000 --- 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 d528f5d8a4a..00000000000 --- 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 8d75d4766f0..00000000000 --- 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 bdf74663976..00000000000 --- 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 ee33113d046..00000000000 --- 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 78b60efcba8..00000000000 --- 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/CoordVector.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/CoordVector.java deleted file mode 100644 index 67e87a35855..00000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/CoordVector.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.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Key; - -/** - * Implements a feature vector as a multi-dimensional point. Coordinates of that point - * (= the features) are stored as double values. The distance between two feature vectors is - * the Euclidian distance between the points. - */ -public final class CoordVector implements Key { - private static final long serialVersionUID = 1L; - - // coordinate array - private double[] coordinates; - - /** - * Initializes a blank coordinate vector. Required for deserialization! - */ - public CoordVector() { - coordinates = null; - } - - /** - * Initializes a coordinate vector. - * - * @param coordinates The coordinate vector of a multi-dimensional point. - */ - public CoordVector(Double[] coordinates) { - this.coordinates = new double[coordinates.length]; - for (int i = 0; i < coordinates.length; i++) { - this.coordinates[i] = coordinates[i]; - } - } - - /** - * Initializes a coordinate vector. - * - * @param coordinates The coordinate vector of a multi-dimensional point. - */ - public CoordVector(double[] coordinates) { - this.coordinates = coordinates; - } - - /** - * Returns the coordinate vector of a multi-dimensional point. - * - * @return The coordinate vector of a multi-dimensional point. - */ - public double[] getCoordinates() { - return this.coordinates; - } - - /** - * Sets the coordinate vector of a multi-dimensional point. - * - * @param coordinates The dimension values of the point. - */ - public void setCoordinates(double[] coordinates) { - this.coordinates = coordinates; - } - - /** - * Computes the Euclidian distance between this coordinate vector and a - * second coordinate vector. - * - * @param cv The coordinate vector to which the distance is computed. - * @return The Euclidian distance to coordinate vector cv. If cv has a - * different length than this coordinate vector, -1 is returned. - */ - public double computeEuclidianDistance(CoordVector cv) { - // check coordinate vector lengths - if (cv.coordinates.length != this.coordinates.length) { - return -1.0; - } - - double quadSum = 0.0; - for (int i = 0; i < this.coordinates.length; i++) { - double diff = this.coordinates[i] - cv.coordinates[i]; - quadSum += diff*diff; - } - return Math.sqrt(quadSum); - } - - - @Override - public void read(DataInputView in) throws IOException { - int length = in.readInt(); - this.coordinates = new double[length]; - for (int i = 0; i < length; i++) { - this.coordinates[i] = in.readDouble(); - } - } - - - @Override - public void write(DataOutputView out) throws IOException { - out.writeInt(this.coordinates.length); - for (int i = 0; i < this.coordinates.length; i++) { - out.writeDouble(this.coordinates[i]); - } - } - - /** - * Compares this coordinate vector to another key. - * - * @return -1 if the other key is not of type CoordVector. If the other - * key is also a CoordVector but its length differs from this - * coordinates vector, -1 is return if this coordinate vector is - * smaller and 1 if it is larger. If both coordinate vectors - * have the same length, the coordinates of both are compared. - * If a coordinate of this coordinate vector is smaller than the - * corresponding coordinate of the other vector -1 is returned - * and 1 otherwise. If all coordinates are identical 0 is - * returned. - */ - @Override - public int compareTo(CoordVector o) { - // check if both coordinate vectors have identical lengths - if (o.coordinates.length > this.coordinates.length) { - return -1; - } - else if (o.coordinates.length < this.coordinates.length) { - return 1; - } - - // compare all coordinates - for (int i = 0; i < this.coordinates.length; i++) { - if (o.coordinates[i] > this.coordinates[i]) { - return -1; - } else if (o.coordinates[i] < this.coordinates[i]) { - return 1; - } - } - return 0; - } -} 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 1e893ce80f9..00000000000 --- 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/PointInFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java deleted file mode 100644 index c5dd8ec5a39..00000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/udfs/PointInFormat.java +++ /dev/null @@ -1,107 +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.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; - -/** - * 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 { - 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) { - - 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] == '|') { - // check if id was already set - if (id == -1) { - id = value; - } - else { - double v = value + ((double) fractionValue) * Math.pow(10, (-1 * (fractionChars - 1))); - this.dimensionValues.add(negative ? -v : v); - } - // reset value - value = 0; - fractionValue = 0; - fractionChars = 0; - negative = false; - } else if (line[pos] == '.') { - fractionChars = 1; - } else if (line[pos] == '-') { - negative = true; - } else { - if (fractionChars == 0) { - value *= 10; - value += line[pos] - '0'; - } else { - fractionValue *= 10; - fractionValue += line[pos] - '0'; - fractionChars++; - } - } - } - - // set the ID - this.idInteger.setValue(id); - record.setField(0, this.idInteger); - - // set the data points - if (this.pointValues.length != this.dimensionValues.size()) { - this.pointValues = new double[this.dimensionValues.size()]; - } - for (int i = 0; i < this.pointValues.length; i++) { - this.pointValues[i] = this.dimensionValues.get(i); - } - - this.point.setCoordinates(this.pointValues); - record.setField(1, this.point); - return record; - } -} 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 410397ef53b..00000000000 --- 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 89e222b64c9..00000000000 --- 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 b94880487f2..00000000000 --- 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 d805b92b4cf..00000000000 --- 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 4bb0cdf117b..00000000000 --- 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 cebe6f94e15..00000000000 --- 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 157e3cf95e8..00000000000 --- 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 ec3c5b4f2d7..00000000000 --- 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 c00d231740c..00000000000 --- 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 a681f64c71c..00000000000 --- 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 377c7eec79f..00000000000 --- 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 d7aab4baf04..00000000000 --- 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 be5452427b6..00000000000 --- 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 b03878bf8ab..00000000000 --- 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 638877f3266..00000000000 --- 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 a391684322c..00000000000 --- 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 64b21bdd220..00000000000 --- 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 6a862b80eeb..00000000000 --- 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 ee103338169..00000000000 --- 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 2e6453c411a..00000000000 --- 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 1e9a9208b99..00000000000 --- 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 67f2f6a4812..00000000000 --- 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 66860cec177..00000000000 --- 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 32c1d6433f8..00000000000 --- 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 37f47b18b4a..00000000000 --- 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 42ddf6e8ada..00000000000 --- 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 2b435315fcf..00000000000 --- 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 f1252373cc4..00000000000 --- 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 b4dc1b494f6..00000000000 --- 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 4563c074bb3..00000000000 --- 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 6d670c84d93..00000000000 --- 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 7af5e1045cf..00000000000 --- 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 511e094feb1..00000000000 --- 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 da64cd2dfa2..00000000000 --- 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 1658c5d94bc..00000000000 --- 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 545e1836532..00000000000 --- 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/InfiniteIntegerInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormat.java deleted file mode 100644 index 61a47c781f1..00000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormat.java +++ /dev/null @@ -1,48 +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.api.java.record.io.GenericInputFormat; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; - -/** - * - */ -public class InfiniteIntegerInputFormat extends GenericInputFormat { - private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - - - @Override - public boolean reachedEnd() throws IOException { - return false; - } - - - @Override - public Record nextRecord(Record record) throws IOException{ - record.setField(0, this.one); - return record; - } -} 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/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java deleted file mode 100644 index f1de48d9013..00000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.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.util; - -import java.io.IOException; - -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 { - private static final long serialVersionUID = 1L; - - private static final int DELAY = 20; - - private final IntValue one = new IntValue(1); - - - @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 - } - - return record; - } -} 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 e5d32c613bd..00000000000 --- 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 e6b85481458..00000000000 --- 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 064f15e4422..00000000000 --- 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 5e880cb1226..00000000000 --- 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 2fcc523424b..00000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java +++ /dev/null @@ -1,166 +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. - * - *

- * - * Note: This example uses the out-dated Record API. - * It is recommended to use the new Java API. - * - */ -@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 780db582b38..00000000000 --- 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; - } - } -} -- GitLab