From 1f378eda803783c053adc7b707bd8f938b0616a4 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 12 Aug 2014 22:48:59 +0200 Subject: [PATCH 001/112] Initial commit --- .gitignore | 12 ++++ LICENSE | 201 +++++++++++++++++++++++++++++++++++++++++++++++++++++ README.md | 2 + 3 files changed, 215 insertions(+) create mode 100644 .gitignore create mode 100644 LICENSE create mode 100644 README.md diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000000..32858aad3c383 --- /dev/null +++ b/.gitignore @@ -0,0 +1,12 @@ +*.class + +# Mobile Tools for Java (J2ME) +.mtj.tmp/ + +# Package Files # +*.jar +*.war +*.ear + +# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml +hs_err_pid* diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000000000..5c304d1a4a7b4 --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed 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. diff --git a/README.md b/README.md new file mode 100644 index 0000000000000..dcd39e5573ee9 --- /dev/null +++ b/README.md @@ -0,0 +1,2 @@ +flink-graphs +============ From 22b7464cf73497f09a55b26be4d654cc339cf1ec Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 12 Aug 2014 22:54:54 +0200 Subject: [PATCH 002/112] Initial Maven project structure and .gitignore --- .gitignore | 23 ++++++++++------- README.md | 2 ++ pom.xml | 76 ++++++++++++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 91 insertions(+), 10 deletions(-) create mode 100644 pom.xml diff --git a/.gitignore b/.gitignore index 32858aad3c383..0d797e53a8d31 100644 --- a/.gitignore +++ b/.gitignore @@ -1,12 +1,15 @@ +.cache +.classpath +.idea +.metadata +.settings +.project +.version.properties +filter.properties +target +tmp *.class - -# Mobile Tools for Java (J2ME) -.mtj.tmp/ - -# Package Files # +*.iml +*.swp *.jar -*.war -*.ear - -# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml -hs_err_pid* +.DS_Store diff --git a/README.md b/README.md index dcd39e5573ee9..367e27235a3ef 100644 --- a/README.md +++ b/README.md @@ -1,2 +1,4 @@ flink-graphs ============ + +Playground for a Graph API diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000000..389d60ce43c2b --- /dev/null +++ b/pom.xml @@ -0,0 +1,76 @@ + + + 4.0.0 + + eu.stratosphere + flink-graphs + 1.0-SNAPSHOT + jar + + + UTF-8 + + + + + apache.snapshots + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + + + + + + + org.apache.flink + flink-java + 0.6-incubating-SNAPSHOT + + + org.apache.flink + flink-clients + 0.6-incubating-SNAPSHOT + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.6 + 1.6 + + + + + From 6103cf9dea81c957a5727e351fecc4e32df017ba Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 27 Aug 2014 17:56:32 +0200 Subject: [PATCH 003/112] Add example for parallel dense ID assignment --- pom.xml | 8 +- .../flinkgraphs/misc/DenseIdAssignment.java | 156 ++++++++++++++++++ 2 files changed, 160 insertions(+), 4 deletions(-) create mode 100644 src/main/java/flinkgraphs/misc/DenseIdAssignment.java diff --git a/pom.xml b/pom.xml index 389d60ce43c2b..f57ab9b9e7950 100644 --- a/pom.xml +++ b/pom.xml @@ -49,12 +49,12 @@ under the License. org.apache.flink flink-java - 0.6-incubating-SNAPSHOT + 0.7-incubating-SNAPSHOT org.apache.flink flink-clients - 0.6-incubating-SNAPSHOT + 0.7-incubating-SNAPSHOT @@ -67,8 +67,8 @@ under the License. maven-compiler-plugin 3.1 - 1.6 - 1.6 + 1.7 + 1.7 diff --git a/src/main/java/flinkgraphs/misc/DenseIdAssignment.java b/src/main/java/flinkgraphs/misc/DenseIdAssignment.java new file mode 100644 index 0000000000000..47464f62d1311 --- /dev/null +++ b/src/main/java/flinkgraphs/misc/DenseIdAssignment.java @@ -0,0 +1,156 @@ +/** + * 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 flinkgraphs.misc; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Random; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.LocalEnvironment; +import org.apache.flink.api.java.functions.RichMapFunction; +import org.apache.flink.api.java.functions.RichMapPartitionFunction; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; + +@SuppressWarnings("serial") +public class DenseIdAssignment implements java.io.Serializable { + + + private static final String TEST_DATA_PATH = "/data/demodata/strings/strings.txt"; + + + public static void main(String[] args) throws Exception { + + LocalEnvironment env = ExecutionEnvironment.createLocalEnvironment(4); + env.enableLogging(); + + DataSet> strings = env.readCsvFile(TEST_DATA_PATH).types(String.class); + + // make strings unique (optional) + DataSet uniqueStrings = strings +// .groupBy(0) +// .reduce(new ReduceFunction>() { +// public Tuple1 reduce(Tuple1 a, Tuple1 b) { +// return a; +// } +// }) + .map(new MapFunction, String>() { + @Override + public String map(Tuple1 value) { + return value.f0; + } + }); + + // count the elements in each partition + DataSet> partitionCounts = uniqueStrings.mapPartition(new PerPartitionCounter()); + + DataSet> result = uniqueStrings.map(new IdAssigner()).withBroadcastSet(partitionCounts, "COUNTERS"); + + result.print(); + +// env.execute(); + + System.out.println(env.getExecutionPlan()); + } + + /** + * Counts the number of elements per partition and returns a (partition-num, count) tuple. + */ + public static final class PerPartitionCounter extends RichMapPartitionFunction> { + + @Override + public void mapPartition(Iterable values, Collector> out) { + long count = 0; + for (String str : values) { + count++; + } + + int partition = getRuntimeContext().getIndexOfThisSubtask(); + out.collect(new Tuple2(partition, count)); + } + } + + public static final class IdAssigner extends RichMapFunction> { + + private long id; + + @Override + public void open(Configuration parameters) { + @SuppressWarnings("unchecked") + List> counters = (List>) (List) getRuntimeContext().getBroadcastVariable("COUNTERS"); + + Collections.sort(counters, new Comparator>() { + public int compare(Tuple2 o1, Tuple2 o2) { + return o1.f0.compareTo(o2.f0); + } + }); + + int ourPartition = getRuntimeContext().getIndexOfThisSubtask(); + + // sum up all counts from partitions before us + for (int i = 0; i < ourPartition; i++) { + id += counters.get(i).f1; + } + } + + @Override + public Tuple2 map(String value) { + return new Tuple2(value, id++); + } + } + + + // -------------------------------------------------------------------------------------------- + // Some test data + // -------------------------------------------------------------------------------------------- + +// public static void main(String[] args) throws Exception { +// writeRandomStrings(TEST_DATA_PATH, 100000000); +// } + + private static void writeRandomStrings(String path, int num) throws IOException { + File f = new File(path); + Random rnd = new Random(); + + try (FileWriter writer = new FileWriter(f)) { + for (int i = 0; i < num; i++) { + writer.write(getRandomLine(rnd)); + } + } + } + + private static String getRandomLine(Random rnd) { + StringBuilder bld = new StringBuilder(21); + for (int i = 0; i < 20; i++) { + bld.append((char) (rnd.nextInt('Z' - 'A') + 1 + 'A')); + } + bld.append('\n'); + return bld.toString(); + } +} From f44986d22b5913cba8e11b6e43a77effd925aaa1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sat, 30 Aug 2014 19:19:21 +0200 Subject: [PATCH 004/112] First mockup of initial API functions --- src/main/java/flinkgraph/api/Edge.java | 40 ++++++++++ .../java/flinkgraph/api/EdgeWithValue.java | 43 ++++++++++ src/main/java/flinkgraph/api/Graph.java | 80 +++++++++++++++++++ .../java/flinkgraph/api/io/EdgeBuilder.java | 29 +++++++ .../flinkgraph/api/io/EdgesCsvBuilder.java | 61 ++++++++++++++ .../api/io/MemoryMatrixEdgesBuilder.java | 40 ++++++++++ .../java/flinkgraph/examples/Example.java | 39 +++++++++ .../internal/AbstractGraphRepresentation.java | 34 ++++++++ .../EdgeAndVertexGraphRepresentation.java | 42 ++++++++++ .../internal/EdgeOnlyGraphRepresentation.java | 67 ++++++++++++++++ .../internal/utils/Tuple1Unpackfunction.java | 32 ++++++++ .../internal/utils/Tuple2ToEdgeMapper.java | 34 ++++++++ .../flinkgraphs/misc/DenseIdAssignment.java | 9 +-- 13 files changed, 544 insertions(+), 6 deletions(-) create mode 100644 src/main/java/flinkgraph/api/Edge.java create mode 100644 src/main/java/flinkgraph/api/EdgeWithValue.java create mode 100644 src/main/java/flinkgraph/api/Graph.java create mode 100644 src/main/java/flinkgraph/api/io/EdgeBuilder.java create mode 100644 src/main/java/flinkgraph/api/io/EdgesCsvBuilder.java create mode 100644 src/main/java/flinkgraph/api/io/MemoryMatrixEdgesBuilder.java create mode 100644 src/main/java/flinkgraph/examples/Example.java create mode 100644 src/main/java/flinkgraph/internal/AbstractGraphRepresentation.java create mode 100644 src/main/java/flinkgraph/internal/EdgeAndVertexGraphRepresentation.java create mode 100644 src/main/java/flinkgraph/internal/EdgeOnlyGraphRepresentation.java create mode 100644 src/main/java/flinkgraph/internal/utils/Tuple1Unpackfunction.java create mode 100644 src/main/java/flinkgraph/internal/utils/Tuple2ToEdgeMapper.java diff --git a/src/main/java/flinkgraph/api/Edge.java b/src/main/java/flinkgraph/api/Edge.java new file mode 100644 index 0000000000000..fe0a2a8d6ff33 --- /dev/null +++ b/src/main/java/flinkgraph/api/Edge.java @@ -0,0 +1,40 @@ +/** + * 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 flinkgraph.api; + +import org.apache.flink.api.java.tuple.Tuple2; + +public class Edge extends Tuple2 { + + private static final long serialVersionUID = 1L; + + public Edge() {} + + public Edge(T value0, T value1) { + super(value0, value1); + } + + public T getSource() { + return f0; + } + + public T getTarget() { + return f1; + } +} diff --git a/src/main/java/flinkgraph/api/EdgeWithValue.java b/src/main/java/flinkgraph/api/EdgeWithValue.java new file mode 100644 index 0000000000000..3040f3fbab78a --- /dev/null +++ b/src/main/java/flinkgraph/api/EdgeWithValue.java @@ -0,0 +1,43 @@ +/** + * 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 flinkgraph.api; + +public class EdgeWithValue extends Edge { + + private static final long serialVersionUID = 1L; + + private V value; + + + public EdgeWithValue() {} + + public EdgeWithValue(T source, T target, V value) { + super(source, target); + this.value = value; + } + + + public V getValue() { + return value; + } + + public void setValue(V value) { + this.value = value; + } +} diff --git a/src/main/java/flinkgraph/api/Graph.java b/src/main/java/flinkgraph/api/Graph.java new file mode 100644 index 0000000000000..97a47e174ebe2 --- /dev/null +++ b/src/main/java/flinkgraph/api/Graph.java @@ -0,0 +1,80 @@ +/** + * 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 flinkgraph.api; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; + +import flinkgraph.api.io.EdgeBuilder; +import flinkgraph.internal.AbstractGraphRepresentation; +import flinkgraph.internal.EdgeOnlyGraphRepresentation; + + +public class Graph> { + + + private AbstractGraphRepresentation graphRepr; + + private long vertexCount = -1; + + private long edgeCount = -1; + + // -------------------------------------------------------------------------------------------- + + public Graph(AbstractGraphRepresentation representation) { + this.graphRepr = representation; + } + + // -------------------------------------------------------------------------------------------- + + + public long getNumberOfVertices() { + if (vertexCount == -1) { + vertexCount = count(graphRepr.getVertexIdSet()); + } + return vertexCount; + } + + public long getNumberOfEdges() { + if (edgeCount == -1) { + edgeCount = count(graphRepr.getEdges()); + } + return edgeCount; + } + + + // -------------------------------------------------------------------------------------------- + // Utility Functions + // -------------------------------------------------------------------------------------------- + + public static int count(DataSet data) { + return -1; // to be implemented + } + + + // -------------------------------------------------------------------------------------------- + // Static Builders + // -------------------------------------------------------------------------------------------- + + public static > Graph fromEdges(ExecutionEnvironment e, Class idType, EdgeBuilder edgeBuilder) { + DataSet> edges = edgeBuilder.createEdges(e, idType); + return new Graph(new EdgeOnlyGraphRepresentation(edges)); + } + +} diff --git a/src/main/java/flinkgraph/api/io/EdgeBuilder.java b/src/main/java/flinkgraph/api/io/EdgeBuilder.java new file mode 100644 index 0000000000000..1635ee70b3ddf --- /dev/null +++ b/src/main/java/flinkgraph/api/io/EdgeBuilder.java @@ -0,0 +1,29 @@ +/** + * 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 flinkgraph.api.io; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; + +import flinkgraph.api.Edge; + +public interface EdgeBuilder { + + public DataSet> createEdges(ExecutionEnvironment env, Class idType); +} diff --git a/src/main/java/flinkgraph/api/io/EdgesCsvBuilder.java b/src/main/java/flinkgraph/api/io/EdgesCsvBuilder.java new file mode 100644 index 0000000000000..b28bd2d78c1a9 --- /dev/null +++ b/src/main/java/flinkgraph/api/io/EdgesCsvBuilder.java @@ -0,0 +1,61 @@ +/** + * 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 flinkgraph.api.io; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; + +import flinkgraph.api.Edge; +import flinkgraph.internal.utils.Tuple2ToEdgeMapper; + +public class EdgesCsvBuilder> implements EdgeBuilder { + + private final String path; + + private int sourcePos = 0; + + private int targetPos = 1; + + + public EdgesCsvBuilder(String path) { + this.path = path; + } + + + public EdgesCsvBuilder sourceColumn(int column) { + this.sourcePos = column; + return this; + } + + public EdgesCsvBuilder targetColumn(int column) { + this.targetPos = column; + return this; + } + + @Override + public DataSet> createEdges(ExecutionEnvironment env, Class idType) { + long mask = 0; + mask |= 0x1 << sourcePos; + mask |= 0x1 << targetPos; + + DataSet> tuples = env.readCsvFile(path).includeFields(mask).types(idType, idType); + return tuples.map(new Tuple2ToEdgeMapper()); + } +} diff --git a/src/main/java/flinkgraph/api/io/MemoryMatrixEdgesBuilder.java b/src/main/java/flinkgraph/api/io/MemoryMatrixEdgesBuilder.java new file mode 100644 index 0000000000000..52372887e97e4 --- /dev/null +++ b/src/main/java/flinkgraph/api/io/MemoryMatrixEdgesBuilder.java @@ -0,0 +1,40 @@ +/** + * 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 flinkgraph.api.io; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; + +import flinkgraph.api.Edge; + +public class MemoryMatrixEdgesBuilder implements EdgeBuilder { + + private final T[][] matrix; + + + public MemoryMatrixEdgesBuilder(T[][] matrix) { + this.matrix = matrix; + } + + + @Override + public DataSet> createEdges(ExecutionEnvironment env, Class idType) { + throw new UnsupportedOperationException("to be implemented"); + } +} diff --git a/src/main/java/flinkgraph/examples/Example.java b/src/main/java/flinkgraph/examples/Example.java new file mode 100644 index 0000000000000..2db5906de87bd --- /dev/null +++ b/src/main/java/flinkgraph/examples/Example.java @@ -0,0 +1,39 @@ +/** + * 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 flinkgraph.examples; + +import org.apache.flink.api.java.ExecutionEnvironment; + +import flinkgraph.api.Graph; +import flinkgraph.api.io.EdgesCsvBuilder; + +public class Example { + + + public static void main(String[] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + + Graph graph = Graph.fromEdges(env, String.class, + new EdgesCsvBuilder("/the/edges/file") + .sourceColumn(2).targetColumn(5)); + + } +} diff --git a/src/main/java/flinkgraph/internal/AbstractGraphRepresentation.java b/src/main/java/flinkgraph/internal/AbstractGraphRepresentation.java new file mode 100644 index 0000000000000..bb4e006cabfc0 --- /dev/null +++ b/src/main/java/flinkgraph/internal/AbstractGraphRepresentation.java @@ -0,0 +1,34 @@ +/** + * 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 flinkgraph.internal; + +import org.apache.flink.api.java.DataSet; + +import flinkgraph.api.Edge; + +/** + * @param The type of the IDs + */ +public abstract class AbstractGraphRepresentation { + + + public abstract DataSet getVertexIdSet(); + + public abstract DataSet> getEdges(); +} diff --git a/src/main/java/flinkgraph/internal/EdgeAndVertexGraphRepresentation.java b/src/main/java/flinkgraph/internal/EdgeAndVertexGraphRepresentation.java new file mode 100644 index 0000000000000..87687595b69d6 --- /dev/null +++ b/src/main/java/flinkgraph/internal/EdgeAndVertexGraphRepresentation.java @@ -0,0 +1,42 @@ +/** + * 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 flinkgraph.internal; + +import org.apache.flink.api.java.DataSet; + +import flinkgraph.api.Edge; + +public class EdgeAndVertexGraphRepresentation extends AbstractGraphRepresentation { + + private DataSet vertices; + + private DataSet> edges; + + + + @Override + public DataSet> getEdges() { + return this.edges; + } + + @Override + public DataSet getVertexIdSet() { + return vertices; + } +} diff --git a/src/main/java/flinkgraph/internal/EdgeOnlyGraphRepresentation.java b/src/main/java/flinkgraph/internal/EdgeOnlyGraphRepresentation.java new file mode 100644 index 0000000000000..ca9953892f4e8 --- /dev/null +++ b/src/main/java/flinkgraph/internal/EdgeOnlyGraphRepresentation.java @@ -0,0 +1,67 @@ +/** + * 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 flinkgraph.internal; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.util.Collector; + +import flinkgraph.api.Edge; +import flinkgraph.internal.utils.Tuple1Unpackfunction; + +@SuppressWarnings("serial") +public class EdgeOnlyGraphRepresentation extends AbstractGraphRepresentation { + + + private final DataSet> edges; + + + public EdgeOnlyGraphRepresentation(DataSet> edges) { + this.edges = edges; + } + + + + @Override + public DataSet> getEdges() { + return edges; + } + + @Override + public DataSet getVertexIdSet() { + DataSet> wrapped = edges.flatMap(new ToIdMapper()).distinct(); + // currently need to go through the tuple wrapped variant because distinct + // is only available on tuples + return wrapped.map(new Tuple1Unpackfunction()); + } + + + // -------------------------------------------------------------------------------------------- + + public static final class ToIdMapper implements FlatMapFunction, Tuple1> { + + @Override + public void flatMap(Edge value, Collector> out) { + // collect both source and target, because we cannot be sure + out.collect(new Tuple1(value.f0)); + out.collect(new Tuple1(value.f1)); + } + } +} diff --git a/src/main/java/flinkgraph/internal/utils/Tuple1Unpackfunction.java b/src/main/java/flinkgraph/internal/utils/Tuple1Unpackfunction.java new file mode 100644 index 0000000000000..b3867e9a72f2f --- /dev/null +++ b/src/main/java/flinkgraph/internal/utils/Tuple1Unpackfunction.java @@ -0,0 +1,32 @@ +/** + * 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 flinkgraph.internal.utils; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple1; + +public class Tuple1Unpackfunction implements MapFunction, T>{ + + private static final long serialVersionUID = 1L; + + @Override + public T map(Tuple1 value) { + return value.f0; + } +} diff --git a/src/main/java/flinkgraph/internal/utils/Tuple2ToEdgeMapper.java b/src/main/java/flinkgraph/internal/utils/Tuple2ToEdgeMapper.java new file mode 100644 index 0000000000000..7ea6fddb2e506 --- /dev/null +++ b/src/main/java/flinkgraph/internal/utils/Tuple2ToEdgeMapper.java @@ -0,0 +1,34 @@ +/** + * 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 flinkgraph.internal.utils; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; + +import flinkgraph.api.Edge; + +public class Tuple2ToEdgeMapper implements MapFunction, Edge> { + + private static final long serialVersionUID = 1L; + + @Override + public Edge map(Tuple2 value) throws Exception { + return new Edge(value.f0, value.f1); + } +} diff --git a/src/main/java/flinkgraphs/misc/DenseIdAssignment.java b/src/main/java/flinkgraphs/misc/DenseIdAssignment.java index 47464f62d1311..837aff58124ed 100644 --- a/src/main/java/flinkgraphs/misc/DenseIdAssignment.java +++ b/src/main/java/flinkgraphs/misc/DenseIdAssignment.java @@ -27,7 +27,6 @@ import java.util.Random; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.LocalEnvironment; @@ -74,9 +73,7 @@ public String map(Tuple1 value) { result.print(); -// env.execute(); - - System.out.println(env.getExecutionPlan()); + env.execute(); } /** @@ -87,7 +84,7 @@ public static final class PerPartitionCounter extends RichMapPartitionFunction values, Collector> out) { long count = 0; - for (String str : values) { + for (@SuppressWarnings("unused") String str : values) { count++; } @@ -134,7 +131,7 @@ public Tuple2 map(String value) { // writeRandomStrings(TEST_DATA_PATH, 100000000); // } - private static void writeRandomStrings(String path, int num) throws IOException { + protected static void writeRandomStrings(String path, int num) throws IOException { File f = new File(path); Random rnd = new Random(); From 0276986098f18ab18f71b6f81e273fc8c2f86886 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 8 Oct 2014 12:18:04 +0200 Subject: [PATCH 005/112] Reset repository --- pom.xml | 10 ++ src/main/java/flinkgraph/api/Edge.java | 40 ----- .../java/flinkgraph/api/EdgeWithValue.java | 43 ----- src/main/java/flinkgraph/api/Graph.java | 80 --------- .../java/flinkgraph/api/io/EdgeBuilder.java | 29 ---- .../flinkgraph/api/io/EdgesCsvBuilder.java | 61 ------- .../api/io/MemoryMatrixEdgesBuilder.java | 40 ----- .../java/flinkgraph/examples/Example.java | 39 ----- .../internal/AbstractGraphRepresentation.java | 34 ---- .../EdgeAndVertexGraphRepresentation.java | 42 ----- .../internal/EdgeOnlyGraphRepresentation.java | 67 -------- .../internal/utils/Tuple1Unpackfunction.java | 32 ---- .../internal/utils/Tuple2ToEdgeMapper.java | 34 ---- .../flinkgraphs/misc/DenseIdAssignment.java | 153 ------------------ 14 files changed, 10 insertions(+), 694 deletions(-) delete mode 100644 src/main/java/flinkgraph/api/Edge.java delete mode 100644 src/main/java/flinkgraph/api/EdgeWithValue.java delete mode 100644 src/main/java/flinkgraph/api/Graph.java delete mode 100644 src/main/java/flinkgraph/api/io/EdgeBuilder.java delete mode 100644 src/main/java/flinkgraph/api/io/EdgesCsvBuilder.java delete mode 100644 src/main/java/flinkgraph/api/io/MemoryMatrixEdgesBuilder.java delete mode 100644 src/main/java/flinkgraph/examples/Example.java delete mode 100644 src/main/java/flinkgraph/internal/AbstractGraphRepresentation.java delete mode 100644 src/main/java/flinkgraph/internal/EdgeAndVertexGraphRepresentation.java delete mode 100644 src/main/java/flinkgraph/internal/EdgeOnlyGraphRepresentation.java delete mode 100644 src/main/java/flinkgraph/internal/utils/Tuple1Unpackfunction.java delete mode 100644 src/main/java/flinkgraph/internal/utils/Tuple2ToEdgeMapper.java delete mode 100644 src/main/java/flinkgraphs/misc/DenseIdAssignment.java diff --git a/pom.xml b/pom.xml index f57ab9b9e7950..d658b2451633d 100644 --- a/pom.xml +++ b/pom.xml @@ -56,6 +56,16 @@ under the License. flink-clients 0.7-incubating-SNAPSHOT + + org.apache.flink + flink-spargel + 0.7-incubating-SNAPSHOT + + + org.apache.flink + flink-avro + 0.7-incubating-SNAPSHOT + @@ -66,9 +66,16 @@ under the License. flink-avro 0.7-incubating-SNAPSHOT + + junit + junit + 4.11 + jar + test + - diff --git a/src/main/java/flink/graphs/Edge.java b/src/main/java/flink/graphs/Edge.java index 74dea86dcd0b6..a3a23863ff4b6 100644 --- a/src/main/java/flink/graphs/Edge.java +++ b/src/main/java/flink/graphs/Edge.java @@ -21,8 +21,20 @@ public class Edge { private K source; - + private K target; - + private V value; + + public K getSource() { + return source; + } + + public K getTarget() { + return target; + } + + public V getValue() { + return value; + } } diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 830916d708892..55f7727184c9d 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -18,31 +18,111 @@ package flink.graphs; +import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.Collector; + +import javax.xml.crypto.Data; public class Graph, VV, EV> { private final DataSet> vertices; - + private final DataSet> edges; - - - + + + public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; } - - + public DataSet> getVertices() { return vertices; } - + public DataSet> getEdges() { return edges; } - - - + + // Return a Vertex Dataset or a new Graph? + public DataSet> mapVertices(final MapFunction mapper) { + + return vertices.map(new MapFunction, Vertex>() { + @Override + public Vertex map(Vertex kvvVertex) throws Exception { + // Return new object for every Vertex not a good idea probably + return new Vertex<>(kvvVertex.getKey(), mapper.map(kvvVertex.getValue())); + } + }); + } + + + // TODO(thvasilo): Add proper edge filtering functionality + public Graph subgraph(final FilterFunction vertexFilter, final FilterFunction edgeFilter) { + + DataSet> filteredVertices = this.vertices.filter(new FilterFunction>() { + @Override + public boolean filter(Vertex kvvVertex) throws Exception { + return vertexFilter.filter(kvvVertex.getValue()); + } + }); + + // Should combine with vertex filter function as well, so that only + // edges that satisfy edge filter *and* connect vertices that satisfy vertex + // filter are returned + DataSet> filteredEdges = this.edges.filter(new FilterFunction>() { + @Override + public boolean filter(Edge kevEdge) throws Exception { + return edgeFilter.filter(kevEdge.getValue()); + } + }); + + return new Graph(filteredVertices, filteredEdges); + } + + public DataSet> outDegrees() { + return this.edges + .groupBy(new KeySelector, K>() { + @Override + public K getKey(Edge kevEdge) throws Exception { + return kevEdge.getSource(); + } + }) + .reduceGroup(new GroupReduceFunction, Tuple2>() { + @Override + public void reduce(Iterable> edges, Collector> integerCollector) throws Exception { + + int count = 0; + for (Edge edge : edges) { + count++; + } + + integerCollector.collect(new Tuple2(edges.iterator().next().getSource(), count)); + } + }); + } + + public Graph pga(CoGroupFunction, Edge, Tuple2> cog, + GroupReduceFunction, Tuple2> gred, + FlatJoinFunction, Vertex, Vertex> fjoin, + int maxIterations){ + + DeltaIteration, Vertex> iteration = this.vertices + .iterateDelta(this.vertices, maxIterations, 0); + + DataSet> p = iteration.getWorkset().coGroup(this.edges).where(0).equalTo(0).with(cog); + + DataSet> g = p.groupBy(0).reduceGroup(gred); + + DataSet> a = g.join(iteration.getSolutionSet()).where(0).equalTo(0).with(fjoin); + + DataSet> result = iteration.closeWith(a, a); + + return new Graph<>(result, this.edges); + } } diff --git a/src/main/java/flink/graphs/Vertex.java b/src/main/java/flink/graphs/Vertex.java index b43b7177a140f..27d59c4e7623c 100644 --- a/src/main/java/flink/graphs/Vertex.java +++ b/src/main/java/flink/graphs/Vertex.java @@ -20,7 +20,21 @@ public class Vertex { - private K key; - + public Vertex(K key, V value) { + this.key = key; + this.value = value; + } + + private K key; + private V value; + + public V getValue() { + return value; + } + + public K getKey() { + return key; + } } + diff --git a/src/test/java/flink/graphs/GraphTest.java b/src/test/java/flink/graphs/GraphTest.java new file mode 100644 index 0000000000000..c015278195437 --- /dev/null +++ b/src/test/java/flink/graphs/GraphTest.java @@ -0,0 +1,70 @@ +package flink.graphs; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.io.LocalCollectionOutputFormat; +import org.junit.Test; +import static org.junit.Assert.*; + +import java.util.ArrayList; +import java.util.List; + + +public class GraphTest { + + // Assume existing graph object + // Vertex values: 0,1,2,3 + // Edges: 0->1, 1->3, 0->3, 1->2 + + Graph graph = new Graph<>(); + + @Test + public void testGraphCreation() { + + } + + @Test + public void testGetVertices() throws Exception { + + } + + @Test + public void testGetEdges() throws Exception { + + } + + @Test + public void testMapVertices() throws Exception { + DataSet> doubled= graph.mapVertices(new MapFunction() { + @Override + public Integer map(Integer value) throws Exception { + return value * 2; + } + }); + + // Compare the two Datasets as lists? + + List> doubledData = new ArrayList<>(); + doubled.output(new LocalCollectionOutputFormat<>(doubledData)); + + + + DataSet> doubledDataset = graph.getVertices() + .map(new MapFunction, Vertex>() { + @Override + public Vertex map(Vertex v) throws Exception { + return new Vertex(v.getKey(), v.getValue() * 2); + } + }); + List> originalDataDoubled = new ArrayList<>(); + doubledDataset.output(new LocalCollectionOutputFormat<>(originalDataDoubled)); + + assertEquals(doubledData, originalDataDoubled); + + // TODO(thvasilo): Test for function that changes the type of the value + } + + @Test + public void testSubgraph() throws Exception { + + } +} \ No newline at end of file From 43e8b56f81006a85699c842a44b00b1b395474dd Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 10 Oct 2014 13:02:17 +0200 Subject: [PATCH 013/112] Initial commit --- README.md | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 README.md diff --git a/README.md b/README.md new file mode 100644 index 0000000000000..b8aa04dfdcfe4 --- /dev/null +++ b/README.md @@ -0,0 +1,4 @@ +flink-graph +=========== + +Graph API for Apache Flink From 0b39cc70130a92d67b908a379fae3347d648dc99 Mon Sep 17 00:00:00 2001 From: Theodore Vasiloudis Date: Fri, 10 Oct 2014 16:09:36 +0200 Subject: [PATCH 014/112] Reverted to using Tuples, added tests. --- src/main/java/flink/graphs/Graph.java | 169 +++++++++++----------- src/test/java/flink/graphs/GraphTest.java | 169 ++++++++++++++++++++-- 2 files changed, 238 insertions(+), 100 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index ed8accb646a70..f5b03e2becf3b 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -35,79 +35,78 @@ @SuppressWarnings("serial") public class Graph & Serializable, VV extends Serializable, - EV extends Serializable>{ + EV extends Serializable> implements Serializable{ - private final DataSet> vertices; + private final DataSet> vertices; - private final DataSet> edges; + private final DataSet> edges; /** a graph is directed by default */ private boolean isUndirected = false; - public Graph(DataSet> vertices, DataSet> edges) { + public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; } - public Graph(DataSet> vertices, DataSet> edges, + public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { this.vertices = vertices; this.edges = edges; this.isUndirected = undirected; } - public DataSet> getVertices() { + public DataSet> getVertices() { return vertices; } - public DataSet> getEdges() { + public DataSet> getEdges() { return edges; } /** - * Apply a function to the attribute of each vertex in the graph - * @param mapper A function that transforms the attribute of each vertex - * @param The type of the vertex attribute after the function has been applied - * @return A DataSet of Vertex which contains the new values of all vertices + * Apply a function to the attribute of each Tuple2 in the graph + * @param mapper A function that transforms the attribute of each Tuple2 + * @return A DataSet of Tuple2 which contains the new values of all vertices */ - public DataSet> mapVertices(final MapFunction mapper) { - // Return a Vertex Dataset or a new Graph? - return vertices.map(new MapFunction, Vertex>() { + //TODO(thvasilo): Make it possible for the function to change the attribute type + public DataSet> mapVertices(final MapFunction mapper) { + // Return a Tuple2 Dataset or a new Graph? + return vertices.map(new MapFunction, Tuple2>() { @Override - public Vertex map(Vertex kvvVertex) throws Exception { - // Return new object for every Vertex not a good idea probably - return new Vertex<>(kvvVertex.getId(), mapper.map(kvvVertex.getValue())); + public Tuple2 map(Tuple2 kvvTuple2) throws Exception { + // Return new object for every Tuple2 not a good idea probably + return new Tuple2<>(kvvTuple2.f0, mapper.map(kvvTuple2.f1)); } }); } - /** * Apply filtering functions to the graph and return a sub-graph that satisfies * the predicates - * @param vertexFilter + * @param Tuple2Filter * @param edgeFilter * @return */ // TODO(thvasilo): Add proper edge filtering functionality - public Graph subgraph(final FilterFunction vertexFilter, final FilterFunction edgeFilter) { + public Graph subgraph(final FilterFunction Tuple2Filter, final FilterFunction edgeFilter) { - DataSet> filteredVertices = this.vertices.filter(new FilterFunction>() { + DataSet> filteredVertices = this.vertices.filter(new FilterFunction>() { @Override - public boolean filter(Vertex kvvVertex) throws Exception { - return vertexFilter.filter(kvvVertex.getValue()); + public boolean filter(Tuple2 kvvTuple2) throws Exception { + return Tuple2Filter.filter(kvvTuple2.f1); } }); - // Should combine with vertex filter function as well, so that only - // edges that satisfy edge filter *and* connect vertices that satisfy vertex + // Should combine with Tuple2 filter function as well, so that only + // edges that satisfy edge filter *and* connect vertices that satisfy Tuple2 // filter are returned - DataSet> filteredEdges = this.edges.filter(new FilterFunction>() { + DataSet> filteredEdges = this.edges.filter(new FilterFunction>() { @Override - public boolean filter(Edge kevEdge) throws Exception { - return edgeFilter.filter(kevEdge.getValue()); + public boolean filter(Tuple3 kevEdge) throws Exception { + return edgeFilter.filter(kevEdge.f2); } }); @@ -121,23 +120,23 @@ public boolean filter(Edge kevEdge) throws Exception { */ public DataSet> outDegrees() { return this.edges - .groupBy(new KeySelector, K>() { + .groupBy(new KeySelector, K>() { @Override - public K getKey(Edge kevEdge) throws Exception { - return kevEdge.getSource(); + public K getKey(Tuple3 kevEdge) throws Exception { + return kevEdge.f0; } }) - .reduceGroup(new GroupReduceFunction, Tuple2>() { + .reduceGroup(new GroupReduceFunction, Tuple2>() { @Override - public void reduce(Iterable> edges, Collector> integerCollector) + public void reduce(Iterable> edges, Collector> integerCollector) throws Exception { int count = 0; - for (Edge edge : edges) { + for (Tuple3 edge : edges) { count++; } - integerCollector.collect(new Tuple2(edges.iterator().next().getSource(), count)); + integerCollector.collect(new Tuple2(edges.iterator().next().f0, count)); } }); } @@ -151,21 +150,21 @@ public void reduce(Iterable> edges, Collector> in * @param * @return */ - public Graph pga(CoGroupFunction, Edge, Tuple2> cog, + public Graph pga(CoGroupFunction, Tuple3, Tuple2> cog, GroupReduceFunction, Tuple2> gred, - FlatJoinFunction, Vertex, Vertex> fjoin, + FlatJoinFunction, Tuple2, Tuple2> fjoin, int maxIterations){ - DeltaIteration, Vertex> iteration = this.vertices + DeltaIteration, Tuple2> iteration = this.vertices .iterateDelta(this.vertices, maxIterations, 0); DataSet> p = iteration.getWorkset().coGroup(this.edges).where(0).equalTo(0).with(cog); DataSet> g = p.groupBy(0).reduceGroup(gred); - DataSet> a = g.join(iteration.getSolutionSet()).where(0).equalTo(0).with(fjoin); + DataSet> a = g.join(iteration.getSolutionSet()).where(0).equalTo(0).with(fjoin); - DataSet> result = iteration.closeWith(a, a); + DataSet> result = iteration.closeWith(a, a); return new Graph<>(result, this.edges); } @@ -180,14 +179,14 @@ public Graph getUndirected() throws UnsupportedOperationException { throw new UnsupportedOperationException(""); } else { - DataSet> undirectedEdges = edges.flatMap( - new FlatMapFunction, Edge>() { - public void flatMap(Edge edge, Collector> out){ + DataSet> undirectedEdges = edges.flatMap( + new FlatMapFunction, Tuple3>() { + public void flatMap(Tuple3 edge, Collector> out){ out.collect(edge); - out.collect(edge.reverse()); + out.collect(new Tuple3(edge.f1, edge.f0, edge.f2)); } }); - return new Graph(vertices, (DataSet>) undirectedEdges, true); + return new Graph(vertices, (DataSet>) undirectedEdges, true); } } @@ -201,42 +200,42 @@ public Graph reverse() throws UnsupportedOperationException { throw new UnsupportedOperationException(""); } else { - DataSet> undirectedEdges = edges.map(new MapFunction, - Edge>() { - public Edge map(Edge edge){ - return edge.reverse(); + DataSet> undirectedEdges = edges.map(new MapFunction, + Tuple3>() { + public Tuple3 map(Tuple3 edge){ + return new Tuple3(edge.f1, edge.f0, edge.f2); } }); - return new Graph(vertices, (DataSet>) undirectedEdges, true); + return new Graph(vertices, (DataSet>) undirectedEdges, true); } } public static & Serializable, VV extends Serializable, EV extends Serializable> Graph - create(DataSet> vertices, DataSet> edges) { + create(DataSet> vertices, DataSet> edges) { return new Graph(vertices, edges); } /** - * Read and create the graph vertex dataset from a csv file + * Read and create the graph Tuple2 dataset from a csv file * @param env * @param filePath * @param delimiter - * @param vertexIdClass - * @param vertexValueClass + * @param Tuple2IdClass + * @param Tuple2ValueClass * @return */ public static & Serializable, VV extends Serializable> - DataSet> readVertexCsvFile(ExecutionEnvironment env, String filePath, - char delimiter, Class vertexIdClass, Class vertexValueClass) { + DataSet> readTuple2CsvFile(ExecutionEnvironment env, String filePath, + char delimiter, Class Tuple2IdClass, Class Tuple2ValueClass) { CsvReader reader = new CsvReader(filePath, env); - DataSet> vertices = reader.fieldDelimiter(delimiter).types(vertexIdClass, vertexValueClass) - .map(new MapFunction, Vertex>() { + DataSet> vertices = reader.fieldDelimiter(delimiter).types(Tuple2IdClass, Tuple2ValueClass) + .map(new MapFunction, Tuple2>() { - public Vertex map(Tuple2 value) throws Exception { - return (Vertex)value; + public Tuple2 map(Tuple2 value) throws Exception { + return (Tuple2)value; } }); return vertices; @@ -247,21 +246,21 @@ public Vertex map(Tuple2 value) throws Exception { * @param env * @param filePath * @param delimiter - * @param vertexIdClass + * @param Tuple2IdClass * @param edgeValueClass * @return */ public static & Serializable, EV extends Serializable> - DataSet> readEdgesCsvFile(ExecutionEnvironment env, String filePath, - char delimiter, Class vertexIdClass, Class edgeValueClass) { + DataSet> readEdgesCsvFile(ExecutionEnvironment env, String filePath, + char delimiter, Class Tuple2IdClass, Class edgeValueClass) { CsvReader reader = new CsvReader(filePath, env); - DataSet> edges = reader.fieldDelimiter(delimiter) - .types(vertexIdClass, vertexIdClass, edgeValueClass) - .map(new MapFunction, Edge>() { + DataSet> edges = reader.fieldDelimiter(delimiter) + .types(Tuple2IdClass, Tuple2IdClass, edgeValueClass) + .map(new MapFunction, Tuple3>() { - public Edge map(Tuple3 value) throws Exception { - return (Edge)value; + public Tuple3 map(Tuple3 value) throws Exception { + return (Tuple3)value; } }); return edges; @@ -271,37 +270,37 @@ public Edge map(Tuple3 value) throws Exception { * Create the graph, by reading a csv file for vertices * and a csv file for the edges * @param env - * @param vertexFilepath - * @param vertexDelimiter + * @param Tuple2Filepath + * @param Tuple2Delimiter * @param edgeFilepath * @param edgeDelimiter - * @param vertexIdClass - * @param vertexValueClass + * @param Tuple2IdClass + * @param Tuple2ValueClass * @param edgeValueClass * @return */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph readGraphFromCsvFile(ExecutionEnvironment env, - String vertexFilepath, char vertexDelimiter, String edgeFilepath, char edgeDelimiter, - Class vertexIdClass, Class vertexValueClass, Class edgeValueClass) { + String Tuple2Filepath, char Tuple2Delimiter, String edgeFilepath, char edgeDelimiter, + Class Tuple2IdClass, Class Tuple2ValueClass, Class edgeValueClass) { - CsvReader vertexReader = new CsvReader(vertexFilepath, env); - DataSet> vertices = vertexReader.fieldDelimiter(vertexDelimiter) - .types(vertexIdClass, vertexValueClass).map(new MapFunction, - Vertex>() { + CsvReader Tuple2Reader = new CsvReader(Tuple2Filepath, env); + DataSet> vertices = Tuple2Reader.fieldDelimiter(Tuple2Delimiter) + .types(Tuple2IdClass, Tuple2ValueClass).map(new MapFunction, + Tuple2>() { - public Vertex map(Tuple2 value) throws Exception { - return (Vertex)value; + public Tuple2 map(Tuple2 value) throws Exception { + return (Tuple2)value; } }); CsvReader edgeReader = new CsvReader(edgeFilepath, env); - DataSet> edges = edgeReader.fieldDelimiter(edgeDelimiter) - .types(vertexIdClass, vertexIdClass, edgeValueClass) - .map(new MapFunction, Edge>() { + DataSet> edges = edgeReader.fieldDelimiter(edgeDelimiter) + .types(Tuple2IdClass, Tuple2IdClass, edgeValueClass) + .map(new MapFunction, Tuple3>() { - public Edge map(Tuple3 value) throws Exception { - return (Edge)value; + public Tuple3 map(Tuple3 value) throws Exception { + return (Tuple3)value; } }); diff --git a/src/test/java/flink/graphs/GraphTest.java b/src/test/java/flink/graphs/GraphTest.java index 176fc73afdc93..bb1411a86e3af 100644 --- a/src/test/java/flink/graphs/GraphTest.java +++ b/src/test/java/flink/graphs/GraphTest.java @@ -1,43 +1,108 @@ package flink.graphs; + + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; +import org.apache.flink.util.Collector; +import org.junit.Before; import org.junit.Test; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; + import static org.junit.Assert.*; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; -public class GraphTest { +public class GraphTest implements Serializable{ // Assume existing graph object - // Vertex values: 0,1,2,3 + // Tuple2 ids and values: 0,1,2,3 // Edges: 0->1, 1->3, 0->3, 1->2 - DataSet> vertices; - DataSet> edges; + static Graph graph; + static ExecutionEnvironment env; + + @Before + public void testSetUp() { + env = ExecutionEnvironment.getExecutionEnvironment(); + setUpGraph(); + } + + + public static void setUpGraph() { + + List> Tuple2List = new ArrayList>(); + + for (int i = 0; i < 4; i++) { + Tuple2 v = new Tuple2(i, i); + Tuple2List.add(v); + } + + + List> edgeList = new ArrayList<>(); + + edgeList.add(new Tuple3(0, 1, 0)); + edgeList.add(new Tuple3(1, 3, 0)); + edgeList.add(new Tuple3(0, 3, 0)); + edgeList.add(new Tuple3(1, 2, 0)); - Graph graph = new Graph(vertices, edges); + DataSet> vertices = env.fromCollection(Tuple2List); + DataSet> edges = env.fromCollection(edgeList); + + graph = new Graph(vertices, edges); + } @Test - public void testGraphCreation() { + public void testCreate() throws Exception { + + List> Tuple2List = new ArrayList>(); + + for (int i = 0; i < 4; i++) { + Tuple2 v = new Tuple2(i, i); + Tuple2List.add(v); + } + + + List> edgeList = new ArrayList<>(); + edgeList.add(new Tuple3(0, 1, 0)); + edgeList.add(new Tuple3(1, 3, 0)); + edgeList.add(new Tuple3(0, 3, 0)); + edgeList.add(new Tuple3(1, 2, 0)); + + DataSet> vertices = env.fromCollection(Tuple2List); + DataSet> edges = env.fromCollection(edgeList); + + Graph g = Graph.create(vertices, edges); + + g.getVertices().print(); + + env.execute(); } @Test public void testGetVertices() throws Exception { - + throw new NotImplementedException(); } @Test public void testGetEdges() throws Exception { - + throw new NotImplementedException(); } @Test public void testMapVertices() throws Exception { - DataSet> doubled= graph.mapVertices(new MapFunction() { + + DataSet> doubled= graph.mapVertices(new MapFunction() { @Override public Integer map(Integer value) throws Exception { return value * 2; @@ -46,28 +111,102 @@ public Integer map(Integer value) throws Exception { // Compare the two Datasets as lists? - List> doubledData = new ArrayList<>(); + List> doubledData = new ArrayList<>(); doubled.output(new LocalCollectionOutputFormat<>(doubledData)); - DataSet> doubledDataset = graph.getVertices() - .map(new MapFunction, Vertex>() { + DataSet> doubledDataset = graph.getVertices() + .map(new MapFunction, Tuple2>() { @Override - public Vertex map(Vertex v) throws Exception { - return new Vertex(v.getId(), v.getValue() * 2); + public Tuple2 map(Tuple2 v) throws Exception { + return new Tuple2(v.f0, v.f1 * 2); } }); - List> originalDataDoubled = new ArrayList<>(); + List> originalDataDoubled = new ArrayList<>(); doubledDataset.output(new LocalCollectionOutputFormat<>(originalDataDoubled)); assertEquals(doubledData, originalDataDoubled); // TODO(thvasilo): Test for function that changes the type of the value + + doubled.print(); + graph.getVertices().print(); + + env.execute(); + } @Test public void testSubgraph() throws Exception { + throw new NotImplementedException(); + } + + @Test + public void testPga() throws Exception { + // Test pga by running connected components + // Expected output is that all vertices end up with the same attribute, 0 + + // Send the vertex attribute to all neighbors + CoGroupFunction, Tuple3, Tuple2> + sendAttribute = + new CoGroupFunction, Tuple3, Tuple2>() { + @Override + public void coGroup(Iterable> vertices, + Iterable> edges, + Collector> tuple2Collector) throws Exception { + for (Tuple2 vertex : vertices) { + for (Tuple3 edge: edges) { + tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); + } + } + } + }; + + // Gather all messages and keep the message with the smallest attribute + GroupReduceFunction, Tuple2> + gatherAttributes = + new GroupReduceFunction, Tuple2>() { + @Override + public void reduce(Iterable> messages, + Collector> msgCollector) throws Exception { + + Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); + for (Tuple2 message : messages) { + if (message.f1 < minTuple.f1) { + minTuple = message.copy(); + } + } + msgCollector.collect(minTuple); + } + }; + + // Check if the produced message is smaller than the current vertex attribute, if yes change attribute + FlatJoinFunction, Tuple2, Tuple2> + apply = + new FlatJoinFunction, Tuple2, Tuple2>() { + @Override + public void join(Tuple2 msg, + Tuple2 vertex, + Collector> vertexCollector) throws Exception { + if (msg.f1 < vertex.f1) { + vertexCollector.collect(msg.copy()); + } + } + }; + + + // Run the pga iterations + Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); + + DataSet> conVerts = connected.getVertices(); + + // All vertices should end up with attribute 0 + conVerts.print(); + //TODO(thvasilo): Automate correctness testing + + env.execute(); } + } \ No newline at end of file From fee97bb4e6daa287b7fd8a678395799fbc7905a0 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Tue, 21 Oct 2014 08:11:02 +0200 Subject: [PATCH 015/112] Added Kostas' functions from the hackathon; added ExecutionEnvironment member to Graph --- src/main/java/flink/graphs/Graph.java | 206 ++++++++++++++++++++- src/main/java/flink/graphs/GraphUtils.java | 27 +++ src/test/java/flink/graphs/GraphTest.java | 4 +- 3 files changed, 225 insertions(+), 12 deletions(-) create mode 100644 src/main/java/flink/graphs/GraphUtils.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index f5b03e2becf3b..bf502a936d57a 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -21,22 +21,30 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.io.CsvReader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.spargel.java.VertexCentricIteration; import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; @SuppressWarnings("serial") public class Graph & Serializable, VV extends Serializable, EV extends Serializable> implements Serializable{ + private final ExecutionEnvironment context; + private final DataSet> vertices; private final DataSet> edges; @@ -46,15 +54,17 @@ public class Graph & Serializable, VV extends Serializab private boolean isUndirected = false; - public Graph(DataSet> vertices, DataSet> edges) { + public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { this.vertices = vertices; this.edges = edges; + this.context = context; } - public Graph(DataSet> vertices, DataSet> edges, + public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, boolean undirected) { this.vertices = vertices; this.edges = edges; + this.context = context; this.isUndirected = undirected; } @@ -110,7 +120,7 @@ public boolean filter(Tuple3 kevEdge) throws Exception { } }); - return new Graph(filteredVertices, filteredEdges); + return new Graph(filteredVertices, filteredEdges, this.context); } @@ -166,7 +176,7 @@ public Graph pga(CoGroupFunction, Tuple3> result = iteration.closeWith(a, a); - return new Graph<>(result, this.edges); + return new Graph<>(result, this.edges, this.context); } /** @@ -186,7 +196,7 @@ public void flatMap(Tuple3 edge, Collector> out){ out.collect(new Tuple3(edge.f1, edge.f0, edge.f2)); } }); - return new Graph(vertices, (DataSet>) undirectedEdges, true); + return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); } } @@ -206,14 +216,14 @@ public Tuple3 map(Tuple3 edge){ return new Tuple3(edge.f1, edge.f0, edge.f2); } }); - return new Graph(vertices, (DataSet>) undirectedEdges, true); + return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); } } public static & Serializable, VV extends Serializable, EV extends Serializable> Graph - create(DataSet> vertices, DataSet> edges) { - return new Graph(vertices, edges); + create(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { + return new Graph(vertices, edges, context); } @@ -282,7 +292,8 @@ public Tuple3 map(Tuple3 value) throws Exception { public static & Serializable, VV extends Serializable, EV extends Serializable> Graph readGraphFromCsvFile(ExecutionEnvironment env, String Tuple2Filepath, char Tuple2Delimiter, String edgeFilepath, char edgeDelimiter, - Class Tuple2IdClass, Class Tuple2ValueClass, Class edgeValueClass) { + Class Tuple2IdClass, Class Tuple2ValueClass, Class edgeValueClass, + ExecutionEnvironment context) { CsvReader Tuple2Reader = new CsvReader(Tuple2Filepath, env); DataSet> vertices = Tuple2Reader.fieldDelimiter(Tuple2Delimiter) @@ -304,7 +315,182 @@ public Tuple3 map(Tuple3 value) throws Exception { } }); - return Graph.create(vertices, edges); + return Graph.create(vertices, edges, context); } + /** + * @return Singleton DataSet containing the vertex count + */ + public DataSet numberOfVertices () { + return GraphUtils.count((DataSet) (DataSet) vertices); + + } + + /** + * + * @return Singleton DataSet containing the edge count + */ + public DataSet numberOfEdges () { + return GraphUtils.count((DataSet) (DataSet) edges); + } + + + /** + * + * @return The IDs of the vertices as DataSet + */ + public DataSet getVertexIds () { + return vertices.map(new MapFunction, K>() { + @Override + public K map(Tuple2 vertex) throws Exception { + return vertex.f0; + } + }); + } + + public DataSet> getEdgeIds () { + return edges.map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Tuple3 edge) throws Exception { + return new Tuple2(edge.f0, edge.f1); + } + }); + } + + public DataSet isWeaklyConnected () { + + DataSet vertexIds = this.getVertexIds(); + DataSet> verticesWithInitialIds = vertexIds + .map(new MapFunction>() { + @Override + public Tuple2 map(K k) throws Exception { + return new Tuple2(k, k); + } + }); + + DataSet> edgeIds = this.getEdgeIds(); + + DeltaIteration, Tuple2> iteration = verticesWithInitialIds + .iterateDelta(verticesWithInitialIds, 10, 0); + + DataSet> changes = iteration.getWorkset() + .join(edgeIds).where(0).equalTo(0) + .with(new JoinFunction, Tuple2, Tuple2>() { + @Override + public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) throws Exception { + return new Tuple2(edge.f1, vertexWithComponent.f1); + } + }) + .groupBy(0) + .aggregate(Aggregations.MIN, 1) + .join(iteration.getSolutionSet()).where(0).equalTo(0) + .with(new FlatJoinFunction, Tuple2, Tuple2>() { + @Override + public void join(Tuple2 candidate, Tuple2 old, Collector> out) throws Exception { + if (candidate.f1.compareTo(old.f1) < 0) { + out.collect(candidate); + } + } + }); + + DataSet> components = iteration.closeWith(changes, changes); + + DataSet result = GraphUtils.count((DataSet) (DataSet) components) + .map(new MapFunction() { + @Override + public Boolean map(Integer n) throws Exception { + if (n == 1) + return false; + else + return true; + } + }); + + return result; + } + + //TODO kostas add functionality + public Graph fromCollection (Collection> vertices, Collection> edges) { + return null; + } + + //TODO kostas add functionality + public DataSet> fromCollection (Collection> vertices) { + return null; + } + + + public Graph addVertex (Tuple2 vertex, List> edges) { + Graph newVertex = this.fromCollection(Arrays.asList(vertex), edges); + return this.union(newVertex); + } + + public Graph removeVertex (Tuple2 vertex) { + + DataSet> vertexToRemove = fromCollection(Arrays.asList(vertex)); + + DataSet> newVertices = getVertices() + .filter(new RichFilterFunction>() { + private Tuple2 vertexToRemove; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + } + + @Override + public boolean filter(Tuple2 vertex) throws Exception { + if (vertex.f0.equals(vertexToRemove.f0)) { + return false; + } else { + return true; + } + } + }).withBroadcastSet(vertexToRemove, "vertexToRemove"); + + DataSet> newEdges = getEdges() + .filter(new RichFilterFunction>() { + private Tuple2 vertexToRemove; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + } + + @Override + public boolean filter(Tuple3 edge) throws Exception { + if (edge.f0.equals(vertexToRemove.f0)) { + return false; + } + if (edge.f1.equals(vertexToRemove.f0)) { + return false; + } + return true; + } + }).withBroadcastSet(vertexToRemove, "vertexToRemove"); + + return new Graph(newVertices, newEdges, this.context); + } + + + public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { + Graph newEdges = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); + return this.union(newEdges); + } + + public Graph union (Graph graph) { + DataSet> unionedVertices = graph.getVertices().union(this.getVertices()); + DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); + return new Graph(unionedVertices, unionedEdges, this.context); + } + + + public Graph passMessages (VertexCentricIteration iteration) { + DataSet> newVertices = iteration.createResult(); + return new Graph(newVertices, edges, this.context); + } + + } diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java new file mode 100644 index 0000000000000..3acde4feccc20 --- /dev/null +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -0,0 +1,27 @@ +package flink.graphs; + + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; + +public class GraphUtils { + + public static DataSet count (DataSet set) { + return set + .map(new MapFunction() { + @Override + public Integer map(Object o) throws Exception { + return 1; + } + }) + .reduce(new ReduceFunction() { + @Override + public Integer reduce(Integer one, Integer two) throws Exception { + return one + two; + } + }) + .first(1); + } + +} diff --git a/src/test/java/flink/graphs/GraphTest.java b/src/test/java/flink/graphs/GraphTest.java index bb1411a86e3af..b6f5c3a12eb31 100644 --- a/src/test/java/flink/graphs/GraphTest.java +++ b/src/test/java/flink/graphs/GraphTest.java @@ -58,7 +58,7 @@ public static void setUpGraph() { DataSet> vertices = env.fromCollection(Tuple2List); DataSet> edges = env.fromCollection(edgeList); - graph = new Graph(vertices, edges); + graph = new Graph(vertices, edges, env); } @Test @@ -82,7 +82,7 @@ public void testCreate() throws Exception { DataSet> vertices = env.fromCollection(Tuple2List); DataSet> edges = env.fromCollection(edgeList); - Graph g = Graph.create(vertices, edges); + Graph g = Graph.create(vertices, edges, env); g.getVertices().print(); From a405e0973d3157933e78a45fb447fcabb9528e7d Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 21 Oct 2014 17:37:50 +0200 Subject: [PATCH 016/112] reflect current status in README --- README.md | 69 +++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/README.md b/README.md index b8aa04dfdcfe4..b1b620423d136 100644 --- a/README.md +++ b/README.md @@ -2,3 +2,72 @@ flink-graph =========== Graph API for Apache Flink + +##Implemented Operations + +###Graph Class +* getVertices() +* getEdges() +* mapVertices() +* subGraph() +* outDegrees() +* pga() +* getUndirected() +* reverse() +* create(vertices, edges) +* readTuple2CsvFile +* readEdgesCsvFile +* readGraphFromCsvFile +* numberOfVertices() +* numberOfEdges() +* getVertexIds() +* getEdgeIds() +* isWeaklyConnected() +* addVertex() +* removeVertex() +* addEdge() +* union() +* passMessages() + + +###Vertex Class + +###Edge Class +* reverse() + +##Tested Operations + +##Wishlist + +###Graph Class +* fromCollection(vertices, edges) +* fromCollection(vertices) +* mapEdges() +* getNeighborhoodGraph(Vertex src, int distance) +* vertexCentricComputation() +* edgeCentricComputation() +* partitionCentricComputation() + +###Vertex Class +* getDegree() +* inDegree() +* outDegree() +* getInNeighbors() +* getOutNeighbors() +* getAllNeighbors() + + +###Edge Class + +##Other (low priority) +* partitionBy +* sample +* centrality +* pagerank +* distance +* clusteringCoefficient +* dfs +* bfs +* sssp +* isIsomorphic +* isSubgraphOf From 2c8d859cafcedddb463d895b584a4d29d1be5d26 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 25 Oct 2014 16:14:42 +0200 Subject: [PATCH 017/112] type information in Graph and in getUndirected method --- src/main/java/flink/graphs/Graph.java | 68 ++++++++++++++++++++++----- 1 file changed, 56 insertions(+), 12 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index f5b03e2becf3b..536dee30bc31a 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -21,15 +21,20 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.util.Collector; -import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; + import java.io.Serializable; @@ -41,6 +46,9 @@ public class Graph & Serializable, VV extends Serializab private final DataSet> edges; + private final TypeInformation> verticesType; + + private final TypeInformation> edgesType; /** a graph is directed by default */ private boolean isUndirected = false; @@ -49,12 +57,31 @@ public class Graph & Serializable, VV extends Serializab public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; + + TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + TypeInformation vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + TypeInformation edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); + + TypeInformation[] vertexTypes = {(BasicTypeInfo)keyType, vertexValueType}; + this.verticesType = new TupleTypeInfo>(vertexTypes); + + TypeInformation[] edgeTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, edgeValueType}; + this.edgesType = new TupleTypeInfo>(edgeTypes); } - public Graph(DataSet> vertices, DataSet> edges, - boolean undirected) { + public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { this.vertices = vertices; this.edges = edges; + + TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + TypeInformation vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + TypeInformation edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); + + TypeInformation[] vertexTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, vertexValueType}; + this.verticesType = new TupleTypeInfo>(vertexTypes); + + TypeInformation[] edgeTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, edgeValueType}; + this.edgesType = new TupleTypeInfo>(edgeTypes); this.isUndirected = undirected; } @@ -71,7 +98,7 @@ public DataSet> getEdges() { * @param mapper A function that transforms the attribute of each Tuple2 * @return A DataSet of Tuple2 which contains the new values of all vertices */ - //TODO(thvasilo): Make it possible for the function to change the attribute type + //TODO(thvasilo): Make it possible for the function to change the attribute type public DataSet> mapVertices(final MapFunction mapper) { // Return a Tuple2 Dataset or a new Graph? return vertices.map(new MapFunction, Tuple2>() { @@ -179,14 +206,31 @@ public Graph getUndirected() throws UnsupportedOperationException { throw new UnsupportedOperationException(""); } else { - DataSet> undirectedEdges = edges.flatMap( - new FlatMapFunction, Tuple3>() { - public void flatMap(Tuple3 edge, Collector> out){ - out.collect(edge); - out.collect(new Tuple3(edge.f1, edge.f0, edge.f2)); - } - }); - return new Graph(vertices, (DataSet>) undirectedEdges, true); + DataSet> undirectedEdges = + edges.union(edges.map(new ReverseEdgesMap(edgesType))); + return new Graph(vertices, undirectedEdges, true); + } + } + + @ConstantFields("0->1;1->0;2->2") + private static final class ReverseEdgesMap implements MapFunction, + Tuple3>, ResultTypeQueryable> { + + private transient TypeInformation> resultType; + + private ReverseEdgesMap(TypeInformation> resultType) + { + this.resultType = resultType; + } + + @Override + public TypeInformation> getProducedType() { + return this.resultType; + } + + @Override + public Tuple3 map(Tuple3 value) { + return new Tuple3(value.f1, value.f0, value.f2); } } From 7ff3f8a463a73495bdc3d2e8f11a204c787b2c9c Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 25 Oct 2014 17:08:53 +0200 Subject: [PATCH 018/112] typeinfo in reverse and getOutdegrees --- src/main/java/flink/graphs/Graph.java | 94 +++++++++++++++------------ 1 file changed, 52 insertions(+), 42 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 536dee30bc31a..4e4faadfbc834 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.DeltaIteration; @@ -34,6 +35,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.hadoop.mapred.MapFileOutputFormat; import java.io.Serializable; @@ -46,6 +48,12 @@ public class Graph & Serializable, VV extends Serializab private final DataSet> edges; + private final TypeInformation vertexKeyType; + + private final TypeInformation vertexValueType; + + private final TypeInformation edgeValueType; + private final TypeInformation> verticesType; private final TypeInformation> edgesType; @@ -58,14 +66,15 @@ public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; - TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - TypeInformation vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - TypeInformation edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); + this.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + this.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + this.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); - TypeInformation[] vertexTypes = {(BasicTypeInfo)keyType, vertexValueType}; + TypeInformation[] vertexTypes = {(BasicTypeInfo)vertexKeyType, vertexValueType}; this.verticesType = new TupleTypeInfo>(vertexTypes); - TypeInformation[] edgeTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, edgeValueType}; + TypeInformation[] edgeTypes = {(BasicTypeInfo)vertexKeyType, (BasicTypeInfo)vertexKeyType, + edgeValueType}; this.edgesType = new TupleTypeInfo>(edgeTypes); } @@ -73,14 +82,15 @@ public Graph(DataSet> vertices, DataSet> edges, b this.vertices = vertices; this.edges = edges; - TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - TypeInformation vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - TypeInformation edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); + this.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + this.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + this.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); - TypeInformation[] vertexTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, vertexValueType}; + TypeInformation[] vertexTypes = {(BasicTypeInfo)vertexKeyType, vertexValueType}; this.verticesType = new TupleTypeInfo>(vertexTypes); - TypeInformation[] edgeTypes = {(BasicTypeInfo)keyType, (BasicTypeInfo)keyType, edgeValueType}; + TypeInformation[] edgeTypes = {(BasicTypeInfo)vertexKeyType, (BasicTypeInfo)vertexKeyType, + edgeValueType}; this.edgesType = new TupleTypeInfo>(edgeTypes); this.isUndirected = undirected; } @@ -143,31 +153,37 @@ public boolean filter(Tuple3 kevEdge) throws Exception { /** * Return the out-degree of all vertices in the graph - * @return A DataSet of Tuple2 containing the out-degrees of the vertices in the graph + * @return A DataSet of Tuple2 */ - public DataSet> outDegrees() { - return this.edges - .groupBy(new KeySelector, K>() { - @Override - public K getKey(Tuple3 kevEdge) throws Exception { - return kevEdge.f0; - } - }) - .reduceGroup(new GroupReduceFunction, Tuple2>() { - @Override - public void reduce(Iterable> edges, Collector> integerCollector) - throws Exception { - - int count = 0; - for (Tuple3 edge : edges) { - count++; - } - - integerCollector.collect(new Tuple2(edges.iterator().next().f0, count)); - } - }); - } + public DataSet> outDegrees() { + + TypeInformation[] types = {(BasicTypeInfo)vertexKeyType, BasicTypeInfo.LONG_TYPE_INFO}; + + return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne( + new TupleTypeInfo>(types))) + .groupBy(0).sum(1); + } + private static final class VertexKeyWithOne implements + MapFunction, Tuple3>, Tuple2>, + ResultTypeQueryable> { + + private transient TypeInformation> resultType; + + private VertexKeyWithOne(TypeInformation> resultType) { + this.resultType = resultType; + } + + public Tuple2 map( + Tuple2, Tuple3> value) { + return new Tuple2(value.f0.f0, 1L); + } + + @Override + public TypeInformation> getProducedType() { + return this.resultType; + } + } /** * Push-Gather-Apply model of graph computation * @param cog @@ -203,7 +219,7 @@ public Graph pga(CoGroupFunction, Tuple3 getUndirected() throws UnsupportedOperationException { if (this.isUndirected) { - throw new UnsupportedOperationException(""); + throw new UnsupportedOperationException("The graph is already undirected."); } else { DataSet> undirectedEdges = @@ -241,15 +257,10 @@ public Tuple3 map(Tuple3 value) { */ public Graph reverse() throws UnsupportedOperationException { if (this.isUndirected) { - throw new UnsupportedOperationException(""); + throw new UnsupportedOperationException("The graph is already undirected."); } else { - DataSet> undirectedEdges = edges.map(new MapFunction, - Tuple3>() { - public Tuple3 map(Tuple3 edge){ - return new Tuple3(edge.f1, edge.f0, edge.f2); - } - }); + DataSet> undirectedEdges = edges.map(new ReverseEdgesMap<>(edgesType)); return new Graph(vertices, (DataSet>) undirectedEdges, true); } } @@ -258,7 +269,6 @@ public Tuple3 map(Tuple3 edge){ EV extends Serializable> Graph create(DataSet> vertices, DataSet> edges) { return new Graph(vertices, edges); - } /** From 5af110c10c2301bcca22e19afeb10fff542153e5 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 25 Oct 2014 18:22:11 +0200 Subject: [PATCH 019/112] add dependency to flink-test-utils --- pom.xml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8b4ce73a392d7..261701f463499 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,12 @@ under the License. flink-avro 0.7-incubating-SNAPSHOT - + + org.apache.flink + flink-test-utils + 0.7-incubating-SNAPSHOT + + junit junit 4.11 From 9ad2c319bec98eec2bac9998cedba4e5b84e8139 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 25 Oct 2014 18:27:15 +0200 Subject: [PATCH 020/112] test for undirected, reverse and outDegrees --- .../flink/graphs/TestGraphOperations.java | 122 ++++++++++++++++++ .../java/flink/graphs/TestGraphUtils.java | 38 ++++++ 2 files changed, 160 insertions(+) create mode 100644 src/test/java/flink/graphs/TestGraphOperations.java create mode 100644 src/test/java/flink/graphs/TestGraphUtils.java diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java new file mode 100644 index 0000000000000..9456c13634bb0 --- /dev/null +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -0,0 +1,122 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestGraphOperations extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 3; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestGraphOperations(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test getUndirected() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + graph.getUndirected().getEdges().writeAsCsv(resultPath); + env.execute(); + return "1,2,12\n" + "2,1,12\n" + + "1,3,13\n" + "3,1,13\n" + + "2,3,23\n" + "3,2,23\n" + + "3,4,34\n" + "4,3,34\n" + + "3,5,35\n" + "5,3,35\n" + + "4,5,45\n" + "5,4,45\n" + + "5,1,51\n" + "1,5,51\n"; + } + case 2: { + /* + * Test reverse() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + graph.reverse().getEdges().writeAsCsv(resultPath); + env.execute(); + return "2,1,12\n" + + "3,1,13\n" + + "3,2,23\n" + + "4,3,34\n" + + "5,3,35\n" + + "5,4,45\n" + + "1,5,51\n"; + } + case 3: { + /* + * Test outDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,1\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java new file mode 100644 index 0000000000000..4bccd30a5fa0f --- /dev/null +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -0,0 +1,38 @@ +package flink.graphs; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; + +public class TestGraphUtils { + + public static final DataSet> getLongLongVertexData( + ExecutionEnvironment env) { + List> vertices = new ArrayList>(); + vertices.add(new Tuple2(1L, 1L)); + vertices.add(new Tuple2(2L, 2L)); + vertices.add(new Tuple2(3L, 3L)); + vertices.add(new Tuple2(4L, 4L)); + vertices.add(new Tuple2(5L, 5L)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getLongLongEdgeData( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 2L, 12L)); + edges.add(new Tuple3(1L, 3L, 13L)); + edges.add(new Tuple3(2L, 3L, 23L)); + edges.add(new Tuple3(3L, 4L, 34L)); + edges.add(new Tuple3(3L, 5L, 35L)); + edges.add(new Tuple3(4L, 5L, 45L)); + edges.add(new Tuple3(5L, 1L, 51L)); + + return env.fromCollection(edges); + } +} From 393902c3b214c2b1de67f0d59e543ec7bc4847c7 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 25 Oct 2014 18:30:20 +0200 Subject: [PATCH 021/112] remove unused imports --- src/main/java/flink/graphs/Graph.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 4e4faadfbc834..99cac1c6048ac 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -21,21 +21,17 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.io.CsvReader; -import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.hadoop.mapred.MapFileOutputFormat; import java.io.Serializable; From c7e805fd747d8cceb4cb678bfa3257434d52e437 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 26 Oct 2014 20:40:32 +0100 Subject: [PATCH 022/112] made functions for getUndirected, outDegrees and reverse static inner --- src/main/java/flink/graphs/Graph.java | 78 +++------------------------ 1 file changed, 7 insertions(+), 71 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 99cac1c6048ac..1c52363009951 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -25,14 +25,9 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; - import java.io.Serializable; @@ -44,16 +39,6 @@ public class Graph & Serializable, VV extends Serializab private final DataSet> edges; - private final TypeInformation vertexKeyType; - - private final TypeInformation vertexValueType; - - private final TypeInformation edgeValueType; - - private final TypeInformation> verticesType; - - private final TypeInformation> edgesType; - /** a graph is directed by default */ private boolean isUndirected = false; @@ -61,33 +46,11 @@ public class Graph & Serializable, VV extends Serializab public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; - - this.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - this.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - this.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); - - TypeInformation[] vertexTypes = {(BasicTypeInfo)vertexKeyType, vertexValueType}; - this.verticesType = new TupleTypeInfo>(vertexTypes); - - TypeInformation[] edgeTypes = {(BasicTypeInfo)vertexKeyType, (BasicTypeInfo)vertexKeyType, - edgeValueType}; - this.edgesType = new TupleTypeInfo>(edgeTypes); } public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { this.vertices = vertices; this.edges = edges; - - this.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - this.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - this.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); - - TypeInformation[] vertexTypes = {(BasicTypeInfo)vertexKeyType, vertexValueType}; - this.verticesType = new TupleTypeInfo>(vertexTypes); - - TypeInformation[] edgeTypes = {(BasicTypeInfo)vertexKeyType, (BasicTypeInfo)vertexKeyType, - edgeValueType}; - this.edgesType = new TupleTypeInfo>(edgeTypes); this.isUndirected = undirected; } @@ -152,34 +115,20 @@ public boolean filter(Tuple3 kevEdge) throws Exception { * @return A DataSet of Tuple2 */ public DataSet> outDegrees() { - - TypeInformation[] types = {(BasicTypeInfo)vertexKeyType, BasicTypeInfo.LONG_TYPE_INFO}; - return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne( - new TupleTypeInfo>(types))) + return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne()) .groupBy(0).sum(1); } private static final class VertexKeyWithOne implements - MapFunction, Tuple3>, Tuple2>, - ResultTypeQueryable> { - - private transient TypeInformation> resultType; - - private VertexKeyWithOne(TypeInformation> resultType) { - this.resultType = resultType; - } + MapFunction, Tuple3>, Tuple2> { public Tuple2 map( Tuple2, Tuple3> value) { return new Tuple2(value.f0.f0, 1L); } - - @Override - public TypeInformation> getProducedType() { - return this.resultType; - } } + /** * Push-Gather-Apply model of graph computation * @param cog @@ -219,28 +168,15 @@ public Graph getUndirected() throws UnsupportedOperationException { } else { DataSet> undirectedEdges = - edges.union(edges.map(new ReverseEdgesMap(edgesType))); + edges.union(edges.map(new ReverseEdgesMap())); return new Graph(vertices, undirectedEdges, true); - } + } } @ConstantFields("0->1;1->0;2->2") private static final class ReverseEdgesMap implements MapFunction, - Tuple3>, ResultTypeQueryable> { - - private transient TypeInformation> resultType; - - private ReverseEdgesMap(TypeInformation> resultType) - { - this.resultType = resultType; - } - - @Override - public TypeInformation> getProducedType() { - return this.resultType; - } + Tuple3> { - @Override public Tuple3 map(Tuple3 value) { return new Tuple3(value.f1, value.f0, value.f2); } @@ -256,7 +192,7 @@ public Graph reverse() throws UnsupportedOperationException { throw new UnsupportedOperationException("The graph is already undirected."); } else { - DataSet> undirectedEdges = edges.map(new ReverseEdgesMap<>(edgesType)); + DataSet> undirectedEdges = edges.map(new ReverseEdgesMap()); return new Graph(vertices, (DataSet>) undirectedEdges, true); } } From a8e4e74142f6bc87db3182cd6ba9bd3a82a2490c Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 28 Oct 2014 22:39:42 +0100 Subject: [PATCH 023/112] test mapVertices --- src/main/java/flink/graphs/Graph.java | 28 +++++++++------ .../flink/graphs/TestGraphOperations.java | 35 ++++++++++++++++++- 2 files changed, 52 insertions(+), 11 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 1c52363009951..d7a625a7433f3 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; + import java.io.Serializable; @@ -67,16 +68,23 @@ public DataSet> getEdges() { * @param mapper A function that transforms the attribute of each Tuple2 * @return A DataSet of Tuple2 which contains the new values of all vertices */ - //TODO(thvasilo): Make it possible for the function to change the attribute type - public DataSet> mapVertices(final MapFunction mapper) { - // Return a Tuple2 Dataset or a new Graph? - return vertices.map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple2 kvvTuple2) throws Exception { - // Return new object for every Tuple2 not a good idea probably - return new Tuple2<>(kvvTuple2.f0, mapper.map(kvvTuple2.f1)); - } - }); + //TODO: support changing the vertex value type + public DataSet> mapVertices(final MapFunction mapper) { + return vertices.map(new ApplyMapperToVertex(mapper)); + } + + private static final class ApplyMapperToVertex implements MapFunction + , Tuple2> { + + private MapFunction innerMapper; + + public ApplyMapperToVertex(MapFunction theMapper) { + this.innerMapper = theMapper; + } + + public Tuple2 map(Tuple2 value) throws Exception { + return new Tuple2(value.f0, innerMapper.map(value.f1)); + } } /** diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 9456c13634bb0..6dab5c09414ec 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -4,7 +4,11 @@ import java.io.IOException; import java.util.Collection; import java.util.LinkedList; + +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.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -14,7 +18,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 3; + private static int NUM_PROGRAMS = 4; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -55,6 +59,7 @@ public static Collection getConfigurations() throws FileNotFoundExcept private static class GraphProgs { + @SuppressWarnings("serial") public static String runProgram(int progId, String resultPath) throws Exception { switch(progId) { @@ -113,6 +118,34 @@ public static String runProgram(int progId, String resultPath) throws Exception "4,1\n" + "5,1\n"; } + case 4: { + /* + * Test mapVertices() keeping the same value type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public Long map(Long value) throws Exception { + return value+1; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,6\n"; + } + case 5: { + /* + * Test subgraph: + */ + } default: throw new IllegalArgumentException("Invalid program id"); } From 1a404d3b462556811f54c9b5d76fe54db3f411eb Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 28 Oct 2014 22:51:26 +0100 Subject: [PATCH 024/112] remove unecessary type from mapVertices --- src/main/java/flink/graphs/Graph.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index d7a625a7433f3..fceaa49d5e5e0 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -69,7 +69,7 @@ public DataSet> getEdges() { * @return A DataSet of Tuple2 which contains the new values of all vertices */ //TODO: support changing the vertex value type - public DataSet> mapVertices(final MapFunction mapper) { + public DataSet> mapVertices(final MapFunction mapper) { return vertices.map(new ApplyMapperToVertex(mapper)); } From d855bbf72a5ec9e04cadb6400b3ca6575a90335b Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 28 Oct 2014 23:47:55 +0100 Subject: [PATCH 025/112] fixed subGraph() and added test --- src/main/java/flink/graphs/Graph.java | 75 +++++++++++++------ .../flink/graphs/TestGraphOperations.java | 23 +++++- 2 files changed, 74 insertions(+), 24 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index fceaa49d5e5e0..7b12708071bdf 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -22,12 +22,14 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; +import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.util.Collector; import java.io.Serializable; @@ -88,35 +90,64 @@ public Tuple2 map(Tuple2 value) throws Exception { } /** - * Apply filtering functions to the graph and return a sub-graph that satisfies - * the predicates - * @param Tuple2Filter + * Apply value-based filtering functions to the graph + * and return a sub-graph that satisfies the predicates + * for both vertex values and edge values. + * @param vertexFilter * @param edgeFilter * @return */ - // TODO(thvasilo): Add proper edge filtering functionality - public Graph subgraph(final FilterFunction Tuple2Filter, final FilterFunction edgeFilter) { - - DataSet> filteredVertices = this.vertices.filter(new FilterFunction>() { - @Override - public boolean filter(Tuple2 kvvTuple2) throws Exception { - return Tuple2Filter.filter(kvvTuple2.f1); - } - }); - - // Should combine with Tuple2 filter function as well, so that only - // edges that satisfy edge filter *and* connect vertices that satisfy Tuple2 - // filter are returned - DataSet> filteredEdges = this.edges.filter(new FilterFunction>() { - @Override - public boolean filter(Tuple3 kevEdge) throws Exception { - return edgeFilter.filter(kevEdge.f2); - } - }); + public Graph subgraph(FilterFunction vertexFilter, FilterFunction edgeFilter) { + + DataSet> filteredVertices = this.vertices.filter( + new ApplyVertexFilter(vertexFilter)); + + DataSet> remainingEdges = this.edges.join(filteredVertices) + .where(0).equalTo(0) + .with(new ProjectEdge()) + .join(filteredVertices).where(1).equalTo(0) + .with(new ProjectEdge()); + + DataSet> filteredEdges = remainingEdges.filter( + new ApplyEdgeFilter(edgeFilter)); return new Graph(filteredVertices, filteredEdges); } + + @ConstantFieldsFirst("0->0;1->1;2->2") + private static final class ProjectEdge implements FlatJoinFunction, Tuple2, + Tuple3> { + public void join(Tuple3 first, + Tuple2 second, Collector> out) { + out.collect(first); + } + } + + private static final class ApplyVertexFilter implements FilterFunction> { + private FilterFunction innerFilter; + + public ApplyVertexFilter(FilterFunction theFilter) { + this.innerFilter = theFilter; + } + + public boolean filter(Tuple2 value) throws Exception { + return innerFilter.filter(value.f1); + } + + } + + private static final class ApplyEdgeFilter implements FilterFunction> { + + private FilterFunction innerFilter; + + public ApplyEdgeFilter(FilterFunction theFilter) { + this.innerFilter = theFilter; + } + public boolean filter(Tuple3 value) throws Exception { + return innerFilter.filter(value.f2); + } + } /** * Return the out-degree of all vertices in the graph diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 6dab5c09414ec..a4aba1c94aa03 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -5,6 +5,7 @@ import java.util.Collection; import java.util.LinkedList; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -18,7 +19,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 4; + private static int NUM_PROGRAMS = 5; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -143,8 +144,26 @@ public Long map(Long value) throws Exception { } case 5: { /* - * Test subgraph: + * Test subgraph: */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + graph.subgraph(new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 2); + } + }, + new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 34); + } + }).getEdges().writeAsCsv(resultPath); + + env.execute(); + return "3,5,35\n" + + "4,5,45\n"; } default: throw new IllegalArgumentException("Invalid program id"); From 88e39ee3a9375e32e7c785a975277fa3f8ec73b5 Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 28 Oct 2014 23:50:40 +0100 Subject: [PATCH 026/112] add tests in readme --- README.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index b1b620423d136..bf9e022d8eac8 100644 --- a/README.md +++ b/README.md @@ -8,12 +8,7 @@ Graph API for Apache Flink ###Graph Class * getVertices() * getEdges() -* mapVertices() -* subGraph() -* outDegrees() * pga() -* getUndirected() -* reverse() * create(vertices, edges) * readTuple2CsvFile * readEdgesCsvFile @@ -36,6 +31,11 @@ Graph API for Apache Flink * reverse() ##Tested Operations +* mapVertices() +* subGraph() +* outDegrees() +* getUndirected() +* reverse() ##Wishlist From b06dc56087ffdaf884f019f63a921ef4cad9182a Mon Sep 17 00:00:00 2001 From: vasia Date: Wed, 5 Nov 2014 01:03:42 +0100 Subject: [PATCH 027/112] mapVertices allows changing the value type --- pom.xml | 10 +- src/main/java/flink/graphs/Graph.java | 57 +++++--- .../flink/graphs/TestGraphOperations.java | 128 +++++++++++++++++- .../java/flink/graphs/TestGraphUtils.java | 75 +++++++++- 4 files changed, 243 insertions(+), 27 deletions(-) diff --git a/pom.xml b/pom.xml index 261701f463499..95ff80c54f29f 100644 --- a/pom.xml +++ b/pom.xml @@ -49,27 +49,27 @@ under the License. org.apache.flink flink-java - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-clients - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-spargel - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-avro - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-test-utils - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT junit diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 7b12708071bdf..d92cba51aeabf 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -26,8 +26,12 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.util.Collector; @@ -44,17 +48,24 @@ public class Graph & Serializable, VV extends Serializab /** a graph is directed by default */ private boolean isUndirected = false; + + private static TypeInformation vertexKeyType; + private static TypeInformation vertexValueType; public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; + Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); } public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { this.vertices = vertices; this.edges = edges; this.isUndirected = undirected; + Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); } public DataSet> getVertices() { @@ -64,29 +75,37 @@ public DataSet> getVertices() { public DataSet> getEdges() { return edges; } - + /** - * Apply a function to the attribute of each Tuple2 in the graph - * @param mapper A function that transforms the attribute of each Tuple2 - * @return A DataSet of Tuple2 which contains the new values of all vertices + * Apply a function to the attribute of each vertex in the graph. + * @param mapper + * @return */ - //TODO: support changing the vertex value type - public DataSet> mapVertices(final MapFunction mapper) { - return vertices.map(new ApplyMapperToVertex(mapper)); + public DataSet> mapVertices(final MapFunction mapper) { + return vertices.map(new ApplyMapperToVertexWithType(mapper)); } - private static final class ApplyMapperToVertex implements MapFunction - , Tuple2> { - - private MapFunction innerMapper; - - public ApplyMapperToVertex(MapFunction theMapper) { - this.innerMapper = theMapper; - } - - public Tuple2 map(Tuple2 value) throws Exception { - return new Tuple2(value.f0, innerMapper.map(value.f1)); - } + private static final class ApplyMapperToVertexWithType implements MapFunction + , Tuple2>, ResultTypeQueryable> { + + private MapFunction innerMapper; + + public ApplyMapperToVertexWithType(MapFunction theMapper) { + this.innerMapper = theMapper; + } + + public Tuple2 map(Tuple2 value) throws Exception { + return new Tuple2(value.f0, innerMapper.map(value.f1)); + } + + @Override + public TypeInformation> getProducedType() { + @SuppressWarnings("unchecked") + TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, + (TypeInformation)vertexValueType); + + return new TupleTypeInfo>(vertexKeyType, newVertexValueType); + } } /** diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index a4aba1c94aa03..8e41483998e78 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -9,6 +9,7 @@ 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.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; @@ -16,10 +17,13 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; +import flink.graphs.TestGraphUtils.DummyCustomType; + @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - - private static int NUM_PROGRAMS = 5; + + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -143,6 +147,126 @@ public Long map(Long value) throws Exception { "5,6\n"; } case 5: { + /* + * Test mapVertices() and change the value type to String + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public String map(Long value) throws Exception { + String stringValue; + if (value == 1) { + stringValue = "one"; + } + else if (value == 2) { + stringValue = "two"; + } + else if (value == 3) { + stringValue = "three"; + } + else if (value == 4) { + stringValue = "four"; + } + else if (value == 5) { + stringValue = "five"; + } + else { + stringValue = ""; + } + + return stringValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,one\n" + + "2,two\n" + + "3,three\n" + + "4,four\n" + + "5,five\n"; + } + case 6: { + /* + * Test mapVertices() and change the value type to a Tuple1 + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { + public Tuple1 map(Long value) throws Exception { + Tuple1 tupleValue = new Tuple1(); + tupleValue.setFields(value); + return tupleValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(1)\n" + + "2,(2)\n" + + "3,(3)\n" + + "4,(4)\n" + + "5,(5)\n"; + } + case 7: { + /* + * Test mapVertices() and change the value type to a custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public DummyCustomType map(Long value) throws Exception { + DummyCustomType dummyValue = new DummyCustomType(); + dummyValue.setIntField(value.intValue()); + return dummyValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(T,1)\n" + + "2,(T,2)\n" + + "3,(T,3)\n" + + "4,(T,4)\n" + + "5,(T,5)\n"; + } + case 8: { + /* + * Test mapVertices() and change the value type to a parameterized custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedVertices = graph.mapVertices( + new MapFunction>() { + public DummyCustomParameterizedType map(Long value) throws Exception { + DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); + dummyValue.setIntField(value.intValue()); + dummyValue.setTField(new Double(value)); + return dummyValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(1.0,1)\n" + + "2,(2.0,2)\n" + + "3,(3.0,3)\n" + + "4,(4.0,4)\n" + + "5,(5.0,5)\n"; + } + case 9: { /* * Test subgraph: */ diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 4bccd30a5fa0f..8bbdc121547dd 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -1,5 +1,6 @@ package flink.graphs; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -35,4 +36,76 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } -} + + public static class DummyCustomType implements Serializable { + private static final long serialVersionUID = 1L; + + private int intField; + private boolean booleanField; + + public DummyCustomType(int intF, boolean boolF) { + this.intField = intF; + this.booleanField = boolF; + } + + public DummyCustomType() { + this.intField = 0; + this.booleanField = true; + } + + public int getIntField() { + return intField; + } + + public void setIntField(int intF) { + this.intField = intF; + } + + public boolean getBooleanField() { + return booleanField; + } + + @Override + public String toString() { + return booleanField ? "(T," + intField + ")" : "(F," + intField + ")"; + } + } + + public static class DummyCustomParameterizedType implements Serializable { + private static final long serialVersionUID = 1L; + + private int intField; + private T tField; + + public DummyCustomParameterizedType(int intF, T tF) { + this.intField = intF; + this.tField = tF; + } + + public DummyCustomParameterizedType() { + this.intField = 0; + this.tField = null; + } + + public int getIntField() { + return intField; + } + + public void setIntField(int intF) { + this.intField = intF; + } + + public void setTField(T tF) { + this.tField = tF; + } + + public T getTField() { + return tField; + } + + @Override + public String toString() { + return "(" + tField.toString() + "," + intField + ")"; + } + } +} \ No newline at end of file From e6def06641f294d8f680fc69fdf91829662ccd3e Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 11 Nov 2014 15:05:10 +0100 Subject: [PATCH 028/112] remove old hackathon test file --- src/test/java/flink/graphs/GraphTest.java | 212 ---------------------- 1 file changed, 212 deletions(-) delete mode 100644 src/test/java/flink/graphs/GraphTest.java diff --git a/src/test/java/flink/graphs/GraphTest.java b/src/test/java/flink/graphs/GraphTest.java deleted file mode 100644 index bb1411a86e3af..0000000000000 --- a/src/test/java/flink/graphs/GraphTest.java +++ /dev/null @@ -1,212 +0,0 @@ -package flink.graphs; - - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.util.Collector; -import org.junit.Before; -import org.junit.Test; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; - -import static org.junit.Assert.*; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - - -public class GraphTest implements Serializable{ - - // Assume existing graph object - // Tuple2 ids and values: 0,1,2,3 - // Edges: 0->1, 1->3, 0->3, 1->2 - - static Graph graph; - static ExecutionEnvironment env; - - @Before - public void testSetUp() { - env = ExecutionEnvironment.getExecutionEnvironment(); - setUpGraph(); - } - - - public static void setUpGraph() { - - List> Tuple2List = new ArrayList>(); - - for (int i = 0; i < 4; i++) { - Tuple2 v = new Tuple2(i, i); - Tuple2List.add(v); - } - - - List> edgeList = new ArrayList<>(); - - edgeList.add(new Tuple3(0, 1, 0)); - edgeList.add(new Tuple3(1, 3, 0)); - edgeList.add(new Tuple3(0, 3, 0)); - edgeList.add(new Tuple3(1, 2, 0)); - - DataSet> vertices = env.fromCollection(Tuple2List); - DataSet> edges = env.fromCollection(edgeList); - - graph = new Graph(vertices, edges); - } - - @Test - public void testCreate() throws Exception { - - List> Tuple2List = new ArrayList>(); - - for (int i = 0; i < 4; i++) { - Tuple2 v = new Tuple2(i, i); - Tuple2List.add(v); - } - - - List> edgeList = new ArrayList<>(); - - edgeList.add(new Tuple3(0, 1, 0)); - edgeList.add(new Tuple3(1, 3, 0)); - edgeList.add(new Tuple3(0, 3, 0)); - edgeList.add(new Tuple3(1, 2, 0)); - - DataSet> vertices = env.fromCollection(Tuple2List); - DataSet> edges = env.fromCollection(edgeList); - - Graph g = Graph.create(vertices, edges); - - g.getVertices().print(); - - env.execute(); - } - - @Test - public void testGetVertices() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testGetEdges() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testMapVertices() throws Exception { - - DataSet> doubled= graph.mapVertices(new MapFunction() { - @Override - public Integer map(Integer value) throws Exception { - return value * 2; - } - }); - - // Compare the two Datasets as lists? - - List> doubledData = new ArrayList<>(); - doubled.output(new LocalCollectionOutputFormat<>(doubledData)); - - - - DataSet> doubledDataset = graph.getVertices() - .map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple2 v) throws Exception { - return new Tuple2(v.f0, v.f1 * 2); - } - }); - List> originalDataDoubled = new ArrayList<>(); - doubledDataset.output(new LocalCollectionOutputFormat<>(originalDataDoubled)); - - assertEquals(doubledData, originalDataDoubled); - - // TODO(thvasilo): Test for function that changes the type of the value - - doubled.print(); - graph.getVertices().print(); - - env.execute(); - - } - - @Test - public void testSubgraph() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testPga() throws Exception { - // Test pga by running connected components - // Expected output is that all vertices end up with the same attribute, 0 - - // Send the vertex attribute to all neighbors - CoGroupFunction, Tuple3, Tuple2> - sendAttribute = - new CoGroupFunction, Tuple3, Tuple2>() { - @Override - public void coGroup(Iterable> vertices, - Iterable> edges, - Collector> tuple2Collector) throws Exception { - for (Tuple2 vertex : vertices) { - for (Tuple3 edge: edges) { - tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); - } - } - } - }; - - // Gather all messages and keep the message with the smallest attribute - GroupReduceFunction, Tuple2> - gatherAttributes = - new GroupReduceFunction, Tuple2>() { - @Override - public void reduce(Iterable> messages, - Collector> msgCollector) throws Exception { - - Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); - for (Tuple2 message : messages) { - if (message.f1 < minTuple.f1) { - minTuple = message.copy(); - } - } - msgCollector.collect(minTuple); - } - }; - - // Check if the produced message is smaller than the current vertex attribute, if yes change attribute - FlatJoinFunction, Tuple2, Tuple2> - apply = - new FlatJoinFunction, Tuple2, Tuple2>() { - @Override - public void join(Tuple2 msg, - Tuple2 vertex, - Collector> vertexCollector) throws Exception { - if (msg.f1 < vertex.f1) { - vertexCollector.collect(msg.copy()); - } - } - }; - - - // Run the pga iterations - Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); - - DataSet> conVerts = connected.getVertices(); - - // All vertices should end up with attribute 0 - conVerts.print(); - //TODO(thvasilo): Automate correctness testing - - env.execute(); - - } - -} \ No newline at end of file From c3234c4d875c2ed4d38bbe771e09d2fe7a28b1b4 Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 11 Nov 2014 17:18:44 +0100 Subject: [PATCH 029/112] added execution environment to create calls in test --- pom.xml | 10 +++++----- src/test/java/flink/graphs/TestGraphOperations.java | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pom.xml b/pom.xml index 261701f463499..95ff80c54f29f 100644 --- a/pom.xml +++ b/pom.xml @@ -49,27 +49,27 @@ under the License. org.apache.flink flink-java - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-clients - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-spargel - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-avro - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT org.apache.flink flink-test-utils - 0.7-incubating-SNAPSHOT + 0.8-incubating-SNAPSHOT junit diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index a4aba1c94aa03..e856c0434b47e 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -71,7 +71,7 @@ public static String runProgram(int progId, String resultPath) throws Exception final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); graph.getUndirected().getEdges().writeAsCsv(resultPath); env.execute(); @@ -90,7 +90,7 @@ public static String runProgram(int progId, String resultPath) throws Exception final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); graph.reverse().getEdges().writeAsCsv(resultPath); env.execute(); @@ -109,7 +109,7 @@ public static String runProgram(int progId, String resultPath) throws Exception final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); graph.outDegrees().writeAsCsv(resultPath); env.execute(); @@ -126,7 +126,7 @@ public static String runProgram(int progId, String resultPath) throws Exception final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); DataSet> mappedVertices = graph.mapVertices(new MapFunction() { public Long map(Long value) throws Exception { @@ -149,7 +149,7 @@ public Long map(Long value) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); graph.subgraph(new FilterFunction() { public boolean filter(Long value) throws Exception { return (value > 2); From fb04d0b689354465748cfbd793255fda142c550e Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 11 Nov 2014 17:24:30 +0100 Subject: [PATCH 030/112] suppress warnings --- src/main/java/flink/graphs/Graph.java | 16 ++++++++-------- src/main/java/flink/graphs/GraphUtils.java | 3 ++- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 10d4a2874e788..19e3f9f64c590 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -18,7 +18,6 @@ package flink.graphs; - import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; @@ -347,20 +346,20 @@ public Tuple3 map(Tuple3 value) throws Exception { /** * @return Singleton DataSet containing the vertex count */ - public DataSet numberOfVertices () { + @SuppressWarnings("unchecked") + public DataSet numberOfVertices () { return GraphUtils.count((DataSet) (DataSet) vertices); - } /** * * @return Singleton DataSet containing the edge count */ - public DataSet numberOfEdges () { + @SuppressWarnings("unchecked") + public DataSet numberOfEdges () { return GraphUtils.count((DataSet) (DataSet) edges); } - /** * * @return The IDs of the vertices as DataSet @@ -383,7 +382,8 @@ public Tuple2 map(Tuple3 edge) throws Exception { }); } - public DataSet isWeaklyConnected () { + @SuppressWarnings("unchecked") + public DataSet isWeaklyConnected () { DataSet vertexIds = this.getVertexIds(); DataSet> verticesWithInitialIds = vertexIds @@ -451,6 +451,7 @@ public Graph addVertex (Tuple2 vertex, List> edg return this.union(newVertex); } + @SuppressWarnings("unchecked") public Graph removeVertex (Tuple2 vertex) { DataSet> vertexToRemove = fromCollection(Arrays.asList(vertex)); @@ -459,7 +460,7 @@ public Graph removeVertex (Tuple2 vertex) { .filter(new RichFilterFunction>() { private Tuple2 vertexToRemove; - @Override + @Override public void open(Configuration parameters) throws Exception { super.open(parameters); this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); @@ -512,7 +513,6 @@ public Graph union (Graph graph) { return new Graph(unionedVertices, unionedEdges, this.context); } - public Graph passMessages (VertexCentricIteration iteration) { DataSet> newVertices = iteration.createResult(); return new Graph(newVertices, edges, this.context); diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java index 3acde4feccc20..7c6cffa07d07b 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -7,7 +7,8 @@ public class GraphUtils { - public static DataSet count (DataSet set) { + @SuppressWarnings("serial") + public static DataSet count (DataSet set) { return set .map(new MapFunction() { @Override From 32cd60ec2bdd768c4fc6ee51297cd2d7e36086e1 Mon Sep 17 00:00:00 2001 From: vasia Date: Wed, 12 Nov 2014 10:25:48 +0100 Subject: [PATCH 031/112] separate test for mapVertices --- .../flink/graphs/TestGraphOperations.java | 154 +----------- .../java/flink/graphs/TestMapVertices.java | 219 ++++++++++++++++++ 2 files changed, 221 insertions(+), 152 deletions(-) create mode 100644 src/test/java/flink/graphs/TestMapVertices.java diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 8e41483998e78..865a7ba0aeecb 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -6,24 +6,17 @@ import java.util.LinkedList; import org.apache.flink.api.common.functions.FilterFunction; -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.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; -import flink.graphs.TestGraphUtils.DummyCustomType; - @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; + private static int NUM_PROGRAMS = 4; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -122,151 +115,8 @@ public static String runProgram(int progId, String resultPath) throws Exception "3,2\n" + "4,1\n" + "5,1\n"; - } + } case 4: { - /* - * Test mapVertices() keeping the same value type - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); - - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public Long map(Long value) throws Exception { - return value+1; - } - }); - - mappedVertices.writeAsCsv(resultPath); - env.execute(); - return "1,2\n" + - "2,3\n" + - "3,4\n" + - "4,5\n" + - "5,6\n"; - } - case 5: { - /* - * Test mapVertices() and change the value type to String - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); - - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public String map(Long value) throws Exception { - String stringValue; - if (value == 1) { - stringValue = "one"; - } - else if (value == 2) { - stringValue = "two"; - } - else if (value == 3) { - stringValue = "three"; - } - else if (value == 4) { - stringValue = "four"; - } - else if (value == 5) { - stringValue = "five"; - } - else { - stringValue = ""; - } - - return stringValue; - } - }); - - mappedVertices.writeAsCsv(resultPath); - env.execute(); - return "1,one\n" + - "2,two\n" + - "3,three\n" + - "4,four\n" + - "5,five\n"; - } - case 6: { - /* - * Test mapVertices() and change the value type to a Tuple1 - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); - - DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { - public Tuple1 map(Long value) throws Exception { - Tuple1 tupleValue = new Tuple1(); - tupleValue.setFields(value); - return tupleValue; - } - }); - - mappedVertices.writeAsCsv(resultPath); - env.execute(); - return "1,(1)\n" + - "2,(2)\n" + - "3,(3)\n" + - "4,(4)\n" + - "5,(5)\n"; - } - case 7: { - /* - * Test mapVertices() and change the value type to a custom type - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); - - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public DummyCustomType map(Long value) throws Exception { - DummyCustomType dummyValue = new DummyCustomType(); - dummyValue.setIntField(value.intValue()); - return dummyValue; - } - }); - - mappedVertices.writeAsCsv(resultPath); - env.execute(); - return "1,(T,1)\n" + - "2,(T,2)\n" + - "3,(T,3)\n" + - "4,(T,4)\n" + - "5,(T,5)\n"; - } - case 8: { - /* - * Test mapVertices() and change the value type to a parameterized custom type - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); - - DataSet>> mappedVertices = graph.mapVertices( - new MapFunction>() { - public DummyCustomParameterizedType map(Long value) throws Exception { - DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); - dummyValue.setIntField(value.intValue()); - dummyValue.setTField(new Double(value)); - return dummyValue; - } - }); - - mappedVertices.writeAsCsv(resultPath); - env.execute(); - return "1,(1.0,1)\n" + - "2,(2.0,2)\n" + - "3,(3.0,3)\n" + - "4,(4.0,4)\n" + - "5,(5.0,5)\n"; - } - case 9: { /* * Test subgraph: */ diff --git a/src/test/java/flink/graphs/TestMapVertices.java b/src/test/java/flink/graphs/TestMapVertices.java new file mode 100644 index 0000000000000..1b2a716102e3d --- /dev/null +++ b/src/test/java/flink/graphs/TestMapVertices.java @@ -0,0 +1,219 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +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.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; +import flink.graphs.TestGraphUtils.DummyCustomType; + +@RunWith(Parameterized.class) +public class TestMapVertices extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 5; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestMapVertices(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test mapVertices() keeping the same value type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public Long map(Long value) throws Exception { + return value+1; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,6\n"; + } + case 2: { + /* + * Test mapVertices() and change the value type to String + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public String map(Long value) throws Exception { + String stringValue; + if (value == 1) { + stringValue = "one"; + } + else if (value == 2) { + stringValue = "two"; + } + else if (value == 3) { + stringValue = "three"; + } + else if (value == 4) { + stringValue = "four"; + } + else if (value == 5) { + stringValue = "five"; + } + else { + stringValue = ""; + } + + return stringValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,one\n" + + "2,two\n" + + "3,three\n" + + "4,four\n" + + "5,five\n"; + } + case 3: { + /* + * Test mapVertices() and change the value type to a Tuple1 + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { + public Tuple1 map(Long value) throws Exception { + Tuple1 tupleValue = new Tuple1(); + tupleValue.setFields(value); + return tupleValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(1)\n" + + "2,(2)\n" + + "3,(3)\n" + + "4,(4)\n" + + "5,(5)\n"; + } + case 4: { + /* + * Test mapVertices() and change the value type to a custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + public DummyCustomType map(Long value) throws Exception { + DummyCustomType dummyValue = new DummyCustomType(); + dummyValue.setIntField(value.intValue()); + return dummyValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(T,1)\n" + + "2,(T,2)\n" + + "3,(T,3)\n" + + "4,(T,4)\n" + + "5,(T,5)\n"; + } + case 5: { + /* + * Test mapVertices() and change the value type to a parameterized custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedVertices = graph.mapVertices( + new MapFunction>() { + public DummyCustomParameterizedType map(Long value) throws Exception { + DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); + dummyValue.setIntField(value.intValue()); + dummyValue.setTField(new Double(value)); + return dummyValue; + } + }); + + mappedVertices.writeAsCsv(resultPath); + env.execute(); + return "1,(1.0,1)\n" + + "2,(2.0,2)\n" + + "3,(3.0,3)\n" + + "4,(4.0,4)\n" + + "5,(5.0,5)\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} From de2eed25bfb50e693a73d88e3901b3eff5a278da Mon Sep 17 00:00:00 2001 From: balidani Date: Wed, 12 Nov 2014 15:21:32 +0100 Subject: [PATCH 032/112] Added mapEdges --- src/main/java/flink/graphs/Graph.java | 43 ++++++++++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index d92cba51aeabf..523212447a8fe 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -48,16 +48,22 @@ public class Graph & Serializable, VV extends Serializab /** a graph is directed by default */ private boolean isUndirected = false; - + private static TypeInformation vertexKeyType; private static TypeInformation vertexValueType; + private static TypeInformation edgeKeyType; + private static TypeInformation edgeValueType; + public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + + Graph.edgeKeyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { @@ -66,6 +72,9 @@ public Graph(DataSet> vertices, DataSet> edges, b this.isUndirected = undirected; Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); + + Graph.edgeKeyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } public DataSet> getVertices() { @@ -107,6 +116,38 @@ public TypeInformation> getProducedType() { return new TupleTypeInfo>(vertexKeyType, newVertexValueType); } } + + /** + * Apply a function to the attribute of each edge in the graph. + * @param mapper + * @return + */ + public DataSet> mapEdges(final MapFunction mapper) { + return edges.map(new ApplyMapperToEdgeWithType(mapper)); + } + + private static final class ApplyMapperToEdgeWithType implements MapFunction + , Tuple3>, ResultTypeQueryable> { + + private MapFunction innerMapper; + + public ApplyMapperToEdgeWithType(MapFunction theMapper) { + this.innerMapper = theMapper; + } + + public Tuple3 map(Tuple3 value) throws Exception { + return new Tuple3(value.f0, value.f1, innerMapper.map(value.f2)); + } + + @Override + public TypeInformation> getProducedType() { + @SuppressWarnings("unchecked") + TypeInformation newEdgeValueType = TypeExtractor.getMapReturnTypes(innerMapper, + (TypeInformation)edgeValueType); + + return new TupleTypeInfo>(edgeKeyType, edgeKeyType, newEdgeValueType); + } + } /** * Apply value-based filtering functions to the graph From 07f72668c8df76b7f63a56bff085e859fbd37882 Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Wed, 12 Nov 2014 15:31:32 +0100 Subject: [PATCH 033/112] Added mapEdges tests --- src/test/java/flink/graphs/TestMapEdges.java | 209 +++++++++++++++++++ 1 file changed, 209 insertions(+) create mode 100644 src/test/java/flink/graphs/TestMapEdges.java diff --git a/src/test/java/flink/graphs/TestMapEdges.java b/src/test/java/flink/graphs/TestMapEdges.java new file mode 100644 index 0000000000000..d393bc01f10eb --- /dev/null +++ b/src/test/java/flink/graphs/TestMapEdges.java @@ -0,0 +1,209 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +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.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; +import flink.graphs.TestGraphUtils.DummyCustomType; + +@RunWith(Parameterized.class) +public class TestMapEdges extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 5; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestMapEdges(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test mapEdges() keeping the same value type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + public Long map(Long value) throws Exception { + return value+1; + } + }); + + mappedEdges.writeAsCsv(resultPath); + env.execute(); + return "1,2,13\n" + + "1,3,14\n" + + "2,3,24\n" + + "3,4,35\n" + + "3,5,36\n" + + "4,5,46\n" + + "5,1,52\n"; + } + case 2: { + /* + * Test mapEdges() and change the value type to String + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + public String map(Long value) throws Exception { + return String.format("string(%d)", value); + } + }); + + mappedEdges.writeAsCsv(resultPath); + env.execute(); + return "1,2,string(12)\n" + + "1,3,string(13)\n" + + "2,3,string(23)\n" + + "3,4,string(34)\n" + + "3,5,string(35)\n" + + "4,5,string(45)\n" + + "5,1,string(51)\n"; + } + case 3: { + /* + * Test mapEdges() and change the value type to a Tuple1 + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedEdges = graph.mapEdges(new MapFunction>() { + public Tuple1 map(Long value) throws Exception { + Tuple1 tupleValue = new Tuple1(); + tupleValue.setFields(value); + return tupleValue; + } + }); + + mappedEdges.writeAsCsv(resultPath); + env.execute(); + return "1,2,(12)\n" + + "1,3,(13)\n" + + "2,3,(23)\n" + + "3,4,(34)\n" + + "3,5,(35)\n" + + "4,5,(45)\n" + + "5,1,(51)\n"; + } + case 4: { + /* + * Test mapEdges() and change the value type to a custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + public DummyCustomType map(Long value) throws Exception { + DummyCustomType dummyValue = new DummyCustomType(); + dummyValue.setIntField(value.intValue()); + return dummyValue; + } + }); + + mappedEdges.writeAsCsv(resultPath); + env.execute(); + return "1,2,(T,12)\n" + + "1,3,(T,13)\n" + + "2,3,(T,23)\n" + + "3,4,(T,34)\n" + + "3,5,(T,35)\n" + + "4,5,(T,45)\n" + + "5,1,(T,51)\n"; + } + case 5: { + /* + * Test mapEdges() and change the value type to a parameterized custom type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet>> mappedEdges = graph.mapEdges( + new MapFunction>() { + public DummyCustomParameterizedType map(Long value) throws Exception { + DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); + dummyValue.setIntField(value.intValue()); + dummyValue.setTField(new Double(value)); + return dummyValue; + } + }); + + mappedEdges.writeAsCsv(resultPath); + env.execute(); + return "1,2,(12.0,12)\n" + + "1,3,(13.0,13)\n" + + "2,3,(23.0,23)\n" + + "3,4,(34.0,34)\n" + + "3,5,(35.0,35)\n" + + "4,5,(45.0,45)\n" + + "5,1,(51.0,51)\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} From 4196c7a1792f7c075633988c8d1fa416633919f3 Mon Sep 17 00:00:00 2001 From: balidani Date: Wed, 12 Nov 2014 19:09:45 +0100 Subject: [PATCH 034/112] Removed unnecessary edgeKeyType field --- src/main/java/flink/graphs/Graph.java | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 523212447a8fe..7ff660c5858a1 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -49,20 +49,17 @@ public class Graph & Serializable, VV extends Serializab /** a graph is directed by default */ private boolean isUndirected = false; - private static TypeInformation vertexKeyType; + private static TypeInformation keyType; private static TypeInformation vertexValueType; - - private static TypeInformation edgeKeyType; private static TypeInformation edgeValueType; public Graph(DataSet> vertices, DataSet> edges) { this.vertices = vertices; this.edges = edges; - Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + + Graph.keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - - Graph.edgeKeyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } @@ -70,10 +67,9 @@ public Graph(DataSet> vertices, DataSet> edges, b this.vertices = vertices; this.edges = edges; this.isUndirected = undirected; - Graph.vertexKeyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + + Graph.keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - - Graph.edgeKeyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } @@ -113,7 +109,7 @@ public TypeInformation> getProducedType() { TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, (TypeInformation)vertexValueType); - return new TupleTypeInfo>(vertexKeyType, newVertexValueType); + return new TupleTypeInfo>(keyType, newVertexValueType); } } @@ -145,7 +141,7 @@ public TypeInformation> getProducedType() { TypeInformation newEdgeValueType = TypeExtractor.getMapReturnTypes(innerMapper, (TypeInformation)edgeValueType); - return new TupleTypeInfo>(edgeKeyType, edgeKeyType, newEdgeValueType); + return new TupleTypeInfo>(keyType, keyType, newEdgeValueType); } } From 712e06c52b1414ccf2c5691c453c7baff11fe11d Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 13 Nov 2014 11:19:26 +0100 Subject: [PATCH 035/112] empty test todos for implemented operations --- .../flink/graphs/TestGraphOperations.java | 60 +++++++++ src/test/java/flink/graphs/TestPGA.java | 122 ++++++++++++++++++ 2 files changed, 182 insertions(+) create mode 100644 src/test/java/flink/graphs/TestPGA.java diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 6824d6a25dcb9..69a44a712abde 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -138,6 +138,66 @@ public boolean filter(Long value) throws Exception { env.execute(); return "3,5,35\n" + "4,5,45\n"; + } + /* + * Test numberOfVertices() + */ + case 5: { + + } + /* + * Test numberOfEdges() + */ + case 6: { + + } + /* + * Test getVertexIds() + */ + case 7: { + + } + /* + * Test getEdgeIds() + */ + case 8: { + + } + /* + * Test isWeaklyConnected() + */ + case 9: { + + } + /* + * Test addVertex() + */ + case 10: { + + } + /* + * Test removeVertex() + */ + case 11: { + + } + /* + * Test addEdge() + */ + case 12: { + + } + /* + * Test union() + */ + case 13: { + + } + /* + * Test passMessages() + */ + case 14: { + } default: throw new IllegalArgumentException("Invalid program id"); diff --git a/src/test/java/flink/graphs/TestPGA.java b/src/test/java/flink/graphs/TestPGA.java new file mode 100644 index 0000000000000..210c5e4b2e0bd --- /dev/null +++ b/src/test/java/flink/graphs/TestPGA.java @@ -0,0 +1,122 @@ +package flink.graphs; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +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.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.util.Collector; +import org.junit.Before; +import org.junit.Test; + +public class TestPGA { + + // Assume existing graph object + // Tuple2 ids and values: 0,1,2,3 + // Edges: 0->1, 1->3, 0->3, 1->2 + + static Graph graph; + static ExecutionEnvironment env; + + @Before + public void testSetUp() { + env = ExecutionEnvironment.getExecutionEnvironment(); + setUpGraph(); + } + + + public static void setUpGraph() { + + List> Tuple2List = new ArrayList>(); + + for (int i = 0; i < 4; i++) { + Tuple2 v = new Tuple2(i, i); + Tuple2List.add(v); + } + + + List> edgeList = new ArrayList<>(); + + edgeList.add(new Tuple3(0, 1, 0)); + edgeList.add(new Tuple3(1, 3, 0)); + edgeList.add(new Tuple3(0, 3, 0)); + edgeList.add(new Tuple3(1, 2, 0)); + + DataSet> vertices = env.fromCollection(Tuple2List); + DataSet> edges = env.fromCollection(edgeList); + + graph = new Graph(vertices, edges, env); + } + @SuppressWarnings("serial") + @Test + public void testPga() throws Exception { + // Test pga by running connected components + // Expected output is that all vertices end up with the same attribute, 0 + + // Send the vertex attribute to all neighbors + CoGroupFunction, Tuple3, Tuple2> + sendAttribute = + new CoGroupFunction, Tuple3, Tuple2>() { + @Override + public void coGroup(Iterable> vertices, + Iterable> edges, + Collector> tuple2Collector) throws Exception { + for (Tuple2 vertex : vertices) { + for (Tuple3 edge: edges) { + tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); + } + } + } + }; + + // Gather all messages and keep the message with the smallest attribute + GroupReduceFunction, Tuple2> + gatherAttributes = + new GroupReduceFunction, Tuple2>() { + @Override + public void reduce(Iterable> messages, + Collector> msgCollector) throws Exception { + + Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); + for (Tuple2 message : messages) { + if (message.f1 < minTuple.f1) { + minTuple = message.copy(); + } + } + msgCollector.collect(minTuple); + } + }; + + // Check if the produced message is smaller than the current vertex attribute, if yes change attribute + FlatJoinFunction, Tuple2, Tuple2> + apply = + new FlatJoinFunction, Tuple2, Tuple2>() { + @Override + public void join(Tuple2 msg, + Tuple2 vertex, + Collector> vertexCollector) throws Exception { + if (msg.f1 < vertex.f1) { + vertexCollector.collect(msg.copy()); + } + } + }; + + + // Run the pga iterations + Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); + + DataSet> conVerts = connected.getVertices(); + + // All vertices should end up with attribute 0 + conVerts.print(); + //TODO(thvasilo): Automate correctness testing + + env.execute(); + + } +} From d9f5d1620fefd79bd251d439e4b2be07c1708e5f Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 13 Nov 2014 12:15:04 +0100 Subject: [PATCH 036/112] fix count to work with Datasets of tuples, added test for numberOfVertices, numberOfEdges --- src/main/java/flink/graphs/Graph.java | 10 +++--- src/main/java/flink/graphs/GraphUtils.java | 35 +++++++++++-------- .../flink/graphs/TestGraphOperations.java | 24 ++++++++++--- 3 files changed, 44 insertions(+), 25 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 10e6a7a89ae4b..60d0810bea0c7 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -45,7 +45,7 @@ @SuppressWarnings("serial") public class Graph & Serializable, VV extends Serializable, - EV extends Serializable> implements Serializable{ + EV extends Serializable> implements Serializable { private final ExecutionEnvironment context; @@ -365,18 +365,18 @@ public Tuple3 map(Tuple3 value) throws Exception { /** * @return Singleton DataSet containing the vertex count */ - @SuppressWarnings("unchecked") +// @SuppressWarnings("unchecked") public DataSet numberOfVertices () { - return GraphUtils.count((DataSet) (DataSet) vertices); + return GraphUtils.count(vertices); } /** * * @return Singleton DataSet containing the edge count */ - @SuppressWarnings("unchecked") +// @SuppressWarnings("unchecked") public DataSet numberOfEdges () { - return GraphUtils.count((DataSet) (DataSet) edges); + return GraphUtils.count(edges); } /** diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java index 7c6cffa07d07b..0273e44db7925 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -1,28 +1,33 @@ package flink.graphs; - import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple; +@SuppressWarnings("serial") public class GraphUtils { - @SuppressWarnings("serial") - public static DataSet count (DataSet set) { + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static DataSet count(DataSet set) { return set - .map(new MapFunction() { - @Override - public Integer map(Object o) throws Exception { - return 1; - } - }) - .reduce(new ReduceFunction() { - @Override - public Integer reduce(Integer one, Integer two) throws Exception { - return one + two; - } - }) + .map(new OneMapper()) + .reduce(new AddOnesReducer()) .first(1); } + private static final class OneMapper implements MapFunction { + @Override + public Integer map(T o) throws Exception { + return 1; + } + } + + private static final class AddOnesReducer implements ReduceFunction { + @Override + public Integer reduce(Integer one, Integer two) throws Exception { + return one + two; + } + } + } diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 69a44a712abde..338c2e74b9fa1 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -16,7 +16,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 4; + private static int NUM_PROGRAMS = 6; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -139,17 +139,31 @@ public boolean filter(Long value) throws Exception { return "3,5,35\n" + "4,5,45\n"; } - /* - * Test numberOfVertices() - */ case 5: { + /* + * Test numberOfVertices() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfVertices().writeAsText(resultPath); + env.execute(); + return "5"; } + case 6: { /* * Test numberOfEdges() */ - case 6: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfEdges().writeAsText(resultPath); + + env.execute(); + return "7"; } /* * Test getVertexIds() From c7258a7da94c4d577e54bec24817a7fcd7b5edf6 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 13 Nov 2014 16:30:55 +0100 Subject: [PATCH 037/112] static inner classes for getVertexIds, getEdgeIds, isWeaklyConnected, tests for former 2 --- src/main/java/flink/graphs/Graph.java | 106 ++++++++++-------- .../flink/graphs/TestGraphOperations.java | 72 ++++++------ 2 files changed, 101 insertions(+), 77 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 60d0810bea0c7..b205eb4e21fb0 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -365,7 +365,6 @@ public Tuple3 map(Tuple3 value) throws Exception { /** * @return Singleton DataSet containing the vertex count */ -// @SuppressWarnings("unchecked") public DataSet numberOfVertices () { return GraphUtils.count(vertices); } @@ -374,7 +373,6 @@ public DataSet numberOfVertices () { * * @return Singleton DataSet containing the edge count */ -// @SuppressWarnings("unchecked") public DataSet numberOfEdges () { return GraphUtils.count(edges); } @@ -384,76 +382,98 @@ public DataSet numberOfEdges () { * @return The IDs of the vertices as DataSet */ public DataSet getVertexIds () { - return vertices.map(new MapFunction, K>() { + return vertices.map(new ExtractVertexIDMapper()); + } + + private static final class ExtractVertexIDMapper implements MapFunction, K> { @Override public K map(Tuple2 vertex) throws Exception { return vertex.f0; } - }); } - public DataSet> getEdgeIds () { - return edges.map(new MapFunction, Tuple2>() { + public DataSet> getEdgeIds () { + return edges.map(new ExtractEdgeIDsMapper()); + } + + private static final class ExtractEdgeIDsMapper implements MapFunction, Tuple2> { @Override public Tuple2 map(Tuple3 edge) throws Exception { return new Tuple2(edge.f0, edge.f1); } - }); } - @SuppressWarnings("unchecked") + /** + * Checks the weak connectivity of a graph. + * @return true if the graph is weakly connected. + */ public DataSet isWeaklyConnected () { + Graph graph; + + if (!(this.isUndirected)) { + // first, convert to an undirected graph + graph = this.getUndirected(); + } + else { + graph = this; + } - DataSet vertexIds = this.getVertexIds(); + DataSet vertexIds = graph.getVertexIds(); DataSet> verticesWithInitialIds = vertexIds - .map(new MapFunction>() { - @Override - public Tuple2 map(K k) throws Exception { - return new Tuple2(k, k); - } - }); + .map(new DuplicateVertexIDMapper()); - DataSet> edgeIds = this.getEdgeIds(); + DataSet> edgeIds = graph.getEdgeIds(); DeltaIteration, Tuple2> iteration = verticesWithInitialIds - .iterateDelta(verticesWithInitialIds, 10, 0); + .iterateDelta(verticesWithInitialIds, 0); DataSet> changes = iteration.getWorkset() .join(edgeIds).where(0).equalTo(0) - .with(new JoinFunction, Tuple2, Tuple2>() { - @Override - public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) throws Exception { - return new Tuple2(edge.f1, vertexWithComponent.f1); - } - }) + .with(new FindNeighborsJoin()) .groupBy(0) .aggregate(Aggregations.MIN, 1) .join(iteration.getSolutionSet()).where(0).equalTo(0) - .with(new FlatJoinFunction, Tuple2, Tuple2>() { - @Override - public void join(Tuple2 candidate, Tuple2 old, Collector> out) throws Exception { - if (candidate.f1.compareTo(old.f1) < 0) { - out.collect(candidate); - } - } - }); + .with(new VertexWithNewComponentJoin()); DataSet> components = iteration.closeWith(changes, changes); - DataSet result = GraphUtils.count((DataSet) (DataSet) components) - .map(new MapFunction() { - @Override - public Boolean map(Integer n) throws Exception { - if (n == 1) - return false; - else - return true; - } - }); - + DataSet result = GraphUtils.count(components) + .map(new CheckIfOneComponentMapper()); return result; } + + private static final class DuplicateVertexIDMapper implements MapFunction> { + @Override + public Tuple2 map(K k) { + return new Tuple2(k, k); + } + } + + private static final class FindNeighborsJoin implements JoinFunction, Tuple2, + Tuple2> { + @Override + public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) { + return new Tuple2(edge.f1, vertexWithComponent.f1); + } + } + private static final class VertexWithNewComponentJoin> + implements FlatJoinFunction, Tuple2, Tuple2> { + @Override + public void join(Tuple2 candidate, Tuple2 old, Collector> out) { + if (candidate.f1.compareTo(old.f1) < 0) { + out.collect(candidate); + } + } + } + + private static final class CheckIfOneComponentMapper implements MapFunction { + @Override + public Boolean map(Integer n) { + return (!(n == 1)); + } + } + //TODO kostas add functionality public Graph fromCollection (Collection> vertices, Collection> edges) { return null; @@ -536,6 +556,4 @@ public Graph passMessages (VertexCentricIteration itera DataSet> newVertices = iteration.createResult(); return new Graph(newVertices, edges, this.context); } - - } diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 338c2e74b9fa1..2388230d2e642 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -16,7 +16,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 6; + private static int NUM_PROGRAMS = 8; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -165,53 +165,59 @@ public boolean filter(Long value) throws Exception { env.execute(); return "7"; } - /* - * Test getVertexIds() - */ case 7: { + /* + * Test getVertexIds() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getVertexIds().writeAsText(resultPath); + + env.execute(); + return "1\n2\n3\n4\n5\n"; } - /* - * Test getEdgeIds() - */ case 8: { + /* + * Test getEdgeIds() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getEdgeIds().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + "1,3\n" + + "2,3\n" + "3,4\n" + + "3,5\n" + "4,5\n" + + "5,1\n"; } - /* - * Test isWeaklyConnected() - */ case 9: { - + /* + * Test addVertex() + */ } - /* - * Test addVertex() - */ case 10: { - + /* + * Test removeVertex() + */ } - /* - * Test removeVertex() - */ case 11: { - + /* + * Test addEdge() + */ } - /* - * Test addEdge() - */ case 12: { - + /* + * Test union() + */ } - /* - * Test union() - */ case 13: { - - } - /* - * Test passMessages() - */ - case 14: { - + /* + * Test passMessages() + */ } default: throw new IllegalArgumentException("Invalid program id"); From 3ac323f37b1c31e5d9919dc363335a2e70f26c21 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 13 Nov 2014 17:27:44 +0100 Subject: [PATCH 038/112] isWeakly connected corrections and test --- src/main/java/flink/graphs/Graph.java | 19 ++- .../java/flink/graphs/TestGraphUtils.java | 12 ++ .../flink/graphs/TestWeaklyConnected.java | 124 ++++++++++++++++++ 3 files changed, 149 insertions(+), 6 deletions(-) create mode 100644 src/test/java/flink/graphs/TestWeaklyConnected.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index b205eb4e21fb0..a4aeaff47bbd6 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -405,9 +405,10 @@ public Tuple2 map(Tuple3 edge) throws Exception { /** * Checks the weak connectivity of a graph. + * @param maxIterations the maximum number of iterations for the inner delta iteration * @return true if the graph is weakly connected. */ - public DataSet isWeaklyConnected () { + public DataSet isWeaklyConnected (int maxIterations) { Graph graph; if (!(this.isUndirected)) { @@ -425,7 +426,7 @@ public DataSet isWeaklyConnected () { DataSet> edgeIds = graph.getEdgeIds(); DeltaIteration, Tuple2> iteration = verticesWithInitialIds - .iterateDelta(verticesWithInitialIds, 0); + .iterateDelta(verticesWithInitialIds, maxIterations, 0); DataSet> changes = iteration.getWorkset() .join(edgeIds).where(0).equalTo(0) @@ -436,9 +437,8 @@ public DataSet isWeaklyConnected () { .with(new VertexWithNewComponentJoin()); DataSet> components = iteration.closeWith(changes, changes); - - DataSet result = GraphUtils.count(components) - .map(new CheckIfOneComponentMapper()); + DataSet result = GraphUtils.count(components.groupBy(1).reduceGroup( + new EmitFirstReducer())).map(new CheckIfOneComponentMapper()); return result; } @@ -467,10 +467,17 @@ public void join(Tuple2 candidate, Tuple2 old, Collector implements + GroupReduceFunction, Tuple2> { + public void reduce(Iterable> values, Collector> out) { + out.collect(values.iterator().next()); + } + } + private static final class CheckIfOneComponentMapper implements MapFunction { @Override public Boolean map(Integer n) { - return (!(n == 1)); + return (n == 1); } } diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 8bbdc121547dd..a4d50c26efa93 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -37,6 +37,18 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } + public static final DataSet> getDisconnectedLongLongEdgeData( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 2L, 12L)); + edges.add(new Tuple3(1L, 3L, 13L)); + edges.add(new Tuple3(2L, 3L, 23L)); + edges.add(new Tuple3(4L, 5L, 45L)); + + return env.fromCollection(edges); + } + + public static class DummyCustomType implements Serializable { private static final long serialVersionUID = 1L; diff --git a/src/test/java/flink/graphs/TestWeaklyConnected.java b/src/test/java/flink/graphs/TestWeaklyConnected.java new file mode 100644 index 0000000000000..3696458139893 --- /dev/null +++ b/src/test/java/flink/graphs/TestWeaklyConnected.java @@ -0,0 +1,124 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestWeaklyConnected extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 4; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestWeaklyConnected(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test isWeaklyConnected() with a connected, directed graph + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.isWeaklyConnected(10).writeAsText(resultPath); + + env.execute(); + return "true\n"; + } + case 2: { + /* + * Test isWeaklyConnected() with a disconnected, directed graph + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getDisconnectedLongLongEdgeData(env), env); + + graph.isWeaklyConnected(10).writeAsText(resultPath); + + env.execute(); + return "false\n"; + } + case 3: { + /* + * Test isWeaklyConnected() with a connected, undirected graph + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env).getUndirected(); + + graph.isWeaklyConnected(10).writeAsText(resultPath); + + env.execute(); + return "true\n"; + } + case 4: { + /* + * Test isWeaklyConnected() with a disconnected, undirected graph + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getDisconnectedLongLongEdgeData(env), env).getUndirected(); + + graph.isWeaklyConnected(10).writeAsText(resultPath); + + env.execute(); + return "false\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} From 092d32fa0b82ca4ba1d15257e5abe92eda903b12 Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Thu, 13 Nov 2014 21:11:27 +0100 Subject: [PATCH 039/112] Updated the Graph constructor --- src/main/java/flink/graphs/Graph.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 7ff660c5858a1..f217dd7d76722 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -46,8 +46,7 @@ public class Graph & Serializable, VV extends Serializab private final DataSet> edges; - /** a graph is directed by default */ - private boolean isUndirected = false; + private boolean isUndirected; private static TypeInformation keyType; private static TypeInformation vertexValueType; @@ -55,12 +54,9 @@ public class Graph & Serializable, VV extends Serializab public Graph(DataSet> vertices, DataSet> edges) { - this.vertices = vertices; - this.edges = edges; - - Graph.keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); + + /** a graph is directed by default */ + this(vertices, edges, false); } public Graph(DataSet> vertices, DataSet> edges, boolean undirected) { From 7b969df4b4d810c5f1b33d5f69e6c51a8ef18138 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?D=C3=A1niel=20Bali?= Date: Mon, 17 Nov 2014 18:06:06 +0100 Subject: [PATCH 040/112] Added mapEdges to the tested methods --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index bf9e022d8eac8..fa7f6f63215a6 100644 --- a/README.md +++ b/README.md @@ -32,6 +32,7 @@ Graph API for Apache Flink ##Tested Operations * mapVertices() +* mapEdges() * subGraph() * outDegrees() * getUndirected() @@ -42,7 +43,6 @@ Graph API for Apache Flink ###Graph Class * fromCollection(vertices, edges) * fromCollection(vertices) -* mapEdges() * getNeighborhoodGraph(Vertex src, int distance) * vertexCentricComputation() * edgeCentricComputation() From d485a8b9ef9abc98732a4434c10dbfc016f96d50 Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Thu, 20 Nov 2014 00:56:18 +0100 Subject: [PATCH 041/112] Added tests for addVertex, removeVertex, addEdge and union --- src/main/java/flink/graphs/Graph.java | 101 +++++++++-------- .../flink/graphs/TestGraphOperations.java | 102 +++++++++++++++++- 2 files changed, 151 insertions(+), 52 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index a4aeaff47bbd6..faf4d0b7ba149 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -481,9 +481,12 @@ public Boolean map(Integer n) { } } - //TODO kostas add functionality public Graph fromCollection (Collection> vertices, Collection> edges) { - return null; + + DataSet> v = context.fromCollection(vertices); + DataSet> e = context.fromCollection(edges); + + return new Graph(v, e, context); } //TODO kostas add functionality @@ -491,62 +494,66 @@ public DataSet> fromCollection (Collection> vertices) return null; } - public Graph addVertex (Tuple2 vertex, List> edges) { Graph newVertex = this.fromCollection(Arrays.asList(vertex), edges); return this.union(newVertex); } - @SuppressWarnings("unchecked") public Graph removeVertex (Tuple2 vertex) { - DataSet> vertexToRemove = fromCollection(Arrays.asList(vertex)); - - DataSet> newVertices = getVertices() - .filter(new RichFilterFunction>() { - private Tuple2 vertexToRemove; - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); - } - - @Override - public boolean filter(Tuple2 vertex) throws Exception { - if (vertex.f0.equals(vertexToRemove.f0)) { - return false; - } else { - return true; - } - } - }).withBroadcastSet(vertexToRemove, "vertexToRemove"); - - DataSet> newEdges = getEdges() - .filter(new RichFilterFunction>() { - private Tuple2 vertexToRemove; - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); - } - - @Override - public boolean filter(Tuple3 edge) throws Exception { - if (edge.f0.equals(vertexToRemove.f0)) { - return false; - } - if (edge.f1.equals(vertexToRemove.f0)) { - return false; - } - return true; - } - }).withBroadcastSet(vertexToRemove, "vertexToRemove"); + DataSet> vertexToRemove = context.fromCollection(Arrays.asList(vertex)); + + DataSet> newVertices = getVertices().filter( + new RemoveVertexFilter()).withBroadcastSet( + vertexToRemove, "vertexToRemove"); + + DataSet> newEdges = getEdges().filter( + new RemoveEdgeFilter()).withBroadcastSet( + vertexToRemove, "vertexToRemove"); return new Graph(newVertices, newEdges, this.context); } + + private static final class RemoveVertexFilter extends RichFilterFunction> { + + private Tuple2 vertexToRemove; + + @SuppressWarnings("unchecked") + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + } + + @Override + public boolean filter(Tuple2 vertex) throws Exception { + return !vertex.f0.equals(vertexToRemove.f0); + } + } + + private static final class RemoveEdgeFilter extends RichFilterFunction> { + private Tuple2 vertexToRemove; + + @SuppressWarnings("unchecked") + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + } + + @Override + public boolean filter(Tuple3 edge) throws Exception { + + if (edge.f0.equals(vertexToRemove.f0)) { + return false; + } + if (edge.f1.equals(vertexToRemove.f0)) { + return false; + } + return true; + } + } public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { Graph newEdges = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 2388230d2e642..ec955854b71c7 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -2,11 +2,15 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.LinkedList; +import java.util.List; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -153,9 +157,9 @@ public boolean filter(Long value) throws Exception { return "5"; } case 6: { - /* - * Test numberOfEdges() - */ + /* + * Test numberOfEdges() + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), @@ -198,21 +202,109 @@ public boolean filter(Long value) throws Exception { /* * Test addVertex() */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> edges = new ArrayList>(); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.addVertex(new Tuple2(6L, 6L), edges); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; + } case 10: { /* * Test removeVertex() */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeVertex(new Tuple2(5L, 5L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n"; } case 11: { /* * Test addEdge() - */ + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> vertices = new ArrayList>(); + List> edges = new ArrayList>(); + + vertices.add(new Tuple2(6L, 6L)); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.union(graph.fromCollection(vertices, edges)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; } case 12: { /* * Test union() - */ + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + + graph = graph.addEdge(new Tuple3(6L, 1L, 61L), + new Tuple2(6L, 6L), new Tuple2(1L, 1L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; } case 13: { /* From 144a79c86ff3cd6ee761fa805f65f8b34984e9c8 Mon Sep 17 00:00:00 2001 From: andralungu Date: Thu, 20 Nov 2014 13:03:54 +0100 Subject: [PATCH 042/112] fromCollection(vertices, edges) implemented and tested --- README.md | 4 +-- src/main/java/flink/graphs/Graph.java | 17 +++++++++ .../flink/graphs/TestGraphOperations.java | 20 ++++++++++- .../java/flink/graphs/TestGraphUtils.java | 35 ++++++++++++++++++- 4 files changed, 72 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index fa7f6f63215a6..c8f4a1e10ae29 100644 --- a/README.md +++ b/README.md @@ -37,12 +37,12 @@ Graph API for Apache Flink * outDegrees() * getUndirected() * reverse() +* fromCollection(vertices, edges) ##Wishlist ###Graph Class -* fromCollection(vertices, edges) -* fromCollection(vertices) +* fromCollection(edges) * getNeighborhoodGraph(Vertex src, int distance) * vertexCentricComputation() * edgeCentricComputation() diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index f217dd7d76722..39187e4dfa2be 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -36,6 +36,7 @@ import org.apache.flink.util.Collector; import java.io.Serializable; +import java.util.Collection; @SuppressWarnings("serial") @@ -384,4 +385,20 @@ public Tuple3 map(Tuple3 value) throws Exception { return Graph.create(vertices, edges); } + /** + * Creates a graph from the given vertex and edge collections + * @param env + * @param v the collection of vertices + * @param e the collection of edges + * @return a new graph formed from the set of edges and vertices + */ + public static & Serializable, VV extends Serializable, + EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, Collection> v, + Collection> e) throws Exception { + DataSet> vertices = env.fromCollection(v); + DataSet> edges = env.fromCollection(e); + + return Graph.create(vertices, edges); + } + } diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 865a7ba0aeecb..f12bbdfd493c3 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -16,7 +16,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 4; + private static int NUM_PROGRAMS = 5; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -139,6 +139,24 @@ public boolean filter(Long value) throws Exception { return "3,5,35\n" + "4,5,45\n"; } + case 5: { + /* + * Test fromCollection: + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.fromCollection(env, TestGraphUtils.getLongLongVertices(env), + TestGraphUtils.getLongLongEdges(env)); + + graph.getEdges().writeAsCsv(resultPath); + env.execute(); + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } default: throw new IllegalArgumentException("Invalid program id"); } diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 8bbdc121547dd..a2da77f8e4644 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -36,7 +36,40 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } - + + /** + * Function that produces an ArrayList of vertices + */ + public static final List> getLongLongVertices( + ExecutionEnvironment env) { + List> vertices = new ArrayList<>(); + vertices.add(new Tuple2(1L, 1L)); + vertices.add(new Tuple2(2L, 2L)); + vertices.add(new Tuple2(3L, 3L)); + vertices.add(new Tuple2(4L, 4L)); + vertices.add(new Tuple2(5L, 5L)); + + return vertices; + } + + /** + * Function that produces an ArrayList of edges + */ + public static final List> getLongLongEdges( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 2L, 12L)); + edges.add(new Tuple3(1L, 3L, 13L)); + edges.add(new Tuple3(2L, 3L, 23L)); + edges.add(new Tuple3(3L, 4L, 34L)); + edges.add(new Tuple3(3L, 5L, 35L)); + edges.add(new Tuple3(4L, 5L, 45L)); + edges.add(new Tuple3(5L, 1L, 51L)); + + return edges; + } + + public static class DummyCustomType implements Serializable { private static final long serialVersionUID = 1L; From 6aae9ac71bd9c400bfa97dc2cd2719f296529385 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 22 Nov 2014 12:26:39 +0100 Subject: [PATCH 043/112] added create methods from set of edges --- src/main/java/flink/graphs/Graph.java | 83 +++++++- .../java/flink/graphs/TestGraphCreation.java | 197 ++++++++++++++++++ 2 files changed, 278 insertions(+), 2 deletions(-) create mode 100644 src/test/java/flink/graphs/TestGraphCreation.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 39187e4dfa2be..519323c9d2da9 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; @@ -33,6 +34,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.types.NullValue; import org.apache.flink.util.Collector; import java.io.Serializable; @@ -289,11 +291,88 @@ public Graph reverse() throws UnsupportedOperationException { } } + /** + * Creates a graph from a dataset of vertices and a dataset of edges + * @param vertices + * @param edges + * @return + */ public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph - create(DataSet> vertices, DataSet> edges) { + EV extends Serializable> Graph create(DataSet> vertices, + DataSet> edges) { + return new Graph(vertices, edges); + } + + /** + * Creates a graph from a DataSet of edges. + * Vertices are created automatically and their values are set to NullValue. + * @param edges + * @return + */ + public static & Serializable, EV extends Serializable> + Graph create(DataSet> edges) { + DataSet> vertices = + edges.flatMap(new EmitSrcAndTarget()).distinct(); + return new Graph(vertices, edges); +} + + /** + * Creates a graph from a DataSet of edges. + * Vertices are created automatically and their values are set + * by applying the provided map function to the vertex ids. + * @param edges the input edges + * @param mapper the map function to set the initial vertex value + * @return + */ + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph create(DataSet> edges, final MapFunction mapper) { + DataSet> vertices = + edges.flatMap(new EmitSrcAndTargetAsTuple1()) + .distinct().map(new ApplyMapperToVertexValuesWithType(mapper)); return new Graph(vertices, edges); } + + private static final class ApplyMapperToVertexValuesWithType implements MapFunction + , Tuple2>, ResultTypeQueryable> { + + private MapFunction innerMapper; + + public ApplyMapperToVertexValuesWithType(MapFunction theMapper) { + this.innerMapper = theMapper; + } + + public Tuple2 map(Tuple1 value) throws Exception { + return new Tuple2(value.f0, innerMapper.map(value.f0)); + } + + @Override + public TypeInformation> getProducedType() { + @SuppressWarnings("unchecked") + TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, + (TypeInformation)keyType); + + return new TupleTypeInfo>(keyType, newVertexValueType); + } + } + + private static final class EmitSrcAndTarget & Serializable, EV extends Serializable> + implements FlatMapFunction, Tuple2> { + public void flatMap(Tuple3 edge, + Collector> out) { + + out.collect(new Tuple2(edge.f0, NullValue.getInstance())); + out.collect(new Tuple2(edge.f1, NullValue.getInstance())); + } + } + + private static final class EmitSrcAndTargetAsTuple1 & Serializable, + EV extends Serializable> implements FlatMapFunction, Tuple1> { + public void flatMap(Tuple3 edge, Collector> out) { + + out.collect(new Tuple1(edge.f0)); + out.collect(new Tuple1(edge.f1)); + } + } /** * Read and create the graph Tuple2 dataset from a csv file diff --git a/src/test/java/flink/graphs/TestGraphCreation.java b/src/test/java/flink/graphs/TestGraphCreation.java new file mode 100644 index 0000000000000..c099b22168817 --- /dev/null +++ b/src/test/java/flink/graphs/TestGraphCreation.java @@ -0,0 +1,197 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.types.NullValue; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; +import flink.graphs.TestGraphUtils.DummyCustomType; + +@RunWith(Parameterized.class) +public class TestGraphCreation extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 6; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestGraphCreation(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test create() with edge dataset and no vertex values + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env)); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(null)\n" + + "2,(null)\n" + + "3,(null)\n" + + "4,(null)\n" + + "5,(null)\n"; + } + case 2: { + /* + * Test create() with edge dataset and a mapper that assigns the id as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), + new MapFunction() { + public Long map(Long vertexId) { + return vertexId; + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,1\n" + + "2,2\n" + + "3,3\n" + + "4,4\n" + + "5,5\n"; + } + case 3: { + /* + * Test create() with edge dataset and a mapper that assigns a double constant as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), + new MapFunction() { + public Double map(Long value) { + return 0.1d; + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,0.1\n" + + "2,0.1\n" + + "3,0.1\n" + + "4,0.1\n" + + "5,0.1\n"; + } + case 4: { + /* + * Test create() with edge dataset and a mapper that assigns a Tuple2 as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph, Long> graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction>() { + public Tuple2 map(Long vertexId) { + return new Tuple2(vertexId*2, 42l); + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(2,42)\n" + + "2,(4,42)\n" + + "3,(6,42)\n" + + "4,(8,42)\n" + + "5,(10,42)\n"; + } + case 5: { + /* + * Test create() with edge dataset and a mapper that assigns a custom vertex value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction() { + public DummyCustomType map(Long vertexId) { + return new DummyCustomType(vertexId.intValue()-1, false); + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(F,0)\n" + + "2,(F,1)\n" + + "3,(F,2)\n" + + "4,(F,3)\n" + + "5,(F,4)\n"; + } + case 6: { + /* + * Test create() with edge dataset and a mapper that assigns a parametrized custom vertex value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph, Long> graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), + new MapFunction>() { + + DummyCustomParameterizedType dummyValue = + new DummyCustomParameterizedType(); + + public DummyCustomParameterizedType map(Long vertexId) { + dummyValue.setIntField(vertexId.intValue()-1); + dummyValue.setTField(vertexId*2.0); + return dummyValue; + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(2.0,0)\n" + + "2,(4.0,1)\n" + + "3,(6.0,2)\n" + + "4,(8.0,3)\n" + + "5,(10.0,4)\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} From 0d098a06e4040457d7ccfa18060f954ed05b3a9d Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Mon, 24 Nov 2014 16:26:41 +0100 Subject: [PATCH 044/112] Added removeEdge, added more tests --- src/main/java/flink/graphs/Graph.java | 68 +++++++-- .../flink/graphs/TestGraphOperations.java | 140 +++++++++++++++--- 2 files changed, 174 insertions(+), 34 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index faf4d0b7ba149..978138376b5f9 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -499,6 +499,11 @@ public Graph addVertex (Tuple2 vertex, List> edg return this.union(newVertex); } + public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { + Graph newEdges = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); + return this.union(newEdges); + } + public Graph removeVertex (Tuple2 vertex) { DataSet> vertexToRemove = context.fromCollection(Arrays.asList(vertex)); @@ -514,6 +519,19 @@ public Graph removeVertex (Tuple2 vertex) { return new Graph(newVertices, newEdges, this.context); } + public Graph removeEdge (Tuple3 edge) { + + DataSet> edgeToRemove = context.fromCollection(Arrays.asList(edge)); + + DataSet> newVertices = getVertices(); + + DataSet> newEdges = getEdges().filter( + new RemoveEdgeFilter()).withBroadcastSet( + edgeToRemove, "edgeToRemove"); + + return new Graph(newVertices, newEdges, this.context); + } + private static final class RemoveVertexFilter extends RichFilterFunction> { private Tuple2 vertexToRemove; @@ -534,32 +552,56 @@ public boolean filter(Tuple2 vertex) throws Exception { private static final class RemoveEdgeFilter extends RichFilterFunction> { private Tuple2 vertexToRemove; + private Tuple3 edgeToRemove; @SuppressWarnings("unchecked") @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + + + List> vertexVariable = null; + List> edgeVariable = null; + + try { + vertexVariable = getRuntimeContext().getBroadcastVariable("vertexToRemove"); + } catch (IllegalArgumentException ex) {} + + try { + edgeVariable = getRuntimeContext().getBroadcastVariable("edgeToRemove"); + } catch (IllegalArgumentException ex) {} + + if (vertexVariable != null) { + vertexToRemove = vertexVariable.get(0); + } + + if (edgeVariable != null) { + edgeToRemove = edgeVariable.get(0); + } } @Override public boolean filter(Tuple3 edge) throws Exception { - if (edge.f0.equals(vertexToRemove.f0)) { - return false; - } - if (edge.f1.equals(vertexToRemove.f0)) { - return false; - } - return true; + if (vertexToRemove != null) { + if (edge.f0.equals(vertexToRemove.f0)) { + return false; + } + if (edge.f1.equals(vertexToRemove.f0)) { + return false; + } + return true; + } else if (edgeToRemove != null) { + if (edge.f0.equals(edgeToRemove.f0) + && edge.f1.equals(edgeToRemove.f1)) { + return false; + } + return true; + } + return true; } } - public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { - Graph newEdges = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); - return this.union(newEdges); - } - public Graph union (Graph graph) { DataSet> unionedVertices = graph.getVertices().union(this.getVertices()); DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index ec955854b71c7..47d9b967a36f3 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -20,7 +20,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 16; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -200,7 +200,7 @@ public boolean filter(Long value) throws Exception { } case 9: { /* - * Test addVertex() + * Test addVertex() -- simple case */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -229,7 +229,7 @@ public boolean filter(Long value) throws Exception { } case 10: { /* - * Test removeVertex() + * Test addVertex() -- add an existing vertex */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -237,7 +237,10 @@ public boolean filter(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph = graph.removeVertex(new Tuple2(5L, 5L)); + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 5L, 15L)); + + graph = graph.addVertex(new Tuple2(1L, 1L), edges); graph.getEdges().writeAsCsv(resultPath); @@ -245,26 +248,47 @@ public boolean filter(Long value) throws Exception { return "1,2,12\n" + "1,3,13\n" + + "1,5,15\n" + "2,3,23\n" + - "3,4,34\n"; + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } case 11: { /* - * Test addEdge() - */ + * Test removeVertex() -- simple case + */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - List> vertices = new ArrayList>(); - List> edges = new ArrayList>(); - vertices.add(new Tuple2(6L, 6L)); - edges.add(new Tuple3(6L, 1L, 61L)); + graph = graph.removeVertex(new Tuple2(5L, 5L)); - graph = graph.union(graph.fromCollection(vertices, edges)); + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n"; + + } + case 12: { + /* + * Test removeVertex() -- remove an invalid vertex + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeVertex(new Tuple2(6L, 6L)); graph.getEdges().writeAsCsv(resultPath); @@ -276,12 +300,11 @@ public boolean filter(Long value) throws Exception { "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + - "5,1,51\n" + - "6,1,61\n"; + "5,1,51\n"; } - case 12: { + case 13: { /* - * Test union() + * Test addEdge() -- simple case */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -304,12 +327,87 @@ public boolean filter(Long value) throws Exception { "3,5,35\n" + "4,5,45\n" + "5,1,51\n" + - "6,1,61\n"; + "6,1,61\n"; } - case 13: { + case 14: { + /* + * Test removeEdge() -- simple case + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeEdge(new Tuple3(5L, 1L, 51L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n"; + + } + case 15: { + /* + * Test removeEdge() -- invalid edge + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeEdge(new Tuple3(6L, 1L, 61L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + + } + case 16: { /* - * Test passMessages() - */ + * Test union() + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> vertices = new ArrayList>(); + List> edges = new ArrayList>(); + + vertices.add(new Tuple2(6L, 6L)); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.union(graph.fromCollection(vertices, edges)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; } default: throw new IllegalArgumentException("Invalid program id"); From bbf4321d179cc0f9235d84ac0e7056b3e116748a Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Wed, 26 Nov 2014 00:27:07 +0100 Subject: [PATCH 045/112] Fixed/added tests, based on Vasia's feedback --- src/main/java/flink/graphs/Graph.java | 123 ++++++++++-------- .../flink/graphs/TestGraphOperations.java | 65 ++++++++- 2 files changed, 128 insertions(+), 60 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 978138376b5f9..e9a2698151eaa 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -30,14 +30,18 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.io.CsvReader; +import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.spargel.java.VertexCentricIteration; import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; +import java.io.IOException; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -494,14 +498,32 @@ public DataSet> fromCollection (Collection> vertices) return null; } - public Graph addVertex (Tuple2 vertex, List> edges) { - Graph newVertex = this.fromCollection(Arrays.asList(vertex), edges); - return this.union(newVertex); + public Graph addVertex (final Tuple2 vertex, List> edges) { + + DataSet> newVertex = this.context.fromCollection(Arrays.asList(vertex)); + + // Take care of empty edge set + if (edges.isEmpty()) { + return Graph.create(getVertices().union(newVertex), getEdges(), context); + } + + // Do not add already existing vertices (and their edges) + DataSet> oldVertices = getVertices(); + DataSet> newVertices = getVertices().union(newVertex).distinct(); + + if (oldVertices.equals(newVertices)) { + return this; + } + + // Add the vertex and its edges + DataSet> newEdges = getEdges().union(context.fromCollection(edges)); + return Graph.create(newVertices, newEdges, context); } public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { - Graph newEdges = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); - return this.union(newEdges); + + Graph partialGraph = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); + return this.union(partialGraph); } public Graph removeVertex (Tuple2 vertex) { @@ -513,25 +535,12 @@ public Graph removeVertex (Tuple2 vertex) { vertexToRemove, "vertexToRemove"); DataSet> newEdges = getEdges().filter( - new RemoveEdgeFilter()).withBroadcastSet( + new VertexRemovalEdgeFilter()).withBroadcastSet( vertexToRemove, "vertexToRemove"); return new Graph(newVertices, newEdges, this.context); } - public Graph removeEdge (Tuple3 edge) { - - DataSet> edgeToRemove = context.fromCollection(Arrays.asList(edge)); - - DataSet> newVertices = getVertices(); - - DataSet> newEdges = getEdges().filter( - new RemoveEdgeFilter()).withBroadcastSet( - edgeToRemove, "edgeToRemove"); - - return new Graph(newVertices, newEdges, this.context); - } - private static final class RemoveVertexFilter extends RichFilterFunction> { private Tuple2 vertexToRemove; @@ -549,56 +558,62 @@ public boolean filter(Tuple2 vertex) throws Exception { } } - private static final class RemoveEdgeFilter extends RichFilterFunction> { + private static final class VertexRemovalEdgeFilter extends RichFilterFunction> { private Tuple2 vertexToRemove; - private Tuple3 edgeToRemove; @SuppressWarnings("unchecked") @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - - List> vertexVariable = null; - List> edgeVariable = null; - - try { - vertexVariable = getRuntimeContext().getBroadcastVariable("vertexToRemove"); - } catch (IllegalArgumentException ex) {} - - try { - edgeVariable = getRuntimeContext().getBroadcastVariable("edgeToRemove"); - } catch (IllegalArgumentException ex) {} - - if (vertexVariable != null) { - vertexToRemove = vertexVariable.get(0); + vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); + } + + @Override + public boolean filter(Tuple3 edge) throws Exception { + + if (edge.f0.equals(vertexToRemove.f0)) { + return false; } - - if (edgeVariable != null) { - edgeToRemove = edgeVariable.get(0); + if (edge.f1.equals(vertexToRemove.f0)) { + return false; } + return true; + } + } + + public Graph removeEdge (Tuple3 edge) { + + DataSet> edgeToRemove = context.fromCollection(Arrays.asList(edge)); + + DataSet> newEdges = getEdges().filter( + new EdgeRemovalEdgeFilter()).withBroadcastSet( + edgeToRemove, "edgeToRemove"); + + return new Graph(this.getVertices(), newEdges, this.context); + } + + private static final class EdgeRemovalEdgeFilter extends RichFilterFunction> { + + private Tuple3 edgeToRemove; + + @SuppressWarnings("unchecked") + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + edgeToRemove = (Tuple3) getRuntimeContext().getBroadcastVariable("edgeToRemove").get(0); } @Override public boolean filter(Tuple3 edge) throws Exception { - if (vertexToRemove != null) { - if (edge.f0.equals(vertexToRemove.f0)) { - return false; - } - if (edge.f1.equals(vertexToRemove.f0)) { - return false; - } - return true; - } else if (edgeToRemove != null) { - if (edge.f0.equals(edgeToRemove.f0) - && edge.f1.equals(edgeToRemove.f1)) { - return false; - } - return true; - } - return true; + if (edge.f0.equals(edgeToRemove.f0) + && edge.f1.equals(edgeToRemove.f1)) { + return false; + } + return true; } } diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 47d9b967a36f3..d4909f7c4a7eb 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -20,7 +20,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 16; + private static int NUM_PROGRAMS = 18; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -257,6 +257,32 @@ public boolean filter(Long value) throws Exception { } case 11: { + /* + * Test addVertex() -- add vertex with empty edge set + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> edges = new ArrayList>(); + + graph = graph.addVertex(new Tuple2(6L, 6L), edges); + + graph.getVertices().writeAsCsv(resultPath); + + env.execute(); + + return "1,1\n" + + "2,2\n" + + "3,3\n" + + "4,4\n" + + "5,5\n" + + "6,6\n"; + + } + case 12: { /* * Test removeVertex() -- simple case */ @@ -278,7 +304,7 @@ public boolean filter(Long value) throws Exception { "3,4,34\n"; } - case 12: { + case 13: { /* * Test removeVertex() -- remove an invalid vertex */ @@ -302,7 +328,7 @@ public boolean filter(Long value) throws Exception { "4,5,45\n" + "5,1,51\n"; } - case 13: { + case 14: { /* * Test addEdge() -- simple case */ @@ -329,7 +355,34 @@ public boolean filter(Long value) throws Exception { "5,1,51\n" + "6,1,61\n"; } - case 14: { + case 15: { + /* + * Test addEdge() -- add already existing edge + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + + graph = graph.addEdge(new Tuple3(1L, 2L, 12L), + new Tuple2(1L, 1L), new Tuple2(2L, 2L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 16: { /* * Test removeEdge() -- simple case */ @@ -353,7 +406,7 @@ public boolean filter(Long value) throws Exception { "4,5,45\n"; } - case 15: { + case 17: { /* * Test removeEdge() -- invalid edge */ @@ -378,7 +431,7 @@ public boolean filter(Long value) throws Exception { "5,1,51\n"; } - case 16: { + case 18: { /* * Test union() */ From c9d68692f494c5611fc83b178e65c461ae4e47ca Mon Sep 17 00:00:00 2001 From: andralungu Date: Wed, 26 Nov 2014 15:30:41 +0100 Subject: [PATCH 046/112] fromCollection(edges) implemented abd tested --- src/main/java/flink/graphs/Graph.java | 31 +++++ .../java/flink/graphs/TestFromCollection.java | 124 ++++++++++++++++++ .../flink/graphs/TestGraphOperations.java | 20 +-- 3 files changed, 156 insertions(+), 19 deletions(-) create mode 100644 src/test/java/flink/graphs/TestFromCollection.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 519323c9d2da9..76e65cdbfb1cc 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -480,4 +480,35 @@ EV extends Serializable> Graph fromCollection(ExecutionEnvironment en return Graph.create(vertices, edges); } + /** + * Vertices may not have a value attached or may receive a value as a result of running the algorithm. + * @param env + * @param e the collection of edges + * @return a new graph formed from the edges, with no value for the vertices + */ + public static & Serializable, VV extends Serializable, + EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, Collection> e) { + + DataSet> edges = env.fromCollection(e); + + return Graph.create(edges); + } + + /** + * Vertices may have an initial value defined by a function. + * @param env + * @param e the collection of edges + * @return a new graph formed from the edges, with a custom value for the vertices, + * determined by the mapping function + */ + public static & Serializable, VV extends Serializable, + EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, + Collection> e, + final MapFunction mapper) { + + DataSet> edges = env.fromCollection(e); + + return Graph.create(edges, mapper); + } + } diff --git a/src/test/java/flink/graphs/TestFromCollection.java b/src/test/java/flink/graphs/TestFromCollection.java new file mode 100644 index 0000000000000..f4977ffb67a4b --- /dev/null +++ b/src/test/java/flink/graphs/TestFromCollection.java @@ -0,0 +1,124 @@ +package flink.graphs; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.types.NullValue; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +@RunWith(Parameterized.class) +public class TestFromCollection extends JavaProgramTestBase { + private static int NUM_PROGRAMS = 3; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestFromCollection(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch (progId) { + case 1: { + /* + * Test fromCollection(vertices, edges): + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.fromCollection(env, TestGraphUtils.getLongLongVertices(env), + TestGraphUtils.getLongLongEdges(env)); + + graph.getEdges().writeAsCsv(resultPath); + env.execute(); + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 2: { + /* + * Test fromCollection(edges) with no initial value for the vertices + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.fromCollection(env, + TestGraphUtils.getLongLongEdges(env)); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(null)\n" + + "2,(null)\n" + + "3,(null)\n" + + "4,(null)\n" + + "5,(null)\n"; + } + case 3: { + /* + * Test fromCollection(edges) with vertices initialised by a + * function that takes the id and doubles it + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.fromCollection(env, TestGraphUtils.getLongLongEdges(env), + new MapFunction() { + public Long map(Long vertexId) { + return vertexId * 2; + } + }); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,4\n" + + "3,6\n" + + "4,8\n" + + "5,10\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} \ No newline at end of file diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index f12bbdfd493c3..865a7ba0aeecb 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -16,7 +16,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 5; + private static int NUM_PROGRAMS = 4; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -139,24 +139,6 @@ public boolean filter(Long value) throws Exception { return "3,5,35\n" + "4,5,45\n"; } - case 5: { - /* - * Test fromCollection: - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.fromCollection(env, TestGraphUtils.getLongLongVertices(env), - TestGraphUtils.getLongLongEdges(env)); - - graph.getEdges().writeAsCsv(resultPath); - env.execute(); - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n"; - } default: throw new IllegalArgumentException("Invalid program id"); } From 70d77295ddbeb9e58f5d73874dd8ad266ab3f05b Mon Sep 17 00:00:00 2001 From: Daniel Bali Date: Thu, 27 Nov 2014 00:54:29 +0100 Subject: [PATCH 047/112] Fixed distinct vertex issues, changed removal filters --- src/main/java/flink/graphs/Graph.java | 91 ++++++++++----------------- 1 file changed, 33 insertions(+), 58 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index e9a2698151eaa..9b551dbab0cf6 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -18,33 +18,32 @@ package flink.graphs; -import org.apache.flink.api.common.functions.*; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; -import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.io.CsvReader; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.io.CsvReader; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.configuration.Configuration; import org.apache.flink.spargel.java.VertexCentricIteration; import org.apache.flink.util.Collector; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.ExecutionEnvironment; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; @SuppressWarnings("serial") @@ -504,19 +503,13 @@ public Graph addVertex (final Tuple2 vertex, List> oldVertices = getVertices(); - DataSet> newVertices = getVertices().union(newVertex).distinct(); - - if (oldVertices.equals(newVertices)) { - return this; + return Graph.create(getVertices().union(newVertex).distinct(), getEdges(), context); } // Add the vertex and its edges + DataSet> newVertices = getVertices().union(newVertex).distinct(); DataSet> newEdges = getEdges().union(context.fromCollection(edges)); + return Graph.create(newVertices, newEdges, context); } @@ -528,29 +521,22 @@ public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple public Graph removeVertex (Tuple2 vertex) { - DataSet> vertexToRemove = context.fromCollection(Arrays.asList(vertex)); - DataSet> newVertices = getVertices().filter( - new RemoveVertexFilter()).withBroadcastSet( - vertexToRemove, "vertexToRemove"); + new RemoveVertexFilter(vertex)); DataSet> newEdges = getEdges().filter( - new VertexRemovalEdgeFilter()).withBroadcastSet( - vertexToRemove, "vertexToRemove"); + new VertexRemovalEdgeFilter(vertex)); return new Graph(newVertices, newEdges, this.context); } - private static final class RemoveVertexFilter extends RichFilterFunction> { + private static final class RemoveVertexFilter implements FilterFunction> { private Tuple2 vertexToRemove; - @SuppressWarnings("unchecked") - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - this.vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); - } + public RemoveVertexFilter(Tuple2 vertex) { + vertexToRemove = vertex; + } @Override public boolean filter(Tuple2 vertex) throws Exception { @@ -558,17 +544,13 @@ public boolean filter(Tuple2 vertex) throws Exception { } } - private static final class VertexRemovalEdgeFilter extends RichFilterFunction> { + private static final class VertexRemovalEdgeFilter implements FilterFunction> { private Tuple2 vertexToRemove; - @SuppressWarnings("unchecked") - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - vertexToRemove = (Tuple2) getRuntimeContext().getBroadcastVariable("vertexToRemove").get(0); - } + public VertexRemovalEdgeFilter(Tuple2 vertex) { + vertexToRemove = vertex; + } @Override public boolean filter(Tuple3 edge) throws Exception { @@ -585,26 +567,19 @@ public boolean filter(Tuple3 edge) throws Exception { public Graph removeEdge (Tuple3 edge) { - DataSet> edgeToRemove = context.fromCollection(Arrays.asList(edge)); - DataSet> newEdges = getEdges().filter( - new EdgeRemovalEdgeFilter()).withBroadcastSet( - edgeToRemove, "edgeToRemove"); + new EdgeRemovalEdgeFilter(edge)); return new Graph(this.getVertices(), newEdges, this.context); } - private static final class EdgeRemovalEdgeFilter extends RichFilterFunction> { + private static final class EdgeRemovalEdgeFilter implements FilterFunction> { private Tuple3 edgeToRemove; - @SuppressWarnings("unchecked") - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - edgeToRemove = (Tuple3) getRuntimeContext().getBroadcastVariable("edgeToRemove").get(0); - } + public EdgeRemovalEdgeFilter(Tuple3 edge) { + edgeToRemove = edge; + } @Override public boolean filter(Tuple3 edge) throws Exception { From b97261ed5bb29bec7b0a56f7e0ddc8b9f075352c Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 27 Nov 2014 19:17:23 +0100 Subject: [PATCH 048/112] added descriptions for add/remove methods --- src/main/java/flink/graphs/Graph.java | 58 +++++++++++++------ .../flink/graphs/TestGraphOperations.java | 8 +-- 2 files changed, 44 insertions(+), 22 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 9b551dbab0cf6..07a08e412c903 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -492,41 +492,55 @@ public Graph fromCollection (Collection> vertices, Colle return new Graph(v, e, context); } - //TODO kostas add functionality - public DataSet> fromCollection (Collection> vertices) { - return null; - } - + /** + * Adds the input vertex and edges to the graph. + * If the vertex already exists in the graph, it will not be added again, + * but the given edges will. + * @param vertex + * @param edges + * @return + */ public Graph addVertex (final Tuple2 vertex, List> edges) { DataSet> newVertex = this.context.fromCollection(Arrays.asList(vertex)); - + // Take care of empty edge set if (edges.isEmpty()) { return Graph.create(getVertices().union(newVertex).distinct(), getEdges(), context); } - + // Add the vertex and its edges DataSet> newVertices = getVertices().union(newVertex).distinct(); DataSet> newEdges = getEdges().union(context.fromCollection(edges)); - + return Graph.create(newVertices, newEdges, context); } - public Graph addEdge (Tuple3 edge, Tuple2 source, Tuple2 target) { - - Graph partialGraph = this.fromCollection(Arrays.asList(source, target), Arrays.asList(edge)); + /** + * Adds the given edge to the graph. + * If the source and target vertices do not exist in the graph, + * they will also be added. + * @param source + * @param target + * @param edgeValue + * @return + */ + public Graph addEdge (Tuple2 source, Tuple2 target, EV edgeValue) { + Graph partialGraph = this.fromCollection(Arrays.asList(source, target), + Arrays.asList(new Tuple3(source.f0, target.f0, edgeValue))); return this.union(partialGraph); } + /** + * Removes the given vertex and its edges from the graph. + * @param vertex + * @return + */ public Graph removeVertex (Tuple2 vertex) { - DataSet> newVertices = getVertices().filter( new RemoveVertexFilter(vertex)); - DataSet> newEdges = getEdges().filter( new VertexRemovalEdgeFilter(vertex)); - return new Graph(newVertices, newEdges, this.context); } @@ -565,16 +579,18 @@ public boolean filter(Tuple3 edge) throws Exception { } } + /** + * Removes all edges that match the given edge from the graph. + * @param edge + * @return + */ public Graph removeEdge (Tuple3 edge) { - DataSet> newEdges = getEdges().filter( new EdgeRemovalEdgeFilter(edge)); - return new Graph(this.getVertices(), newEdges, this.context); } private static final class EdgeRemovalEdgeFilter implements FilterFunction> { - private Tuple3 edgeToRemove; public EdgeRemovalEdgeFilter(Tuple3 edge) { @@ -592,8 +608,14 @@ public boolean filter(Tuple3 edge) throws Exception { } } + /** + * Performs union on the vertices and edges sets of the input graphs + * removing duplicate vertices but maintaining duplicate edges. + * @param graph + * @return + */ public Graph union (Graph graph) { - DataSet> unionedVertices = graph.getVertices().union(this.getVertices()); + DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); return new Graph(unionedVertices, unionedEdges, this.context); } diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index d4909f7c4a7eb..89d617318eb45 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -339,8 +339,8 @@ public boolean filter(Long value) throws Exception { TestGraphUtils.getLongLongEdgeData(env), env); - graph = graph.addEdge(new Tuple3(6L, 1L, 61L), - new Tuple2(6L, 6L), new Tuple2(1L, 1L)); + graph = graph.addEdge(new Tuple2(6L, 6L), new Tuple2(1L, 1L), + 61L); graph.getEdges().writeAsCsv(resultPath); @@ -366,8 +366,8 @@ public boolean filter(Long value) throws Exception { TestGraphUtils.getLongLongEdgeData(env), env); - graph = graph.addEdge(new Tuple3(1L, 2L, 12L), - new Tuple2(1L, 1L), new Tuple2(2L, 2L)); + graph = graph.addEdge(new Tuple2(1L, 1L), new Tuple2(2L, 2L), + 12L); graph.getEdges().writeAsCsv(resultPath); From c527f6aaa145ba4aed07190f265521bcc2378c9c Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 27 Nov 2014 19:22:01 +0100 Subject: [PATCH 049/112] reorganized tests and readme --- README.md | 11 +- src/test/java/flink/graphs/GraphTest.java | 212 ------------ .../java/flink/graphs/TestGraphMutations.java | 305 ++++++++++++++++++ .../flink/graphs/TestGraphOperations.java | 235 +------------- src/test/java/flink/graphs/TestPGA.java | 244 +++++++------- 5 files changed, 433 insertions(+), 574 deletions(-) delete mode 100644 src/test/java/flink/graphs/GraphTest.java create mode 100644 src/test/java/flink/graphs/TestGraphMutations.java diff --git a/README.md b/README.md index bf9e022d8eac8..c620eafc0390b 100644 --- a/README.md +++ b/README.md @@ -18,12 +18,6 @@ Graph API for Apache Flink * getVertexIds() * getEdgeIds() * isWeaklyConnected() -* addVertex() -* removeVertex() -* addEdge() -* union() -* passMessages() - ###Vertex Class @@ -36,6 +30,11 @@ Graph API for Apache Flink * outDegrees() * getUndirected() * reverse() +* addVertex() +* removeVertex() +* addEdge() +* removeEdge() +* union() ##Wishlist diff --git a/src/test/java/flink/graphs/GraphTest.java b/src/test/java/flink/graphs/GraphTest.java deleted file mode 100644 index b6f5c3a12eb31..0000000000000 --- a/src/test/java/flink/graphs/GraphTest.java +++ /dev/null @@ -1,212 +0,0 @@ -package flink.graphs; - - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.util.Collector; -import org.junit.Before; -import org.junit.Test; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; - -import static org.junit.Assert.*; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - - -public class GraphTest implements Serializable{ - - // Assume existing graph object - // Tuple2 ids and values: 0,1,2,3 - // Edges: 0->1, 1->3, 0->3, 1->2 - - static Graph graph; - static ExecutionEnvironment env; - - @Before - public void testSetUp() { - env = ExecutionEnvironment.getExecutionEnvironment(); - setUpGraph(); - } - - - public static void setUpGraph() { - - List> Tuple2List = new ArrayList>(); - - for (int i = 0; i < 4; i++) { - Tuple2 v = new Tuple2(i, i); - Tuple2List.add(v); - } - - - List> edgeList = new ArrayList<>(); - - edgeList.add(new Tuple3(0, 1, 0)); - edgeList.add(new Tuple3(1, 3, 0)); - edgeList.add(new Tuple3(0, 3, 0)); - edgeList.add(new Tuple3(1, 2, 0)); - - DataSet> vertices = env.fromCollection(Tuple2List); - DataSet> edges = env.fromCollection(edgeList); - - graph = new Graph(vertices, edges, env); - } - - @Test - public void testCreate() throws Exception { - - List> Tuple2List = new ArrayList>(); - - for (int i = 0; i < 4; i++) { - Tuple2 v = new Tuple2(i, i); - Tuple2List.add(v); - } - - - List> edgeList = new ArrayList<>(); - - edgeList.add(new Tuple3(0, 1, 0)); - edgeList.add(new Tuple3(1, 3, 0)); - edgeList.add(new Tuple3(0, 3, 0)); - edgeList.add(new Tuple3(1, 2, 0)); - - DataSet> vertices = env.fromCollection(Tuple2List); - DataSet> edges = env.fromCollection(edgeList); - - Graph g = Graph.create(vertices, edges, env); - - g.getVertices().print(); - - env.execute(); - } - - @Test - public void testGetVertices() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testGetEdges() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testMapVertices() throws Exception { - - DataSet> doubled= graph.mapVertices(new MapFunction() { - @Override - public Integer map(Integer value) throws Exception { - return value * 2; - } - }); - - // Compare the two Datasets as lists? - - List> doubledData = new ArrayList<>(); - doubled.output(new LocalCollectionOutputFormat<>(doubledData)); - - - - DataSet> doubledDataset = graph.getVertices() - .map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple2 v) throws Exception { - return new Tuple2(v.f0, v.f1 * 2); - } - }); - List> originalDataDoubled = new ArrayList<>(); - doubledDataset.output(new LocalCollectionOutputFormat<>(originalDataDoubled)); - - assertEquals(doubledData, originalDataDoubled); - - // TODO(thvasilo): Test for function that changes the type of the value - - doubled.print(); - graph.getVertices().print(); - - env.execute(); - - } - - @Test - public void testSubgraph() throws Exception { - throw new NotImplementedException(); - } - - @Test - public void testPga() throws Exception { - // Test pga by running connected components - // Expected output is that all vertices end up with the same attribute, 0 - - // Send the vertex attribute to all neighbors - CoGroupFunction, Tuple3, Tuple2> - sendAttribute = - new CoGroupFunction, Tuple3, Tuple2>() { - @Override - public void coGroup(Iterable> vertices, - Iterable> edges, - Collector> tuple2Collector) throws Exception { - for (Tuple2 vertex : vertices) { - for (Tuple3 edge: edges) { - tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); - } - } - } - }; - - // Gather all messages and keep the message with the smallest attribute - GroupReduceFunction, Tuple2> - gatherAttributes = - new GroupReduceFunction, Tuple2>() { - @Override - public void reduce(Iterable> messages, - Collector> msgCollector) throws Exception { - - Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); - for (Tuple2 message : messages) { - if (message.f1 < minTuple.f1) { - minTuple = message.copy(); - } - } - msgCollector.collect(minTuple); - } - }; - - // Check if the produced message is smaller than the current vertex attribute, if yes change attribute - FlatJoinFunction, Tuple2, Tuple2> - apply = - new FlatJoinFunction, Tuple2, Tuple2>() { - @Override - public void join(Tuple2 msg, - Tuple2 vertex, - Collector> vertexCollector) throws Exception { - if (msg.f1 < vertex.f1) { - vertexCollector.collect(msg.copy()); - } - } - }; - - - // Run the pga iterations - Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); - - DataSet> conVerts = connected.getVertices(); - - // All vertices should end up with attribute 0 - conVerts.print(); - //TODO(thvasilo): Automate correctness testing - - env.execute(); - - } - -} \ No newline at end of file diff --git a/src/test/java/flink/graphs/TestGraphMutations.java b/src/test/java/flink/graphs/TestGraphMutations.java new file mode 100644 index 0000000000000..4236b1a654ca7 --- /dev/null +++ b/src/test/java/flink/graphs/TestGraphMutations.java @@ -0,0 +1,305 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestGraphMutations extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 9; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestGraphMutations(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test addVertex() -- simple case + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> edges = new ArrayList>(); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.addVertex(new Tuple2(6L, 6L), edges); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; + + } + case 2: { + /* + * Test addVertex() -- add an existing vertex + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 5L, 15L)); + + graph = graph.addVertex(new Tuple2(1L, 1L), edges); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "1,5,15\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + + } + case 3: { + /* + * Test addVertex() -- add vertex with empty edge set + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> edges = new ArrayList>(); + + graph = graph.addVertex(new Tuple2(6L, 6L), edges); + + graph.getVertices().writeAsCsv(resultPath); + + env.execute(); + + return "1,1\n" + + "2,2\n" + + "3,3\n" + + "4,4\n" + + "5,5\n" + + "6,6\n"; + + } + case 4: { + /* + * Test removeVertex() -- simple case + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeVertex(new Tuple2(5L, 5L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n"; + + } + case 5: { + /* + * Test removeVertex() -- remove an invalid vertex + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeVertex(new Tuple2(6L, 6L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 6: { + /* + * Test addEdge() -- simple case + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + + graph = graph.addEdge(new Tuple2(6L, 6L), new Tuple2(1L, 1L), + 61L); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; + } + case 7: { + /* + * Test addEdge() -- add already existing edge + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + + graph = graph.addEdge(new Tuple2(1L, 1L), new Tuple2(2L, 2L), + 12L); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 8: { + /* + * Test removeEdge() -- simple case + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeEdge(new Tuple3(5L, 1L, 51L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n"; + + } + case 9: { + /* + * Test removeEdge() -- invalid edge + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph = graph.removeEdge(new Tuple3(6L, 1L, 61L)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 89d617318eb45..dffd1efae352d 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -20,7 +20,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 18; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -199,239 +199,6 @@ public boolean filter(Long value) throws Exception { "5,1\n"; } case 9: { - /* - * Test addVertex() -- simple case - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - List> edges = new ArrayList>(); - edges.add(new Tuple3(6L, 1L, 61L)); - - graph = graph.addVertex(new Tuple2(6L, 6L), edges); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n" + - "6,1,61\n"; - - } - case 10: { - /* - * Test addVertex() -- add an existing vertex - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 5L, 15L)); - - graph = graph.addVertex(new Tuple2(1L, 1L), edges); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "1,5,15\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n"; - - } - case 11: { - /* - * Test addVertex() -- add vertex with empty edge set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - List> edges = new ArrayList>(); - - graph = graph.addVertex(new Tuple2(6L, 6L), edges); - - graph.getVertices().writeAsCsv(resultPath); - - env.execute(); - - return "1,1\n" + - "2,2\n" + - "3,3\n" + - "4,4\n" + - "5,5\n" + - "6,6\n"; - - } - case 12: { - /* - * Test removeVertex() -- simple case - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeVertex(new Tuple2(5L, 5L)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n"; - - } - case 13: { - /* - * Test removeVertex() -- remove an invalid vertex - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeVertex(new Tuple2(6L, 6L)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n"; - } - case 14: { - /* - * Test addEdge() -- simple case - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - - graph = graph.addEdge(new Tuple2(6L, 6L), new Tuple2(1L, 1L), - 61L); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n" + - "6,1,61\n"; - } - case 15: { - /* - * Test addEdge() -- add already existing edge - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - - graph = graph.addEdge(new Tuple2(1L, 1L), new Tuple2(2L, 2L), - 12L); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n"; - } - case 16: { - /* - * Test removeEdge() -- simple case - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeEdge(new Tuple3(5L, 1L, 51L)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n"; - - } - case 17: { - /* - * Test removeEdge() -- invalid edge - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeEdge(new Tuple3(6L, 1L, 61L)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n"; - - } - case 18: { /* * Test union() */ diff --git a/src/test/java/flink/graphs/TestPGA.java b/src/test/java/flink/graphs/TestPGA.java index 210c5e4b2e0bd..4d68604cc0125 100644 --- a/src/test/java/flink/graphs/TestPGA.java +++ b/src/test/java/flink/graphs/TestPGA.java @@ -1,122 +1,122 @@ -package flink.graphs; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.util.Collector; -import org.junit.Before; -import org.junit.Test; - -public class TestPGA { - - // Assume existing graph object - // Tuple2 ids and values: 0,1,2,3 - // Edges: 0->1, 1->3, 0->3, 1->2 - - static Graph graph; - static ExecutionEnvironment env; - - @Before - public void testSetUp() { - env = ExecutionEnvironment.getExecutionEnvironment(); - setUpGraph(); - } - - - public static void setUpGraph() { - - List> Tuple2List = new ArrayList>(); - - for (int i = 0; i < 4; i++) { - Tuple2 v = new Tuple2(i, i); - Tuple2List.add(v); - } - - - List> edgeList = new ArrayList<>(); - - edgeList.add(new Tuple3(0, 1, 0)); - edgeList.add(new Tuple3(1, 3, 0)); - edgeList.add(new Tuple3(0, 3, 0)); - edgeList.add(new Tuple3(1, 2, 0)); - - DataSet> vertices = env.fromCollection(Tuple2List); - DataSet> edges = env.fromCollection(edgeList); - - graph = new Graph(vertices, edges, env); - } - @SuppressWarnings("serial") - @Test - public void testPga() throws Exception { - // Test pga by running connected components - // Expected output is that all vertices end up with the same attribute, 0 - - // Send the vertex attribute to all neighbors - CoGroupFunction, Tuple3, Tuple2> - sendAttribute = - new CoGroupFunction, Tuple3, Tuple2>() { - @Override - public void coGroup(Iterable> vertices, - Iterable> edges, - Collector> tuple2Collector) throws Exception { - for (Tuple2 vertex : vertices) { - for (Tuple3 edge: edges) { - tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); - } - } - } - }; - - // Gather all messages and keep the message with the smallest attribute - GroupReduceFunction, Tuple2> - gatherAttributes = - new GroupReduceFunction, Tuple2>() { - @Override - public void reduce(Iterable> messages, - Collector> msgCollector) throws Exception { - - Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); - for (Tuple2 message : messages) { - if (message.f1 < minTuple.f1) { - minTuple = message.copy(); - } - } - msgCollector.collect(minTuple); - } - }; - - // Check if the produced message is smaller than the current vertex attribute, if yes change attribute - FlatJoinFunction, Tuple2, Tuple2> - apply = - new FlatJoinFunction, Tuple2, Tuple2>() { - @Override - public void join(Tuple2 msg, - Tuple2 vertex, - Collector> vertexCollector) throws Exception { - if (msg.f1 < vertex.f1) { - vertexCollector.collect(msg.copy()); - } - } - }; - - - // Run the pga iterations - Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); - - DataSet> conVerts = connected.getVertices(); - - // All vertices should end up with attribute 0 - conVerts.print(); - //TODO(thvasilo): Automate correctness testing - - env.execute(); - - } -} +//package flink.graphs; +// +//import java.util.ArrayList; +//import java.util.List; +// +//import org.apache.flink.api.common.functions.CoGroupFunction; +//import org.apache.flink.api.common.functions.FlatJoinFunction; +//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.tuple.Tuple2; +//import org.apache.flink.api.java.tuple.Tuple3; +//import org.apache.flink.util.Collector; +//import org.junit.Before; +//import org.junit.Test; +// +//public class TestPGA { +// +// // Assume existing graph object +// // Tuple2 ids and values: 0,1,2,3 +// // Edges: 0->1, 1->3, 0->3, 1->2 +// +// static Graph graph; +// static ExecutionEnvironment env; +// +// @Before +// public void testSetUp() { +// env = ExecutionEnvironment.getExecutionEnvironment(); +// setUpGraph(); +// } +// +// +// public static void setUpGraph() { +// +// List> Tuple2List = new ArrayList>(); +// +// for (int i = 0; i < 4; i++) { +// Tuple2 v = new Tuple2(i, i); +// Tuple2List.add(v); +// } +// +// +// List> edgeList = new ArrayList<>(); +// +// edgeList.add(new Tuple3(0, 1, 0)); +// edgeList.add(new Tuple3(1, 3, 0)); +// edgeList.add(new Tuple3(0, 3, 0)); +// edgeList.add(new Tuple3(1, 2, 0)); +// +// DataSet> vertices = env.fromCollection(Tuple2List); +// DataSet> edges = env.fromCollection(edgeList); +// +// graph = new Graph(vertices, edges, env); +// } +// @SuppressWarnings("serial") +// @Test +// public void testPga() throws Exception { +// // Test pga by running connected components +// // Expected output is that all vertices end up with the same attribute, 0 +// +// // Send the vertex attribute to all neighbors +// CoGroupFunction, Tuple3, Tuple2> +// sendAttribute = +// new CoGroupFunction, Tuple3, Tuple2>() { +// @Override +// public void coGroup(Iterable> vertices, +// Iterable> edges, +// Collector> tuple2Collector) throws Exception { +// for (Tuple2 vertex : vertices) { +// for (Tuple3 edge: edges) { +// tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); +// } +// } +// } +// }; +// +// // Gather all messages and keep the message with the smallest attribute +// GroupReduceFunction, Tuple2> +// gatherAttributes = +// new GroupReduceFunction, Tuple2>() { +// @Override +// public void reduce(Iterable> messages, +// Collector> msgCollector) throws Exception { +// +// Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); +// for (Tuple2 message : messages) { +// if (message.f1 < minTuple.f1) { +// minTuple = message.copy(); +// } +// } +// msgCollector.collect(minTuple); +// } +// }; +// +// // Check if the produced message is smaller than the current vertex attribute, if yes change attribute +// FlatJoinFunction, Tuple2, Tuple2> +// apply = +// new FlatJoinFunction, Tuple2, Tuple2>() { +// @Override +// public void join(Tuple2 msg, +// Tuple2 vertex, +// Collector> vertexCollector) throws Exception { +// if (msg.f1 < vertex.f1) { +// vertexCollector.collect(msg.copy()); +// } +// } +// }; +// +// +// // Run the pga iterations +// Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); +// +// DataSet> conVerts = connected.getVertices(); +// +// // All vertices should end up with attribute 0 +// conVerts.print(); +// //TODO(thvasilo): Automate correctness testing +// +// env.execute(); +// +// } +//} From 6a630b0b5697a4d7112c43eb2bf1dc0a562f3396 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 27 Nov 2014 19:34:51 +0100 Subject: [PATCH 050/112] added vertex-centric iteration, removed pga --- README.md | 4 +- src/main/java/flink/graphs/Graph.java | 46 +++---- src/main/java/flink/graphs/GraphUtils.java | 11 +- .../TestVertexCentricConnectedComponents.java | 117 ++++++++++++++++++ 4 files changed, 143 insertions(+), 35 deletions(-) create mode 100644 src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java diff --git a/README.md b/README.md index c620eafc0390b..663f57e21ae72 100644 --- a/README.md +++ b/README.md @@ -8,7 +8,6 @@ Graph API for Apache Flink ###Graph Class * getVertices() * getEdges() -* pga() * create(vertices, edges) * readTuple2CsvFile * readEdgesCsvFile @@ -17,7 +16,6 @@ Graph API for Apache Flink * numberOfEdges() * getVertexIds() * getEdgeIds() -* isWeaklyConnected() ###Vertex Class @@ -35,6 +33,8 @@ Graph API for Apache Flink * addEdge() * removeEdge() * union() +* isWeaklyConnected() +* runVertexCentricIteration() ##Wishlist diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 07a08e412c903..937fae75afce4 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -42,7 +42,9 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.spargel.java.MessagingFunction; import org.apache.flink.spargel.java.VertexCentricIteration; +import org.apache.flink.spargel.java.VertexUpdateFunction; import org.apache.flink.util.Collector; @@ -199,34 +201,6 @@ public Tuple2 map( return new Tuple2(value.f0.f0, 1L); } } - - /** - * Push-Gather-Apply model of graph computation - * @param cog - * @param gred - * @param fjoin - * @param maxIterations - * @param - * @return - */ - public Graph pga(CoGroupFunction, Tuple3, Tuple2> cog, - GroupReduceFunction, Tuple2> gred, - FlatJoinFunction, Tuple2, Tuple2> fjoin, - int maxIterations){ - - DeltaIteration, Tuple2> iteration = this.vertices - .iterateDelta(this.vertices, maxIterations, 0); - - DataSet> p = iteration.getWorkset().coGroup(this.edges).where(0).equalTo(0).with(cog); - - DataSet> g = p.groupBy(0).reduceGroup(gred); - - DataSet> a = g.join(iteration.getSolutionSet()).where(0).equalTo(0).with(fjoin); - - DataSet> result = iteration.closeWith(a, a); - - return new Graph<>(result, this.edges, this.context); - } /** * Convert the directed graph into an undirected graph @@ -620,8 +594,18 @@ public Graph union (Graph graph) { return new Graph(unionedVertices, unionedEdges, this.context); } - public Graph passMessages (VertexCentricIteration iteration) { - DataSet> newVertices = iteration.createResult(); - return new Graph(newVertices, edges, this.context); + /** + * Runs a Vertex-Centric iteration on the graph. + * @param vertexUpdateFunction + * @param messagingFunction + * @param maximumNumberOfIterations + * @return + */ + public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, + MessagingFunction messagingFunction, int maximumNumberOfIterations) { + DataSet> newVertices = vertices.runOperation( + VertexCentricIteration.withValuedEdges(edges, vertexUpdateFunction, messagingFunction, + maximumNumberOfIterations)); + return new Graph(newVertices, edges, context); } } diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java index 0273e44db7925..945075d4d6b39 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -1,8 +1,12 @@ package flink.graphs; +import java.util.ArrayList; +import java.util.List; + import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple; @SuppressWarnings("serial") @@ -10,8 +14,12 @@ public class GraphUtils { @SuppressWarnings({ "unchecked", "rawtypes" }) public static DataSet count(DataSet set) { + List list = new ArrayList<>(); + list.add(0); + DataSet initialCount = ExecutionEnvironment.getExecutionEnvironment().fromCollection(list); return set .map(new OneMapper()) + .union(initialCount) .reduce(new AddOnesReducer()) .first(1); } @@ -29,5 +37,4 @@ public Integer reduce(Integer one, Integer two) throws Exception { return one + two; } } - -} +} \ No newline at end of file diff --git a/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java new file mode 100644 index 0000000000000..e0e4c1426b0e9 --- /dev/null +++ b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java @@ -0,0 +1,117 @@ +/* + * 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 flink.graphs; + +import java.io.BufferedReader; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.spargel.java.MessageIterator; +import org.apache.flink.spargel.java.MessagingFunction; +import org.apache.flink.spargel.java.VertexUpdateFunction; +import org.apache.flink.test.testdata.ConnectedComponentsData; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.types.NullValue; + +@SuppressWarnings("serial") +public class TestVertexCentricConnectedComponents extends JavaProgramTestBase { + + private static final long SEED = 9487520347802987L; + + private static final int NUM_VERTICES = 1000; + + private static final int NUM_EDGES = 10000; + + private String resultPath; + + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempFilePath("results"); + } + + @Override + protected void testProgram() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet vertexIds = env.generateSequence(1, NUM_VERTICES); + DataSet edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED).split("\n")); + + DataSet> edges = edgeString.map(new EdgeParser()); + + DataSet> initialVertices = vertexIds.map(new IdAssigner()); + Graph graph = Graph.create(initialVertices, edges, env); + + Graph result = graph.runVertexCentricIteration(new CCUpdater(), new CCMessager(), 100); + + result.getVertices().writeAsCsv(resultPath, "\n", " "); + env.execute(); + } + + public static final class CCUpdater extends VertexUpdateFunction { + @Override + public void updateVertex(Long vertexKey, Long vertexValue, MessageIterator inMessages) { + long min = Long.MAX_VALUE; + for (long msg : inMessages) { + min = Math.min(min, msg); + } + if (min < vertexValue) { + setNewVertexValue(min); + } + } + } + + public static final class CCMessager extends MessagingFunction { + @Override + public void sendMessages(Long vertexId, Long componentId) { + sendMessageToAllNeighbors(componentId); + } + } + + /** + * A map function that takes a Long value and creates a 2-tuple out of it: + *
(Long value) -> (value, value)
+ */ + public static final class IdAssigner implements MapFunction> { + @Override + public Tuple2 map(Long value) { + return new Tuple2(value, value); + } + } + + @Override + protected void postSubmit() throws Exception { + for (BufferedReader reader : getResultReader(resultPath)) { + ConnectedComponentsData.checkOddEvenResult(reader); + } + } + + public static final class EdgeParser extends RichMapFunction> { + public Tuple3 map(String value) { + String[] nums = value.split(" "); + return new Tuple3(Long.parseLong(nums[0]), Long.parseLong(nums[1]), + NullValue.getInstance()); + } + } +} \ No newline at end of file From 3cf8b99e16ca9e34880d395d5572acc6149c92ca Mon Sep 17 00:00:00 2001 From: andralungu Date: Sat, 29 Nov 2014 09:52:01 +0100 Subject: [PATCH 051/112] inDegrees and getDegrees implemented and tested --- README.md | 2 + src/main/java/flink/graphs/Graph.java | 27 +- .../flink/graphs/TestGraphOperations.java | 334 ++++++++++-------- 3 files changed, 209 insertions(+), 154 deletions(-) diff --git a/README.md b/README.md index f76295a49e4d0..189905b1a33eb 100644 --- a/README.md +++ b/README.md @@ -21,6 +21,8 @@ Graph API for Apache Flink * mapEdges() * subGraph() * outDegrees() +* inDegrees() +* getDegrees() * getUndirected() * reverse() * addVertex() diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 48a131305b10e..20e481d6badbb 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -220,11 +220,30 @@ public boolean filter(Tuple3 value) throws Exception { * Return the out-degree of all vertices in the graph * @return A DataSet of Tuple2 */ - public DataSet> outDegrees() { + public DataSet> outDegrees() { - return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne()) - .groupBy(0).sum(1); - } + return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne()) + .groupBy(0).sum(1); + } + + /** + * Return the in-degree of all vertices in the graph + * @return A DataSet of Tuple2 + */ + public DataSet> inDegrees() { + + return vertices.join(edges).where(0).equalTo(1).map(new VertexKeyWithOne()) + .groupBy(0).sum(1); + } + + /** + * Return the degree of all vertices in the graph + * @return A DataSet of Tuple2 + */ + public DataSet> getDegrees() { + + return outDegrees().union(inDegrees()).groupBy(0).sum(1); + } private static final class VertexKeyWithOne implements MapFunction, Tuple3>, Tuple2> { diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index dffd1efae352d..b7118609f3d70 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -20,16 +20,16 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; - + private static int NUM_PROGRAMS = 11; + private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; private String expectedResult; - + public TestGraphOperations(Configuration config) { super(config); } - + @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); @@ -39,12 +39,12 @@ protected void preSubmit() throws Exception { protected void testProgram() throws Exception { expectedResult = GraphProgs.runProgram(curProgId, resultPath); } - + @Override protected void postSubmit() throws Exception { compareResultsByLinesInMemory(expectedResult, resultPath); } - + @Parameters public static Collection getConfigurations() throws FileNotFoundException, IOException { @@ -55,184 +55,218 @@ public static Collection getConfigurations() throws FileNotFoundExcept config.setInteger("ProgramId", i); tConfigs.add(config); } - + return toParameterList(tConfigs); } - + private static class GraphProgs { - + @SuppressWarnings("serial") public static String runProgram(int progId, String resultPath) throws Exception { - + switch(progId) { - case 1: { + case 1: { /* * Test getUndirected() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.getUndirected().getEdges().writeAsCsv(resultPath); - env.execute(); - return "1,2,12\n" + "2,1,12\n" + - "1,3,13\n" + "3,1,13\n" + - "2,3,23\n" + "3,2,23\n" + - "3,4,34\n" + "4,3,34\n" + - "3,5,35\n" + "5,3,35\n" + - "4,5,45\n" + "5,4,45\n" + - "5,1,51\n" + "1,5,51\n"; - } - case 2: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.getUndirected().getEdges().writeAsCsv(resultPath); + env.execute(); + return "1,2,12\n" + "2,1,12\n" + + "1,3,13\n" + "3,1,13\n" + + "2,3,23\n" + "3,2,23\n" + + "3,4,34\n" + "4,3,34\n" + + "3,5,35\n" + "5,3,35\n" + + "4,5,45\n" + "5,4,45\n" + + "5,1,51\n" + "1,5,51\n"; + } + case 2: { /* * Test reverse() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.reverse().getEdges().writeAsCsv(resultPath); - env.execute(); - return "2,1,12\n" + - "3,1,13\n" + - "3,2,23\n" + - "4,3,34\n" + - "5,3,35\n" + - "5,4,45\n" + - "1,5,51\n"; - } - case 3: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.reverse().getEdges().writeAsCsv(resultPath); + env.execute(); + return "2,1,12\n" + + "3,1,13\n" + + "3,2,23\n" + + "4,3,34\n" + + "5,3,35\n" + + "5,4,45\n" + + "1,5,51\n"; + } + case 3: { /* * Test outDegrees() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.outDegrees().writeAsCsv(resultPath); - env.execute(); - return "1,2\n" + - "2,1\n" + - "3,2\n" + - "4,1\n" + - "5,1\n"; - } - case 4: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,1\n"; + } + case 4: { + /* + * Test inDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.inDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,1\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,2\n"; + } + case 5: { + /* + * Test getDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.getDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,3\n" + + "2,2\n" + + "3,4\n" + + "4,2\n" + + "5,3\n"; + } + case 6: { /* * Test subgraph: */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.subgraph(new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 2); - } - }, - new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 34); - } - }).getEdges().writeAsCsv(resultPath); - - env.execute(); - return "3,5,35\n" + - "4,5,45\n"; - } - case 5: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.subgraph(new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 2); + } + }, + new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 34); + } + }).getEdges().writeAsCsv(resultPath); + + env.execute(); + return "3,5,35\n" + + "4,5,45\n"; + } + case 7: { /* * Test numberOfVertices() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.numberOfVertices().writeAsText(resultPath); - - env.execute(); - return "5"; - } - case 6: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfVertices().writeAsText(resultPath); + + env.execute(); + return "5"; + } + case 8: { /* * Test numberOfEdges() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.numberOfEdges().writeAsText(resultPath); - - env.execute(); - return "7"; - } - case 7: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfEdges().writeAsText(resultPath); + + env.execute(); + return "7"; + } + case 9: { /* * Test getVertexIds() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.getVertexIds().writeAsText(resultPath); - - env.execute(); - return "1\n2\n3\n4\n5\n"; - } - case 8: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getVertexIds().writeAsText(resultPath); + + env.execute(); + return "1\n2\n3\n4\n5\n"; + } + case 10: { /* * Test getEdgeIds() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.getEdgeIds().writeAsCsv(resultPath); - - env.execute(); - return "1,2\n" + "1,3\n" + - "2,3\n" + "3,4\n" + - "3,5\n" + "4,5\n" + - "5,1\n"; - } - case 9: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getEdgeIds().writeAsCsv(resultPath); + + env.execute(); + return "1,2\n" + "1,3\n" + + "2,3\n" + "3,4\n" + + "3,5\n" + "4,5\n" + + "5,1\n"; + } + case 11: { /* * Test union() */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - List> vertices = new ArrayList>(); - List> edges = new ArrayList>(); - - vertices.add(new Tuple2(6L, 6L)); - edges.add(new Tuple3(6L, 1L, 61L)); - - graph = graph.union(graph.fromCollection(vertices, edges)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n" + - "6,1,61\n"; - } - default: - throw new IllegalArgumentException("Invalid program id"); + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> vertices = new ArrayList>(); + List> edges = new ArrayList>(); + + vertices.add(new Tuple2(6L, 6L)); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.union(graph.fromCollection(vertices, edges)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); } } } - + } From d429f568de7f7c0bcddfcfa9f4768a800479a5d8 Mon Sep 17 00:00:00 2001 From: andralungu Date: Sat, 29 Nov 2014 12:35:24 +0100 Subject: [PATCH 052/112] subgraph(vertexFilter) and subgraph(edgeFilter) implemented and tested --- README.md | 4 +- src/main/java/flink/graphs/Graph.java | 35 ++ .../flink/graphs/TestGraphOperations.java | 340 ++++++++++-------- 3 files changed, 227 insertions(+), 152 deletions(-) diff --git a/README.md b/README.md index f76295a49e4d0..bc42cfd010052 100644 --- a/README.md +++ b/README.md @@ -19,7 +19,9 @@ Graph API for Apache Flink ##Tested Operations * mapVertices() * mapEdges() -* subGraph() +* subgraph() +* subgraph(vertexFilter) +* subgraph(edgeFilter) * outDegrees() * getUndirected() * reverse() diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 48a131305b10e..09210bf9d3a0e 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -180,6 +180,41 @@ public Graph subgraph(FilterFunction vertexFilter, FilterFunction return new Graph(filteredVertices, filteredEdges, this.context); } + + /** + * Apply value-based filtering functions to the graph + * and return a sub-graph that satisfies the predicates + * only for the vertices. + * @param vertexFilter + * @return + */ + public Graph subgraphVertexPredicate(FilterFunction vertexFilter) { + + DataSet> filteredVertices = this.vertices.filter( + new ApplyVertexFilter(vertexFilter)); + + DataSet> remainingEdges = this.edges.join(filteredVertices) + .where(0).equalTo(0) + .with(new ProjectEdge()) + .join(filteredVertices).where(1).equalTo(0) + .with(new ProjectEdge()); + + return new Graph(filteredVertices, remainingEdges, this.context); + } + + /** + * Apply value-based filtering functions to the graph + * and return a sub-graph that satisfies the predicates + * only for the edges. + * @param edgeFilter + * @return + */ + public Graph subgraphEdgePredicate(FilterFunction edgeFilter) { + DataSet> filteredEdges = this.edges.filter( + new ApplyEdgeFilter(edgeFilter)); + + return new Graph(this.vertices, filteredEdges, this.context); + } @ConstantFieldsFirst("0->0;1->1;2->2") private static final class ProjectEdge implements FlatJoinFunction, Tuple2, diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index dffd1efae352d..ad371089a84a7 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -20,16 +20,16 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 9; - + private static int NUM_PROGRAMS = 11; + private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; private String expectedResult; - + public TestGraphOperations(Configuration config) { super(config); } - + @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); @@ -39,12 +39,12 @@ protected void preSubmit() throws Exception { protected void testProgram() throws Exception { expectedResult = GraphProgs.runProgram(curProgId, resultPath); } - + @Override protected void postSubmit() throws Exception { compareResultsByLinesInMemory(expectedResult, resultPath); } - + @Parameters public static Collection getConfigurations() throws FileNotFoundException, IOException { @@ -55,184 +55,222 @@ public static Collection getConfigurations() throws FileNotFoundExcept config.setInteger("ProgramId", i); tConfigs.add(config); } - + return toParameterList(tConfigs); } - + private static class GraphProgs { - + @SuppressWarnings("serial") public static String runProgram(int progId, String resultPath) throws Exception { - + switch(progId) { - case 1: { + case 1: { /* * Test getUndirected() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.getUndirected().getEdges().writeAsCsv(resultPath); - env.execute(); - return "1,2,12\n" + "2,1,12\n" + - "1,3,13\n" + "3,1,13\n" + - "2,3,23\n" + "3,2,23\n" + - "3,4,34\n" + "4,3,34\n" + - "3,5,35\n" + "5,3,35\n" + - "4,5,45\n" + "5,4,45\n" + - "5,1,51\n" + "1,5,51\n"; - } - case 2: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.getUndirected().getEdges().writeAsCsv(resultPath); + env.execute(); + return "1,2,12\n" + "2,1,12\n" + + "1,3,13\n" + "3,1,13\n" + + "2,3,23\n" + "3,2,23\n" + + "3,4,34\n" + "4,3,34\n" + + "3,5,35\n" + "5,3,35\n" + + "4,5,45\n" + "5,4,45\n" + + "5,1,51\n" + "1,5,51\n"; + } + case 2: { /* * Test reverse() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.reverse().getEdges().writeAsCsv(resultPath); - env.execute(); - return "2,1,12\n" + - "3,1,13\n" + - "3,2,23\n" + - "4,3,34\n" + - "5,3,35\n" + - "5,4,45\n" + - "1,5,51\n"; - } - case 3: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.reverse().getEdges().writeAsCsv(resultPath); + env.execute(); + return "2,1,12\n" + + "3,1,13\n" + + "3,2,23\n" + + "4,3,34\n" + + "5,3,35\n" + + "5,4,45\n" + + "1,5,51\n"; + } + case 3: { /* * Test outDegrees() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.outDegrees().writeAsCsv(resultPath); - env.execute(); - return "1,2\n" + - "2,1\n" + - "3,2\n" + - "4,1\n" + - "5,1\n"; - } - case 4: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,1\n"; + } + case 4: { /* - * Test subgraph: + * Test subgraph(vertexFilter, edgeFilter): */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.subgraph(new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 2); - } - }, - new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 34); - } - }).getEdges().writeAsCsv(resultPath); - - env.execute(); - return "3,5,35\n" + - "4,5,45\n"; - } - case 5: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.subgraph(new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 2); + } + }, + new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 34); + } + }).getEdges().writeAsCsv(resultPath); + + env.execute(); + return "3,5,35\n" + + "4,5,45\n"; + } + case 5: { + /* + * Test subgraph(vertexFilter): + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.subgraphVertexPredicate(new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 2); + } + }).getEdges().writeAsCsv(resultPath); + + env.execute(); + return "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n"; + } + case 6: { + /* + * Test subgraph(edgeFilter): + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.subgraphEdgePredicate(new FilterFunction() { + public boolean filter(Long value) throws Exception { + return (value > 34); + } + }).getEdges().writeAsCsv(resultPath); + + env.execute(); + return "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 7: { /* * Test numberOfVertices() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.numberOfVertices().writeAsText(resultPath); - - env.execute(); - return "5"; - } - case 6: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfVertices().writeAsText(resultPath); + + env.execute(); + return "5"; + } + case 8: { /* * Test numberOfEdges() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.numberOfEdges().writeAsText(resultPath); - - env.execute(); - return "7"; - } - case 7: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.numberOfEdges().writeAsText(resultPath); + + env.execute(); + return "7"; + } + case 9: { /* * Test getVertexIds() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.getVertexIds().writeAsText(resultPath); - - env.execute(); - return "1\n2\n3\n4\n5\n"; - } - case 8: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getVertexIds().writeAsText(resultPath); + + env.execute(); + return "1\n2\n3\n4\n5\n"; + } + case 10: { /* * Test getEdgeIds() */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - graph.getEdgeIds().writeAsCsv(resultPath); - - env.execute(); - return "1,2\n" + "1,3\n" + - "2,3\n" + "3,4\n" + - "3,5\n" + "4,5\n" + - "5,1\n"; - } - case 9: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + graph.getEdgeIds().writeAsCsv(resultPath); + + env.execute(); + return "1,2\n" + "1,3\n" + + "2,3\n" + "3,4\n" + + "3,5\n" + "4,5\n" + + "5,1\n"; + } + case 11: { /* * Test union() */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - List> vertices = new ArrayList>(); - List> edges = new ArrayList>(); - - vertices.add(new Tuple2(6L, 6L)); - edges.add(new Tuple3(6L, 1L, 61L)); - - graph = graph.union(graph.fromCollection(vertices, edges)); - - graph.getEdges().writeAsCsv(resultPath); - - env.execute(); - - return "1,2,12\n" + - "1,3,13\n" + - "2,3,23\n" + - "3,4,34\n" + - "3,5,35\n" + - "4,5,45\n" + - "5,1,51\n" + - "6,1,61\n"; - } - default: - throw new IllegalArgumentException("Invalid program id"); + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + List> vertices = new ArrayList>(); + List> edges = new ArrayList>(); + + vertices.add(new Tuple2(6L, 6L)); + edges.add(new Tuple3(6L, 1L, 61L)); + + graph = graph.union(graph.fromCollection(vertices, edges)); + + graph.getEdges().writeAsCsv(resultPath); + + env.execute(); + + return "1,2,12\n" + + "1,3,13\n" + + "2,3,23\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n" + + "6,1,61\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); } } } - + } From 39fb9f78e5f6605a894dca5dd9b329f2435d79a0 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 29 Nov 2014 15:41:30 +0100 Subject: [PATCH 053/112] outDegrees with coGroup --- src/main/java/flink/graphs/Graph.java | 19 ++++++++++++++++-- .../flink/graphs/TestGraphOperations.java | 20 ++++++++++++++++++- 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 20e481d6badbb..6336a0e9cfbd5 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.List; +import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.FlatMapFunction; @@ -222,10 +223,24 @@ public boolean filter(Tuple3 value) throws Exception { */ public DataSet> outDegrees() { - return vertices.join(edges).where(0).equalTo(0).map(new VertexKeyWithOne()) - .groupBy(0).sum(1); + return vertices.coGroup(edges).where(0).equalTo(0) + .with(new CountNeighborsCoGroup()); } + private static final class CountNeighborsCoGroup implements CoGroupFunction, + Tuple3, Tuple2> { + @SuppressWarnings("unused") + public void coGroup(Iterable> vertex, + Iterable> outEdges, + Collector> out) { + long count = 0; + for (Tuple3 edge : outEdges) { + count++; + } + out.collect(new Tuple2(vertex.iterator().next().f0, count)); + } + } + /** * Return the in-degree of all vertices in the graph * @return A DataSet of Tuple2 diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index b7118609f3d70..1bc9b32b84187 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -13,6 +13,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.types.NullValue; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -20,7 +21,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 11; + private static int NUM_PROGRAMS = 12; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -263,6 +264,23 @@ public boolean filter(Long value) throws Exception { "5,1,51\n" + "6,1,61\n"; } + case 12: { + /* + * Test getDegrees() with disconnected data + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = + Graph.create(TestGraphUtils.getDisconnectedLongLongEdgeData(env), env); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,1\n" + + "3,0\n" + + "4,1\n" + + "5,0\n"; + } default: throw new IllegalArgumentException("Invalid program id"); } From 372870560410d73864a4c96217382efbba1cea5d Mon Sep 17 00:00:00 2001 From: andralungu Date: Sat, 29 Nov 2014 20:17:50 +0100 Subject: [PATCH 054/112] filterOnVertices and filterOnEdges implemented and tested --- README.md | 4 ++-- src/main/java/flink/graphs/Graph.java | 4 ++-- src/test/java/flink/graphs/TestGraphOperations.java | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index bc42cfd010052..0e3e6b754833f 100644 --- a/README.md +++ b/README.md @@ -20,8 +20,8 @@ Graph API for Apache Flink * mapVertices() * mapEdges() * subgraph() -* subgraph(vertexFilter) -* subgraph(edgeFilter) +* filterOnVertices(vertexFilter) +* filterOnEdges(edgeFilter) * outDegrees() * getUndirected() * reverse() diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 09210bf9d3a0e..a9bb6db99b4b2 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -188,7 +188,7 @@ public Graph subgraph(FilterFunction vertexFilter, FilterFunction * @param vertexFilter * @return */ - public Graph subgraphVertexPredicate(FilterFunction vertexFilter) { + public Graph filterOnVertices(FilterFunction vertexFilter) { DataSet> filteredVertices = this.vertices.filter( new ApplyVertexFilter(vertexFilter)); @@ -209,7 +209,7 @@ public Graph subgraphVertexPredicate(FilterFunction vertexFilter) * @param edgeFilter * @return */ - public Graph subgraphEdgePredicate(FilterFunction edgeFilter) { + public Graph filterOnEdges(FilterFunction edgeFilter) { DataSet> filteredEdges = this.edges.filter( new ApplyEdgeFilter(edgeFilter)); diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index ad371089a84a7..fba9393ccbcf0 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -122,7 +122,7 @@ public static String runProgram(int progId, String resultPath) throws Exception } case 4: { /* - * Test subgraph(vertexFilter, edgeFilter): + * Test subgraph: */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -145,13 +145,13 @@ public boolean filter(Long value) throws Exception { } case 5: { /* - * Test subgraph(vertexFilter): + * Test filterOnVertices: */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph.subgraphVertexPredicate(new FilterFunction() { + graph.filterOnVertices(new FilterFunction() { public boolean filter(Long value) throws Exception { return (value > 2); } @@ -164,13 +164,13 @@ public boolean filter(Long value) throws Exception { } case 6: { /* - * Test subgraph(edgeFilter): + * Test filterOnEdges: */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph.subgraphEdgePredicate(new FilterFunction() { + graph.filterOnEdges(new FilterFunction() { public boolean filter(Long value) throws Exception { return (value > 34); } From a645e3ba4409eb3b5f6f8517660e33e1499a4c98 Mon Sep 17 00:00:00 2001 From: andralungu Date: Sun, 30 Nov 2014 14:41:05 +0100 Subject: [PATCH 055/112] inDegree, outDegree, getDegrees implemented and tested --- src/main/java/flink/graphs/Graph.java | 5 +- src/test/java/flink/graphs/TestDegrees.java | 153 ++++++++++++++++++ .../flink/graphs/TestGraphOperations.java | 65 +------- .../java/flink/graphs/TestGraphUtils.java | 16 ++ 4 files changed, 178 insertions(+), 61 deletions(-) create mode 100644 src/test/java/flink/graphs/TestDegrees.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 6336a0e9cfbd5..96b6e497e4b75 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -229,7 +229,6 @@ public DataSet> outDegrees() { private static final class CountNeighborsCoGroup implements CoGroupFunction, Tuple3, Tuple2> { - @SuppressWarnings("unused") public void coGroup(Iterable> vertex, Iterable> outEdges, Collector> out) { @@ -247,8 +246,8 @@ public void coGroup(Iterable> vertex, */ public DataSet> inDegrees() { - return vertices.join(edges).where(0).equalTo(1).map(new VertexKeyWithOne()) - .groupBy(0).sum(1); + return vertices.coGroup(edges).where(0).equalTo(1) + .with(new CountNeighborsCoGroup()); } /** diff --git a/src/test/java/flink/graphs/TestDegrees.java b/src/test/java/flink/graphs/TestDegrees.java new file mode 100644 index 0000000000000..a829e0e588e3a --- /dev/null +++ b/src/test/java/flink/graphs/TestDegrees.java @@ -0,0 +1,153 @@ +package flink.graphs; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +@RunWith(Parameterized.class) +public class TestDegrees extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 5; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestDegrees(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch (progId) { + case 1: { + /* + * Test outDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,1\n"; + } + case 2: { + /* + * Test outDegrees() no outgoing edges + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env); + + graph.outDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,3\n" + + "2,1\n" + + "3,1\n" + + "4,1\n" + + "5,0\n"; + } + case 3: { + /* + * Test inDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.inDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,1\n" + + "2,1\n" + + "3,2\n" + + "4,1\n" + + "5,2\n"; + } + case 4: { + /* + * Test inDegrees() no ingoing edge + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env); + + graph.inDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,0\n" + + "2,1\n" + + "3,1\n" + + "4,1\n" + + "5,3\n"; + } + case 5: { + /* + * Test getDegrees() + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + graph.getDegrees().writeAsCsv(resultPath); + env.execute(); + return "1,3\n" + + "2,2\n" + + "3,4\n" + + "4,2\n" + + "5,3\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + + } + } + } +} diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 1bc9b32b84187..f510d8084a7c4 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -21,7 +21,7 @@ @RunWith(Parameterized.class) public class TestGraphOperations extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 12; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -105,57 +105,6 @@ public static String runProgram(int progId, String resultPath) throws Exception "1,5,51\n"; } case 3: { - /* - * Test outDegrees() - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.outDegrees().writeAsCsv(resultPath); - env.execute(); - return "1,2\n" + - "2,1\n" + - "3,2\n" + - "4,1\n" + - "5,1\n"; - } - case 4: { - /* - * Test inDegrees() - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.inDegrees().writeAsCsv(resultPath); - env.execute(); - return "1,1\n" + - "2,1\n" + - "3,2\n" + - "4,1\n" + - "5,2\n"; - } - case 5: { - /* - * Test getDegrees() - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env), env); - - graph.getDegrees().writeAsCsv(resultPath); - env.execute(); - return "1,3\n" + - "2,2\n" + - "3,4\n" + - "4,2\n" + - "5,3\n"; - } - case 6: { /* * Test subgraph: */ @@ -178,7 +127,7 @@ public boolean filter(Long value) throws Exception { return "3,5,35\n" + "4,5,45\n"; } - case 7: { + case 4: { /* * Test numberOfVertices() */ @@ -191,7 +140,7 @@ public boolean filter(Long value) throws Exception { env.execute(); return "5"; } - case 8: { + case 5: { /* * Test numberOfEdges() */ @@ -204,7 +153,7 @@ public boolean filter(Long value) throws Exception { env.execute(); return "7"; } - case 9: { + case 6: { /* * Test getVertexIds() */ @@ -217,7 +166,7 @@ public boolean filter(Long value) throws Exception { env.execute(); return "1\n2\n3\n4\n5\n"; } - case 10: { + case 7: { /* * Test getEdgeIds() */ @@ -233,7 +182,7 @@ public boolean filter(Long value) throws Exception { "3,5\n" + "4,5\n" + "5,1\n"; } - case 11: { + case 8: { /* * Test union() */ @@ -264,7 +213,7 @@ public boolean filter(Long value) throws Exception { "5,1,51\n" + "6,1,61\n"; } - case 12: { + case 9: { /* * Test getDegrees() with disconnected data */ diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index ced72fa470f3b..5de35a034bc14 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -37,6 +37,22 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } + /** + * A graph that has at least one vertex with no ingoing/outgoing edges + */ + public static final DataSet> getLongLongEdgeDataWithZeroDegree( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Tuple3(1L, 2L, 12L)); + edges.add(new Tuple3(1L, 4L, 14L)); + edges.add(new Tuple3(1L, 5L, 15L)); + edges.add(new Tuple3(2L, 3L, 23L)); + edges.add(new Tuple3(3L, 5L, 35L)); + edges.add(new Tuple3(4L, 5L, 45L)); + + return env.fromCollection(edges); + } + /** * Function that produces an ArrayList of vertices */ From 21297bfa8a38e07c58d2a9950d31c2b633ca5eb5 Mon Sep 17 00:00:00 2001 From: andralungu Date: Sun, 30 Nov 2014 15:20:47 +0100 Subject: [PATCH 056/112] cleared VertexKeyWith one class definintion --- src/main/java/flink/graphs/Graph.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 96b6e497e4b75..867f0dc4333c1 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -259,15 +259,6 @@ public DataSet> getDegrees() { return outDegrees().union(inDegrees()).groupBy(0).sum(1); } - private static final class VertexKeyWithOne implements - MapFunction, Tuple3>, Tuple2> { - - public Tuple2 map( - Tuple2, Tuple3> value) { - return new Tuple2(value.f0.f0, 1L); - } - } - /** * Convert the directed graph into an undirected graph * by adding all inverse-direction edges. From 6d118b2469fad2bf8e017b971cf2a859c0c8cce2 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 30 Nov 2014 18:38:50 +0100 Subject: [PATCH 057/112] replace fromCollection with fromElements; remove unused import --- src/main/java/flink/graphs/Graph.java | 7 ++++--- src/test/java/flink/graphs/TestGraphOperations.java | 1 - 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index e494ea7beef9b..f2d1e5eda4105 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -264,6 +264,7 @@ public DataSet> outDegrees() { private static final class CountNeighborsCoGroup implements CoGroupFunction, Tuple3, Tuple2> { + @SuppressWarnings("unused") public void coGroup(Iterable> vertex, Iterable> outEdges, Collector> out) { @@ -644,9 +645,9 @@ public Graph fromCollection (Collection> vertices, Colle * @param edges * @return */ - public Graph addVertex (final Tuple2 vertex, List> edges) { - - DataSet> newVertex = this.context.fromCollection(Arrays.asList(vertex)); + @SuppressWarnings("unchecked") + public Graph addVertex (final Tuple2 vertex, List> edges) { + DataSet> newVertex = this.context.fromElements(vertex); // Take care of empty edge set if (edges.isEmpty()) { diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index 2996c5e0c8419..f81ae6da2569a 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -13,7 +13,6 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; -import org.apache.flink.types.NullValue; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; From 3de99e5442e00edfa45985a7676ff55bef9dc977 Mon Sep 17 00:00:00 2001 From: vasia Date: Wed, 3 Dec 2014 23:52:28 +0100 Subject: [PATCH 058/112] replaces tuples with vertex and edge classes --- src/main/java/flink/graphs/Edge.java | 8 +- .../java/flink/graphs/EdgeToTuple3Map.java | 17 + src/main/java/flink/graphs/Graph.java | 313 +++++++----------- src/main/java/flink/graphs/GraphUtils.java | 2 +- .../java/flink/graphs/Tuple2ToVertexMap.java | 17 + src/main/java/flink/graphs/Vertex.java | 4 +- .../java/flink/graphs/VertexToTuple2Map.java | 17 + src/test/java/flink/graphs/TestDegrees.java | 1 - .../java/flink/graphs/TestGraphMutations.java | 74 ++--- .../flink/graphs/TestGraphOperations.java | 10 +- .../java/flink/graphs/TestGraphUtils.java | 94 +++--- src/test/java/flink/graphs/TestMapEdges.java | 11 +- .../java/flink/graphs/TestMapVertices.java | 11 +- .../TestVertexCentricConnectedComponents.java | 18 +- 14 files changed, 272 insertions(+), 325 deletions(-) create mode 100644 src/main/java/flink/graphs/EdgeToTuple3Map.java create mode 100644 src/main/java/flink/graphs/Tuple2ToVertexMap.java create mode 100644 src/main/java/flink/graphs/VertexToTuple2Map.java diff --git a/src/main/java/flink/graphs/Edge.java b/src/main/java/flink/graphs/Edge.java index c7f6229568db3..e8500b4a230c6 100644 --- a/src/main/java/flink/graphs/Edge.java +++ b/src/main/java/flink/graphs/Edge.java @@ -22,7 +22,8 @@ import org.apache.flink.api.java.tuple.Tuple3; -public class Edge extends Tuple3{ +public class Edge & Serializable, V extends Serializable> + extends Tuple3{ private static final long serialVersionUID = 1L; @@ -40,12 +41,7 @@ public Edge(K src, K trg, V val) { } public Edge reverse() { - if (f2 != null) { return new Edge(this.f1, this.f0, this.f2); - } - else { - return new Edge(this.f1, this.f0); - } } public void setSource(K src) { diff --git a/src/main/java/flink/graphs/EdgeToTuple3Map.java b/src/main/java/flink/graphs/EdgeToTuple3Map.java new file mode 100644 index 0000000000000..2ec75c29ecdcc --- /dev/null +++ b/src/main/java/flink/graphs/EdgeToTuple3Map.java @@ -0,0 +1,17 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple3; + +public class EdgeToTuple3Map & Serializable, + EV extends Serializable> implements MapFunction, Tuple3> { + + private static final long serialVersionUID = 1L; + + public Tuple3 map(Edge edge) { + return new Tuple3(edge.f0, edge.f1, edge.f2); + } + +} diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index f2d1e5eda4105..b19bc9f72217a 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -10,7 +10,8 @@ * 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, + * distributed under the License is distributed on an "AS IS" BASIS,K extends Serializablr & Comparable, + VV implements Serializable, * 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. @@ -40,7 +41,6 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -56,9 +56,9 @@ public class Graph & Serializable, VV extends Serializab private final ExecutionEnvironment context; - private final DataSet> vertices; + private final DataSet> vertices; - private final DataSet> edges; + private final DataSet> edges; private boolean isUndirected; @@ -67,13 +67,13 @@ public class Graph & Serializable, VV extends Serializab private static TypeInformation edgeValueType; - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { + public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { /** a graph is directed by default */ this(vertices, edges, context, false); } - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, + public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, boolean undirected) { this.vertices = vertices; this.edges = edges; @@ -85,11 +85,11 @@ public Graph(DataSet> vertices, DataSet> edges, E Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } - public DataSet> getVertices() { + public DataSet> getVertices() { return vertices; } - public DataSet> getEdges() { + public DataSet> getEdges() { return edges; } @@ -98,12 +98,13 @@ public DataSet> getEdges() { * @param mapper * @return */ - public DataSet> mapVertices(final MapFunction mapper) { + public DataSet> mapVertices(final MapFunction mapper) { return vertices.map(new ApplyMapperToVertexWithType(mapper)); } - private static final class ApplyMapperToVertexWithType implements MapFunction - , Tuple2>, ResultTypeQueryable> { + private static final class ApplyMapperToVertexWithType & Serializable, + VV extends Serializable, NV extends Serializable> implements MapFunction + , Vertex>, ResultTypeQueryable> { private MapFunction innerMapper; @@ -111,17 +112,17 @@ public ApplyMapperToVertexWithType(MapFunction theMapper) { this.innerMapper = theMapper; } - public Tuple2 map(Tuple2 value) throws Exception { - return new Tuple2(value.f0, innerMapper.map(value.f1)); + public Vertex map(Vertex value) throws Exception { + return new Vertex(value.f0, innerMapper.map(value.f1)); } @Override - public TypeInformation> getProducedType() { + public TypeInformation> getProducedType() { @SuppressWarnings("unchecked") TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, (TypeInformation)vertexValueType); - return new TupleTypeInfo>(keyType, newVertexValueType); + return new TupleTypeInfo>(keyType, newVertexValueType); } } @@ -130,12 +131,13 @@ public TypeInformation> getProducedType() { * @param mapper * @return */ - public DataSet> mapEdges(final MapFunction mapper) { + public DataSet> mapEdges(final MapFunction mapper) { return edges.map(new ApplyMapperToEdgeWithType(mapper)); } - private static final class ApplyMapperToEdgeWithType implements MapFunction - , Tuple3>, ResultTypeQueryable> { + private static final class ApplyMapperToEdgeWithType & Serializable, + EV extends Serializable, NV extends Serializable> implements MapFunction + , Edge>, ResultTypeQueryable> { private MapFunction innerMapper; @@ -143,17 +145,17 @@ public ApplyMapperToEdgeWithType(MapFunction theMapper) { this.innerMapper = theMapper; } - public Tuple3 map(Tuple3 value) throws Exception { - return new Tuple3(value.f0, value.f1, innerMapper.map(value.f2)); + public Edge map(Edge value) throws Exception { + return new Edge(value.f0, value.f1, innerMapper.map(value.f2)); } @Override - public TypeInformation> getProducedType() { + public TypeInformation> getProducedType() { @SuppressWarnings("unchecked") TypeInformation newEdgeValueType = TypeExtractor.getMapReturnTypes(innerMapper, (TypeInformation)edgeValueType); - return new TupleTypeInfo>(keyType, keyType, newEdgeValueType); + return new TupleTypeInfo>(keyType, keyType, newEdgeValueType); } } @@ -167,16 +169,16 @@ public TypeInformation> getProducedType() { */ public Graph subgraph(FilterFunction vertexFilter, FilterFunction edgeFilter) { - DataSet> filteredVertices = this.vertices.filter( + DataSet> filteredVertices = this.vertices.filter( new ApplyVertexFilter(vertexFilter)); - DataSet> remainingEdges = this.edges.join(filteredVertices) + DataSet> remainingEdges = this.edges.join(filteredVertices) .where(0).equalTo(0) .with(new ProjectEdge()) .join(filteredVertices).where(1).equalTo(0) .with(new ProjectEdge()); - DataSet> filteredEdges = remainingEdges.filter( + DataSet> filteredEdges = remainingEdges.filter( new ApplyEdgeFilter(edgeFilter)); return new Graph(filteredVertices, filteredEdges, this.context); @@ -191,10 +193,10 @@ public Graph subgraph(FilterFunction vertexFilter, FilterFunction */ public Graph filterOnVertices(FilterFunction vertexFilter) { - DataSet> filteredVertices = this.vertices.filter( + DataSet> filteredVertices = this.vertices.filter( new ApplyVertexFilter(vertexFilter)); - DataSet> remainingEdges = this.edges.join(filteredVertices) + DataSet> remainingEdges = this.edges.join(filteredVertices) .where(0).equalTo(0) .with(new ProjectEdge()) .join(filteredVertices).where(1).equalTo(0) @@ -211,22 +213,24 @@ public Graph filterOnVertices(FilterFunction vertexFilter) { * @return */ public Graph filterOnEdges(FilterFunction edgeFilter) { - DataSet> filteredEdges = this.edges.filter( + DataSet> filteredEdges = this.edges.filter( new ApplyEdgeFilter(edgeFilter)); return new Graph(this.vertices, filteredEdges, this.context); } @ConstantFieldsFirst("0->0;1->1;2->2") - private static final class ProjectEdge implements FlatJoinFunction, Tuple2, - Tuple3> { - public void join(Tuple3 first, - Tuple2 second, Collector> out) { + private static final class ProjectEdge & Serializable, + VV extends Serializable, EV extends Serializable> implements FlatJoinFunction, Vertex, + Edge> { + public void join(Edge first, + Vertex second, Collector> out) { out.collect(first); } } - private static final class ApplyVertexFilter implements FilterFunction> { + private static final class ApplyVertexFilter & Serializable, + VV extends Serializable> implements FilterFunction> { private FilterFunction innerFilter; @@ -234,20 +238,21 @@ public ApplyVertexFilter(FilterFunction theFilter) { this.innerFilter = theFilter; } - public boolean filter(Tuple2 value) throws Exception { + public boolean filter(Vertex value) throws Exception { return innerFilter.filter(value.f1); } } - private static final class ApplyEdgeFilter implements FilterFunction> { + private static final class ApplyEdgeFilter & Serializable, + EV extends Serializable> implements FilterFunction> { private FilterFunction innerFilter; public ApplyEdgeFilter(FilterFunction theFilter) { this.innerFilter = theFilter; } - public boolean filter(Tuple3 value) throws Exception { + public boolean filter(Edge value) throws Exception { return innerFilter.filter(value.f2); } } @@ -262,14 +267,14 @@ public DataSet> outDegrees() { .with(new CountNeighborsCoGroup()); } - private static final class CountNeighborsCoGroup implements CoGroupFunction, - Tuple3, Tuple2> { + private static final class CountNeighborsCoGroup & Serializable, + VV extends Serializable, EV extends Serializable> implements CoGroupFunction, + Edge, Tuple2> { @SuppressWarnings("unused") - public void coGroup(Iterable> vertex, - Iterable> outEdges, - Collector> out) { + public void coGroup(Iterable> vertex, + Iterable> outEdges, Collector> out) { long count = 0; - for (Tuple3 edge : outEdges) { + for (Edge edge : outEdges) { count++; } out.collect(new Tuple2(vertex.iterator().next().f0, count)); @@ -291,7 +296,6 @@ public DataSet> inDegrees() { * @return A DataSet of Tuple2 */ public DataSet> getDegrees() { - return outDegrees().union(inDegrees()).groupBy(0).sum(1); } @@ -305,18 +309,19 @@ public Graph getUndirected() throws UnsupportedOperationException { throw new UnsupportedOperationException("The graph is already undirected."); } else { - DataSet> undirectedEdges = + DataSet> undirectedEdges = edges.union(edges.map(new ReverseEdgesMap())); return new Graph(vertices, undirectedEdges, this.context, true); } } @ConstantFields("0->1;1->0;2->2") - private static final class ReverseEdgesMap implements MapFunction, - Tuple3> { + private static final class ReverseEdgesMap & Serializable, + EV extends Serializable> implements MapFunction, + Edge> { - public Tuple3 map(Tuple3 value) { - return new Tuple3(value.f1, value.f0, value.f2); + public Edge map(Edge value) { + return new Edge(value.f1, value.f0, value.f2); } } @@ -330,8 +335,8 @@ public Graph reverse() throws UnsupportedOperationException { throw new UnsupportedOperationException("The graph is already undirected."); } else { - DataSet> undirectedEdges = edges.map(new ReverseEdgesMap()); - return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); + DataSet> undirectedEdges = edges.map(new ReverseEdgesMap()); + return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); } } @@ -343,7 +348,8 @@ public Graph reverse() throws UnsupportedOperationException { */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph - create(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { + create(DataSet> vertices, DataSet> edges, + ExecutionEnvironment context) { return new Graph(vertices, edges, context); } @@ -354,8 +360,8 @@ public Graph reverse() throws UnsupportedOperationException { * @return */ public static & Serializable, EV extends Serializable> - Graph create(DataSet> edges, ExecutionEnvironment context) { - DataSet> vertices = + Graph create(DataSet> edges, ExecutionEnvironment context) { + DataSet> vertices = edges.flatMap(new EmitSrcAndTarget()).distinct(); return new Graph(vertices, edges, context); } @@ -369,16 +375,17 @@ Graph create(DataSet> edges, ExecutionEnviron * @return */ public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph create(DataSet> edges, final MapFunction mapper, + Graph create(DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { - DataSet> vertices = + DataSet> vertices = edges.flatMap(new EmitSrcAndTargetAsTuple1()) .distinct().map(new ApplyMapperToVertexValuesWithType(mapper)); return new Graph(vertices, edges, context); } - private static final class ApplyMapperToVertexValuesWithType implements MapFunction - , Tuple2>, ResultTypeQueryable> { + private static final class ApplyMapperToVertexValuesWithType & Serializable, + VV extends Serializable> implements MapFunction + , Vertex>, ResultTypeQueryable> { private MapFunction innerMapper; @@ -386,33 +393,33 @@ public ApplyMapperToVertexValuesWithType(MapFunction theMapper) { this.innerMapper = theMapper; } - public Tuple2 map(Tuple1 value) throws Exception { - return new Tuple2(value.f0, innerMapper.map(value.f0)); + public Vertex map(Tuple1 value) throws Exception { + return new Vertex(value.f0, innerMapper.map(value.f0)); } @Override - public TypeInformation> getProducedType() { + public TypeInformation> getProducedType() { @SuppressWarnings("unchecked") TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, (TypeInformation)keyType); - return new TupleTypeInfo>(keyType, newVertexValueType); + return new TupleTypeInfo>(keyType, newVertexValueType); } } private static final class EmitSrcAndTarget & Serializable, EV extends Serializable> - implements FlatMapFunction, Tuple2> { - public void flatMap(Tuple3 edge, - Collector> out) { + implements FlatMapFunction, Vertex> { + public void flatMap(Edge edge, + Collector> out) { - out.collect(new Tuple2(edge.f0, NullValue.getInstance())); - out.collect(new Tuple2(edge.f1, NullValue.getInstance())); + out.collect(new Vertex(edge.f0, NullValue.getInstance())); + out.collect(new Vertex(edge.f1, NullValue.getInstance())); } } private static final class EmitSrcAndTargetAsTuple1 & Serializable, - EV extends Serializable> implements FlatMapFunction, Tuple1> { - public void flatMap(Tuple3 edge, Collector> out) { + EV extends Serializable> implements FlatMapFunction, Tuple1> { + public void flatMap(Edge edge, Collector> out) { out.collect(new Tuple1(edge.f0)); out.collect(new Tuple1(edge.f1)); @@ -443,73 +450,6 @@ public Tuple2 map(Tuple2 value) throws Exception { return vertices; } - /** - * Read and create the graph edge dataset from a csv file - * @param env - * @param filePath - * @param delimiter - * @param Tuple2IdClass - * @param edgeValueClass - * @return - */ - public static & Serializable, EV extends Serializable> - DataSet> readEdgesCsvFile(ExecutionEnvironment env, String filePath, - char delimiter, Class Tuple2IdClass, Class edgeValueClass) { - - CsvReader reader = new CsvReader(filePath, env); - DataSet> edges = reader.fieldDelimiter(delimiter) - .types(Tuple2IdClass, Tuple2IdClass, edgeValueClass) - .map(new MapFunction, Tuple3>() { - - public Tuple3 map(Tuple3 value) throws Exception { - return (Tuple3)value; - } - }); - return edges; - } - - /** - * Create the graph, by reading a csv file for vertices - * and a csv file for the edges - * @param env - * @param Tuple2Filepath - * @param Tuple2Delimiter - * @param edgeFilepath - * @param edgeDelimiter - * @param Tuple2IdClass - * @param Tuple2ValueClass - * @param edgeValueClass - * @return - */ - public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph readGraphFromCsvFile(ExecutionEnvironment env, - String Tuple2Filepath, char Tuple2Delimiter, String edgeFilepath, char edgeDelimiter, - Class Tuple2IdClass, Class Tuple2ValueClass, Class edgeValueClass, - ExecutionEnvironment context) { - - CsvReader Tuple2Reader = new CsvReader(Tuple2Filepath, env); - DataSet> vertices = Tuple2Reader.fieldDelimiter(Tuple2Delimiter) - .types(Tuple2IdClass, Tuple2ValueClass).map(new MapFunction, - Tuple2>() { - - public Tuple2 map(Tuple2 value) throws Exception { - return (Tuple2)value; - } - }); - - CsvReader edgeReader = new CsvReader(edgeFilepath, env); - DataSet> edges = edgeReader.fieldDelimiter(edgeDelimiter) - .types(Tuple2IdClass, Tuple2IdClass, edgeValueClass) - .map(new MapFunction, Tuple3>() { - - public Tuple3 map(Tuple3 value) throws Exception { - return (Tuple3)value; - } - }); - - return Graph.create(vertices, edges, context); - } - /** * @return Singleton DataSet containing the vertex count */ @@ -533,9 +473,10 @@ public DataSet getVertexIds () { return vertices.map(new ExtractVertexIDMapper()); } - private static final class ExtractVertexIDMapper implements MapFunction, K> { + private static final class ExtractVertexIDMapper & Serializable, + VV extends Serializable> implements MapFunction, K> { @Override - public K map(Tuple2 vertex) throws Exception { + public K map(Vertex vertex) { return vertex.f0; } } @@ -544,9 +485,10 @@ public DataSet> getEdgeIds () { return edges.map(new ExtractEdgeIDsMapper()); } - private static final class ExtractEdgeIDsMapper implements MapFunction, Tuple2> { + private static final class ExtractEdgeIDsMapper & Serializable, + EV extends Serializable> implements MapFunction, Tuple2> { @Override - public Tuple2 map(Tuple3 edge) throws Exception { + public Tuple2 map(Edge edge) throws Exception { return new Tuple2(edge.f0, edge.f1); } } @@ -629,10 +571,10 @@ public Boolean map(Integer n) { } } - public Graph fromCollection (Collection> vertices, Collection> edges) { + public Graph fromCollection (Collection> vertices, Collection> edges) { - DataSet> v = context.fromCollection(vertices); - DataSet> e = context.fromCollection(edges); + DataSet> v = context.fromCollection(vertices); + DataSet> e = context.fromCollection(edges); return new Graph(v, e, context); } @@ -646,8 +588,8 @@ public Graph fromCollection (Collection> vertices, Colle * @return */ @SuppressWarnings("unchecked") - public Graph addVertex (final Tuple2 vertex, List> edges) { - DataSet> newVertex = this.context.fromElements(vertex); + public Graph addVertex (final Vertex vertex, List> edges) { + DataSet> newVertex = this.context.fromElements(vertex); // Take care of empty edge set if (edges.isEmpty()) { @@ -655,8 +597,8 @@ public Graph addVertex (final Tuple2 vertex, List> newVertices = getVertices().union(newVertex).distinct(); - DataSet> newEdges = getEdges().union(context.fromCollection(edges)); + DataSet> newVertices = getVertices().union(newVertex).distinct(); + DataSet> newEdges = getEdges().union(context.fromCollection(edges)); return Graph.create(newVertices, newEdges, context); } @@ -670,9 +612,9 @@ public Graph addVertex (final Tuple2 vertex, List addEdge (Tuple2 source, Tuple2 target, EV edgeValue) { + public Graph addEdge (Vertex source, Vertex target, EV edgeValue) { Graph partialGraph = this.fromCollection(Arrays.asList(source, target), - Arrays.asList(new Tuple3(source.f0, target.f0, edgeValue))); + Arrays.asList(new Edge(source.f0, target.f0, edgeValue))); return this.union(partialGraph); } @@ -681,38 +623,40 @@ public Graph addEdge (Tuple2 source, Tuple2 target, EV ed * @param vertex * @return */ - public Graph removeVertex (Tuple2 vertex) { - DataSet> newVertices = getVertices().filter( + public Graph removeVertex (Vertex vertex) { + DataSet> newVertices = getVertices().filter( new RemoveVertexFilter(vertex)); - DataSet> newEdges = getEdges().filter( + DataSet> newEdges = getEdges().filter( new VertexRemovalEdgeFilter(vertex)); return new Graph(newVertices, newEdges, this.context); } - private static final class RemoveVertexFilter implements FilterFunction> { + private static final class RemoveVertexFilter & Serializable, + VV extends Serializable> implements FilterFunction> { - private Tuple2 vertexToRemove; + private Vertex vertexToRemove; - public RemoveVertexFilter(Tuple2 vertex) { + public RemoveVertexFilter(Vertex vertex) { vertexToRemove = vertex; } @Override - public boolean filter(Tuple2 vertex) throws Exception { + public boolean filter(Vertex vertex) throws Exception { return !vertex.f0.equals(vertexToRemove.f0); } } - private static final class VertexRemovalEdgeFilter implements FilterFunction> { + private static final class VertexRemovalEdgeFilter & Serializable, + VV extends Serializable, EV extends Serializable> implements FilterFunction> { - private Tuple2 vertexToRemove; + private Vertex vertexToRemove; - public VertexRemovalEdgeFilter(Tuple2 vertex) { + public VertexRemovalEdgeFilter(Vertex vertex) { vertexToRemove = vertex; } @Override - public boolean filter(Tuple3 edge) throws Exception { + public boolean filter(Edge edge) throws Exception { if (edge.f0.equals(vertexToRemove.f0)) { return false; @@ -729,27 +673,24 @@ public boolean filter(Tuple3 edge) throws Exception { * @param edge * @return */ - public Graph removeEdge (Tuple3 edge) { - DataSet> newEdges = getEdges().filter( - new EdgeRemovalEdgeFilter(edge)); + public Graph removeEdge (Edge edge) { + DataSet> newEdges = getEdges().filter( + new EdgeRemovalEdgeFilter(edge)); return new Graph(this.getVertices(), newEdges, this.context); } - private static final class EdgeRemovalEdgeFilter implements FilterFunction> { - private Tuple3 edgeToRemove; + private static final class EdgeRemovalEdgeFilter & Serializable, + EV extends Serializable> implements FilterFunction> { + private Edge edgeToRemove; - public EdgeRemovalEdgeFilter(Tuple3 edge) { + public EdgeRemovalEdgeFilter(Edge edge) { edgeToRemove = edge; } @Override - public boolean filter(Tuple3 edge) throws Exception { - - if (edge.f0.equals(edgeToRemove.f0) - && edge.f1.equals(edgeToRemove.f1)) { - return false; - } - return true; + public boolean filter(Edge edge) { + return (!(edge.f0.equals(edgeToRemove.f0) + && edge.f1.equals(edgeToRemove.f1))); } } @@ -760,8 +701,8 @@ public boolean filter(Tuple3 edge) throws Exception { * @return */ public Graph union (Graph graph) { - DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); - DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); + DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); + DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); return new Graph(unionedVertices, unionedEdges, this.context); } @@ -774,10 +715,10 @@ public Graph union (Graph graph) { */ public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, MessagingFunction messagingFunction, int maximumNumberOfIterations) { - DataSet> newVertices = vertices.runOperation( - VertexCentricIteration.withValuedEdges(edges, vertexUpdateFunction, messagingFunction, - maximumNumberOfIterations)); - return new Graph(newVertices, edges, context); + DataSet> newVertices = vertices.map(new VertexToTuple2Map()).runOperation( + VertexCentricIteration.withValuedEdges(edges.map(new EdgeToTuple3Map()), + vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); + return new Graph(newVertices.map(new Tuple2ToVertexMap()), edges, context); } /** @@ -788,10 +729,10 @@ public Graph runVertexCentricIteration(VertexUpdateFunction & Serializable, VV extends Serializable, - EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, Collection> v, - Collection> e) throws Exception { - DataSet> vertices = env.fromCollection(v); - DataSet> edges = env.fromCollection(e); + EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, + Collection> v, Collection> e) throws Exception { + DataSet> vertices = env.fromCollection(v); + DataSet> edges = env.fromCollection(e); return Graph.create(vertices, edges, env); } @@ -804,9 +745,9 @@ EV extends Serializable> Graph fromCollection(ExecutionEnvironment en */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, - Collection> e) { + Collection> e) { - DataSet> edges = env.fromCollection(e); + DataSet> edges = env.fromCollection(e); return Graph.create(edges, env); } @@ -820,9 +761,9 @@ EV extends Serializable> Graph fromCollection(ExecutionEnviron */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, - Collection> e, + Collection> e, final MapFunction mapper) { - DataSet> edges = env.fromCollection(e); + DataSet> edges = env.fromCollection(e); return Graph.create(edges, mapper, env); } diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java index 945075d4d6b39..850daa5ab90d7 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -36,5 +36,5 @@ private static final class AddOnesReducer implements ReduceFunction { public Integer reduce(Integer one, Integer two) throws Exception { return one + two; } - } + } } \ No newline at end of file diff --git a/src/main/java/flink/graphs/Tuple2ToVertexMap.java b/src/main/java/flink/graphs/Tuple2ToVertexMap.java new file mode 100644 index 0000000000000..7cc7c2477d856 --- /dev/null +++ b/src/main/java/flink/graphs/Tuple2ToVertexMap.java @@ -0,0 +1,17 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; + +public class Tuple2ToVertexMap & Serializable, + VV extends Serializable> implements MapFunction, Vertex> { + + private static final long serialVersionUID = 1L; + + public Vertex map(Tuple2 vertex) { + return new Vertex(vertex.f0, vertex.f1); + } + +} diff --git a/src/main/java/flink/graphs/Vertex.java b/src/main/java/flink/graphs/Vertex.java index cc87ba35bf841..e589096ea5dd1 100644 --- a/src/main/java/flink/graphs/Vertex.java +++ b/src/main/java/flink/graphs/Vertex.java @@ -22,7 +22,8 @@ import org.apache.flink.api.java.tuple.Tuple2; -public class Vertex extends Tuple2 { +public class Vertex & Serializable, V extends Serializable> + extends Tuple2 { private static final long serialVersionUID = 1L; @@ -48,5 +49,4 @@ public void setId(K id) { public void setValue(V val) { this.f1 = val; } - } diff --git a/src/main/java/flink/graphs/VertexToTuple2Map.java b/src/main/java/flink/graphs/VertexToTuple2Map.java new file mode 100644 index 0000000000000..717e13c7953bf --- /dev/null +++ b/src/main/java/flink/graphs/VertexToTuple2Map.java @@ -0,0 +1,17 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; + +public class VertexToTuple2Map & Serializable, + VV extends Serializable> implements MapFunction, Tuple2> { + + private static final long serialVersionUID = 1L; + + public Tuple2 map(Vertex vertex) { + return new Tuple2(vertex.f0, vertex.f1); + } + +} diff --git a/src/test/java/flink/graphs/TestDegrees.java b/src/test/java/flink/graphs/TestDegrees.java index eca4fb573413a..ea46ae7955971 100644 --- a/src/test/java/flink/graphs/TestDegrees.java +++ b/src/test/java/flink/graphs/TestDegrees.java @@ -56,7 +56,6 @@ public static Collection getConfigurations() throws FileNotFoundExcept private static class GraphProgs { - @SuppressWarnings("serial") public static String runProgram(int progId, String resultPath) throws Exception { switch (progId) { diff --git a/src/test/java/flink/graphs/TestGraphMutations.java b/src/test/java/flink/graphs/TestGraphMutations.java index 4236b1a654ca7..deeab04818994 100644 --- a/src/test/java/flink/graphs/TestGraphMutations.java +++ b/src/test/java/flink/graphs/TestGraphMutations.java @@ -8,8 +8,6 @@ import java.util.List; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -73,13 +71,10 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - List> edges = new ArrayList>(); - edges.add(new Tuple3(6L, 1L, 61L)); - - graph = graph.addVertex(new Tuple2(6L, 6L), edges); - + List> edges = new ArrayList>(); + edges.add(new Edge(6L, 1L, 61L)); + graph = graph.addVertex(new Vertex(6L, 6L), edges); graph.getEdges().writeAsCsv(resultPath); - env.execute(); return "1,2,12\n" + @@ -102,15 +97,12 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 5L, 15L)); - - graph = graph.addVertex(new Tuple2(1L, 1L), edges); - + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 5L, 15L)); + graph = graph.addVertex(new Vertex(1L, 1L), edges); graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,3,13\n" + "1,5,15\n" + @@ -130,15 +122,11 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - List> edges = new ArrayList>(); - - graph = graph.addVertex(new Tuple2(6L, 6L), edges); - + List> edges = new ArrayList>(); + graph = graph.addVertex(new Vertex(6L, 6L), edges); graph.getVertices().writeAsCsv(resultPath); - env.execute(); - + return "1,1\n" + "2,2\n" + "3,3\n" + @@ -156,13 +144,10 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeVertex(new Tuple2(5L, 5L)); - + graph = graph.removeVertex(new Vertex(5L, 5L)); graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + @@ -178,13 +163,10 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeVertex(new Tuple2(6L, 6L)); - + graph = graph.removeVertex(new Vertex(6L, 6L)); graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + @@ -202,15 +184,11 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - - graph = graph.addEdge(new Tuple2(6L, 6L), new Tuple2(1L, 1L), + graph = graph.addEdge(new Vertex(6L, 6L), new Vertex(1L, 1L), 61L); - graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + @@ -229,15 +207,11 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - - graph = graph.addEdge(new Tuple2(1L, 1L), new Tuple2(2L, 2L), + graph = graph.addEdge(new Vertex(1L, 1L), new Vertex(2L, 2L), 12L); - graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,2,12\n" + "1,3,13\n" + @@ -256,13 +230,10 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeEdge(new Tuple3(5L, 1L, 51L)); - + graph = graph.removeEdge(new Edge(5L, 1L, 51L)); graph.getEdges().writeAsCsv(resultPath); - env.execute(); - + return "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + @@ -280,11 +251,8 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - - graph = graph.removeEdge(new Tuple3(6L, 1L, 61L)); - + graph = graph.removeEdge(new Edge(6L, 1L, 61L)); graph.getEdges().writeAsCsv(resultPath); - env.execute(); return "1,2,12\n" + diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index f81ae6da2569a..a49396dd43696 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -9,8 +9,6 @@ import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -229,11 +227,11 @@ public boolean filter(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - List> vertices = new ArrayList>(); - List> edges = new ArrayList>(); + List> vertices = new ArrayList>(); + List> edges = new ArrayList>(); - vertices.add(new Tuple2(6L, 6L)); - edges.add(new Tuple3(6L, 1L, 61L)); + vertices.add(new Vertex(6L, 6L)); + edges.add(new Edge(6L, 1L, 61L)); graph = graph.union(graph.fromCollection(vertices, edges)); diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 5de35a034bc14..9e5ebb20610f7 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -6,33 +6,31 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; public class TestGraphUtils { - public static final DataSet> getLongLongVertexData( + public static final DataSet> getLongLongVertexData( ExecutionEnvironment env) { - List> vertices = new ArrayList>(); - vertices.add(new Tuple2(1L, 1L)); - vertices.add(new Tuple2(2L, 2L)); - vertices.add(new Tuple2(3L, 3L)); - vertices.add(new Tuple2(4L, 4L)); - vertices.add(new Tuple2(5L, 5L)); + List> vertices = new ArrayList>(); + vertices.add(new Vertex(1L, 1L)); + vertices.add(new Vertex(2L, 2L)); + vertices.add(new Vertex(3L, 3L)); + vertices.add(new Vertex(4L, 4L)); + vertices.add(new Vertex(5L, 5L)); return env.fromCollection(vertices); } - public static final DataSet> getLongLongEdgeData( + public static final DataSet> getLongLongEdgeData( ExecutionEnvironment env) { - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 2L, 12L)); - edges.add(new Tuple3(1L, 3L, 13L)); - edges.add(new Tuple3(2L, 3L, 23L)); - edges.add(new Tuple3(3L, 4L, 34L)); - edges.add(new Tuple3(3L, 5L, 35L)); - edges.add(new Tuple3(4L, 5L, 45L)); - edges.add(new Tuple3(5L, 1L, 51L)); + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12L)); + edges.add(new Edge(1L, 3L, 13L)); + edges.add(new Edge(2L, 3L, 23L)); + edges.add(new Edge(3L, 4L, 34L)); + edges.add(new Edge(3L, 5L, 35L)); + edges.add(new Edge(4L, 5L, 45L)); + edges.add(new Edge(5L, 1L, 51L)); return env.fromCollection(edges); } @@ -40,15 +38,15 @@ public static final DataSet> getLongLongEdgeData( /** * A graph that has at least one vertex with no ingoing/outgoing edges */ - public static final DataSet> getLongLongEdgeDataWithZeroDegree( + public static final DataSet> getLongLongEdgeDataWithZeroDegree( ExecutionEnvironment env) { - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 2L, 12L)); - edges.add(new Tuple3(1L, 4L, 14L)); - edges.add(new Tuple3(1L, 5L, 15L)); - edges.add(new Tuple3(2L, 3L, 23L)); - edges.add(new Tuple3(3L, 5L, 35L)); - edges.add(new Tuple3(4L, 5L, 45L)); + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12L)); + edges.add(new Edge(1L, 4L, 14L)); + edges.add(new Edge(1L, 5L, 15L)); + edges.add(new Edge(2L, 3L, 23L)); + edges.add(new Edge(3L, 5L, 35L)); + edges.add(new Edge(4L, 5L, 45L)); return env.fromCollection(edges); } @@ -56,25 +54,25 @@ public static final DataSet> getLongLongEdgeDataWithZer /** * Function that produces an ArrayList of vertices */ - public static final List> getLongLongVertices( + public static final List> getLongLongVertices( ExecutionEnvironment env) { - List> vertices = new ArrayList<>(); - vertices.add(new Tuple2(1L, 1L)); - vertices.add(new Tuple2(2L, 2L)); - vertices.add(new Tuple2(3L, 3L)); - vertices.add(new Tuple2(4L, 4L)); - vertices.add(new Tuple2(5L, 5L)); + List> vertices = new ArrayList<>(); + vertices.add(new Vertex(1L, 1L)); + vertices.add(new Vertex(2L, 2L)); + vertices.add(new Vertex(3L, 3L)); + vertices.add(new Vertex(4L, 4L)); + vertices.add(new Vertex(5L, 5L)); return vertices; } - public static final DataSet> getDisconnectedLongLongEdgeData( + public static final DataSet> getDisconnectedLongLongEdgeData( ExecutionEnvironment env) { - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 2L, 12L)); - edges.add(new Tuple3(1L, 3L, 13L)); - edges.add(new Tuple3(2L, 3L, 23L)); - edges.add(new Tuple3(4L, 5L, 45L)); + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12L)); + edges.add(new Edge(1L, 3L, 13L)); + edges.add(new Edge(2L, 3L, 23L)); + edges.add(new Edge(4L, 5L, 45L)); return env.fromCollection(edges); } @@ -82,16 +80,16 @@ public static final DataSet> getDisconnectedLongLongEdg /** * Function that produces an ArrayList of edges */ - public static final List> getLongLongEdges( + public static final List> getLongLongEdges( ExecutionEnvironment env) { - List> edges = new ArrayList>(); - edges.add(new Tuple3(1L, 2L, 12L)); - edges.add(new Tuple3(1L, 3L, 13L)); - edges.add(new Tuple3(2L, 3L, 23L)); - edges.add(new Tuple3(3L, 4L, 34L)); - edges.add(new Tuple3(3L, 5L, 35L)); - edges.add(new Tuple3(4L, 5L, 45L)); - edges.add(new Tuple3(5L, 1L, 51L)); + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12L)); + edges.add(new Edge(1L, 3L, 13L)); + edges.add(new Edge(2L, 3L, 23L)); + edges.add(new Edge(3L, 4L, 34L)); + edges.add(new Edge(3L, 5L, 35L)); + edges.add(new Edge(4L, 5L, 45L)); + edges.add(new Edge(5L, 1L, 51L)); return edges; } diff --git a/src/test/java/flink/graphs/TestMapEdges.java b/src/test/java/flink/graphs/TestMapEdges.java index 2666ec34b2600..4fdb3af5ea77f 100644 --- a/src/test/java/flink/graphs/TestMapEdges.java +++ b/src/test/java/flink/graphs/TestMapEdges.java @@ -9,7 +9,6 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -76,7 +75,7 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { public Long map(Long value) throws Exception { return value+1; } @@ -101,7 +100,7 @@ public Long map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { public String map(Long value) throws Exception { return String.format("string(%d)", value); } @@ -126,7 +125,7 @@ public String map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedEdges = graph.mapEdges(new MapFunction>() { + DataSet>> mappedEdges = graph.mapEdges(new MapFunction>() { public Tuple1 map(Long value) throws Exception { Tuple1 tupleValue = new Tuple1(); tupleValue.setFields(value); @@ -153,7 +152,7 @@ public Tuple1 map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { + DataSet> mappedEdges = graph.mapEdges(new MapFunction() { public DummyCustomType map(Long value) throws Exception { DummyCustomType dummyValue = new DummyCustomType(); dummyValue.setIntField(value.intValue()); @@ -180,7 +179,7 @@ public DummyCustomType map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedEdges = graph.mapEdges( + DataSet>> mappedEdges = graph.mapEdges( new MapFunction>() { public DummyCustomParameterizedType map(Long value) throws Exception { DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); diff --git a/src/test/java/flink/graphs/TestMapVertices.java b/src/test/java/flink/graphs/TestMapVertices.java index d06e1ef3295d3..309af0d6ead7f 100644 --- a/src/test/java/flink/graphs/TestMapVertices.java +++ b/src/test/java/flink/graphs/TestMapVertices.java @@ -9,7 +9,6 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -76,7 +75,7 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { public Long map(Long value) throws Exception { return value+1; } @@ -99,7 +98,7 @@ public Long map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { public String map(Long value) throws Exception { String stringValue; if (value == 1) { @@ -142,7 +141,7 @@ else if (value == 5) { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { + DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { public Tuple1 map(Long value) throws Exception { Tuple1 tupleValue = new Tuple1(); tupleValue.setFields(value); @@ -167,7 +166,7 @@ public Tuple1 map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { + DataSet> mappedVertices = graph.mapVertices(new MapFunction() { public DummyCustomType map(Long value) throws Exception { DummyCustomType dummyValue = new DummyCustomType(); dummyValue.setIntField(value.intValue()); @@ -192,7 +191,7 @@ public DummyCustomType map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedVertices = graph.mapVertices( + DataSet>> mappedVertices = graph.mapVertices( new MapFunction>() { public DummyCustomParameterizedType map(Long value) throws Exception { DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); diff --git a/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java index e0e4c1426b0e9..9a45718d78329 100644 --- a/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java +++ b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java @@ -23,8 +23,6 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.spargel.java.MessageIterator; @@ -58,9 +56,9 @@ protected void testProgram() throws Exception { DataSet vertexIds = env.generateSequence(1, NUM_VERTICES); DataSet edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED).split("\n")); - DataSet> edges = edgeString.map(new EdgeParser()); + DataSet> edges = edgeString.map(new EdgeParser()); - DataSet> initialVertices = vertexIds.map(new IdAssigner()); + DataSet> initialVertices = vertexIds.map(new IdAssigner()); Graph graph = Graph.create(initialVertices, edges, env); Graph result = graph.runVertexCentricIteration(new CCUpdater(), new CCMessager(), 100); @@ -93,10 +91,10 @@ public void sendMessages(Long vertexId, Long componentId) { * A map function that takes a Long value and creates a 2-tuple out of it: *
(Long value) -> (value, value)
*/ - public static final class IdAssigner implements MapFunction> { + public static final class IdAssigner implements MapFunction> { @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); + public Vertex map(Long value) { + return new Vertex(value, value); } } @@ -107,10 +105,10 @@ protected void postSubmit() throws Exception { } } - public static final class EdgeParser extends RichMapFunction> { - public Tuple3 map(String value) { + public static final class EdgeParser extends RichMapFunction> { + public Edge map(String value) { String[] nums = value.split(" "); - return new Tuple3(Long.parseLong(nums[0]), Long.parseLong(nums[1]), + return new Edge(Long.parseLong(nums[0]), Long.parseLong(nums[1]), NullValue.getInstance()); } } From f0bad4cff6199ab9a31b31737e11abe4f2f7cb0c Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 4 Dec 2014 19:46:20 +0100 Subject: [PATCH 059/112] count to use Graph's execution environment --- src/main/java/flink/graphs/Graph.java | 6 +++--- src/main/java/flink/graphs/GraphUtils.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index b19bc9f72217a..6f79e7913d6e5 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -454,7 +454,7 @@ public Tuple2 map(Tuple2 value) throws Exception { * @return Singleton DataSet containing the vertex count */ public DataSet numberOfVertices () { - return GraphUtils.count(vertices); + return GraphUtils.count(vertices, context); } /** @@ -462,7 +462,7 @@ public DataSet numberOfVertices () { * @return Singleton DataSet containing the edge count */ public DataSet numberOfEdges () { - return GraphUtils.count(edges); + return GraphUtils.count(edges, context); } /** @@ -528,7 +528,7 @@ public DataSet isWeaklyConnected (int maxIterations) { DataSet> components = iteration.closeWith(changes, changes); DataSet result = GraphUtils.count(components.groupBy(1).reduceGroup( - new EmitFirstReducer())).map(new CheckIfOneComponentMapper()); + new EmitFirstReducer()), context).map(new CheckIfOneComponentMapper()); return result; } diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/GraphUtils.java index 850daa5ab90d7..1790b4e469bcd 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/GraphUtils.java @@ -13,10 +13,10 @@ public class GraphUtils { @SuppressWarnings({ "unchecked", "rawtypes" }) - public static DataSet count(DataSet set) { + public static DataSet count(DataSet set, ExecutionEnvironment env) { List list = new ArrayList<>(); list.add(0); - DataSet initialCount = ExecutionEnvironment.getExecutionEnvironment().fromCollection(list); + DataSet initialCount = env.fromCollection(list); return set .map(new OneMapper()) .union(initialCount) From c7fae0130c750b9dd059f0f6ddc9465c39c840f2 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 4 Dec 2014 19:46:49 +0100 Subject: [PATCH 060/112] simple graph metrics example --- .../flink/graphs/example/GraphMetrics.java | 115 ++++++++++++++++++ .../utils/EdgeWithLongIdNullValueParser.java | 16 +++ .../graphs/example/utils/ExampleUtils.java | 72 +++++++++++ 3 files changed, 203 insertions(+) create mode 100644 src/main/java/flink/graphs/example/GraphMetrics.java create mode 100644 src/main/java/flink/graphs/example/utils/EdgeWithLongIdNullValueParser.java create mode 100644 src/main/java/flink/graphs/example/utils/ExampleUtils.java diff --git a/src/main/java/flink/graphs/example/GraphMetrics.java b/src/main/java/flink/graphs/example/GraphMetrics.java new file mode 100644 index 0000000000000..b1c0a77429416 --- /dev/null +++ b/src/main/java/flink/graphs/example/GraphMetrics.java @@ -0,0 +1,115 @@ +package flink.graphs.example; + +import java.util.Collection; + +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.testdata.ConnectedComponentsData; +import org.apache.flink.types.NullValue; + +import flink.graphs.Edge; +import flink.graphs.Graph; +import flink.graphs.example.utils.EdgeWithLongIdNullValueParser; +import flink.graphs.example.utils.ExampleUtils; + +/** + * + * A simple example to illustrate the basic functionality of the graph-api. + * The program creates a random graph and computes and prints + * the following metrics: + * - number of vertices + * - number of edges + * - average node degree + * - the vertex ids with the max/min in- and out-degrees + * + */ +public class GraphMetrics implements ProgramDescription { + + static final int NUM_EDGES = 1000; + static final int NUM_VERTICES = 100; + static final long SEED = 9876; + + + @Override + public String getDescription() { + return null; + } + + public static void main(String[] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + /** create random input edges **/ + DataSet edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges( + NUM_EDGES, NUM_VERTICES, SEED).split("\n")); + DataSet> edges = edgeString.map(new EdgeWithLongIdNullValueParser()); + + /** create the graph from the edges **/ + Graph graph = Graph.create(edges, env); + + /** get the number of vertices **/ + DataSet numVertices = graph.numberOfVertices(); + + /** get the number of edges **/ + DataSet numEdges = graph.numberOfEdges(); + + /** compute the average node degree **/ + DataSet> verticesWithDegrees = graph.getDegrees(); + + DataSet avgNodeDegree = verticesWithDegrees.project(1).types(Long.class) + .aggregate(Aggregations.SUM, 0).map(new AvgNodeDegreeMapper()) + .withBroadcastSet(numVertices, "numberOfVertices"); + + /** find the vertex with the maximum in-degree **/ + DataSet maxInDegreeVertex = graph.inDegrees().maxBy(1).map(new ProjectVertexId()); + + /** find the vertex with the minimum in-degree **/ + DataSet minInDegreeVertex = graph.inDegrees().minBy(1).map(new ProjectVertexId()); + + /** find the vertex with the maximum out-degree **/ + DataSet maxOutDegreeVertex = graph.outDegrees().maxBy(1).map(new ProjectVertexId()); + + /** find the vertex with the minimum out-degree **/ + DataSet minOutDegreeVertex = graph.outDegrees().minBy(1).map(new ProjectVertexId()); + + /** print the results **/ + ExampleUtils.printResult(numVertices, "Total number of vertices", env); + ExampleUtils.printResult(numEdges, "Total number of edges", env); + ExampleUtils.printResult(avgNodeDegree, "Average node degree", env); + ExampleUtils.printResult(maxInDegreeVertex, "Vertex with Max in-degree", env); + ExampleUtils.printResult(minInDegreeVertex, "Vertex with Min in-degree", env); + ExampleUtils.printResult(maxOutDegreeVertex, "Vertex with Max out-degree", env); + ExampleUtils.printResult(minOutDegreeVertex, "Vertex with Min out-degree", env); + + env.execute(); + } + + @SuppressWarnings("serial") + private static final class AvgNodeDegreeMapper extends RichMapFunction, Double> { + + private int numberOfVertices; + + @Override + public void open(Configuration parameters) throws Exception { + Collection bCastSet = getRuntimeContext() + .getBroadcastVariable("numberOfVertices"); + numberOfVertices = bCastSet.iterator().next(); + } + + public Double map(Tuple1 sum) { + return (double) (sum.f0 / numberOfVertices) ; + } + } + + @SuppressWarnings("serial") + private static final class ProjectVertexId implements MapFunction, Long> { + public Long map(Tuple2 value) { return value.f0; } + } +} diff --git a/src/main/java/flink/graphs/example/utils/EdgeWithLongIdNullValueParser.java b/src/main/java/flink/graphs/example/utils/EdgeWithLongIdNullValueParser.java new file mode 100644 index 0000000000000..8c131e4bbda7b --- /dev/null +++ b/src/main/java/flink/graphs/example/utils/EdgeWithLongIdNullValueParser.java @@ -0,0 +1,16 @@ +package flink.graphs.example.utils; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.types.NullValue; + +import flink.graphs.Edge; + +public class EdgeWithLongIdNullValueParser extends RichMapFunction> { + private static final long serialVersionUID = 1L; + + public Edge map(String value) { + String[] nums = value.split(" "); + return new Edge(Long.parseLong(nums[0]), Long.parseLong(nums[1]), + NullValue.getInstance()); + } +} diff --git a/src/main/java/flink/graphs/example/utils/ExampleUtils.java b/src/main/java/flink/graphs/example/utils/ExampleUtils.java new file mode 100644 index 0000000000000..9d3b44b1b020b --- /dev/null +++ b/src/main/java/flink/graphs/example/utils/ExampleUtils.java @@ -0,0 +1,72 @@ +package flink.graphs.example.utils; + +import java.io.PrintStream; + +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; + +public class ExampleUtils { + + @SuppressWarnings({ "serial", "unchecked", "rawtypes" }) + public static void printResult(DataSet set, String msg, ExecutionEnvironment env) { + set.output(new PrintingOutputFormatWithMessage(msg) { + }); + } + + public static class PrintingOutputFormatWithMessage implements OutputFormat { + + private static final long serialVersionUID = 1L; + + private transient PrintStream stream; + + private transient String prefix; + + private String message; + + // -------------------------------------------------------------------------------------------- + + /** + * Instantiates a printing output format that prints to standard out. + */ + public PrintingOutputFormatWithMessage() {} + + public PrintingOutputFormatWithMessage(String msg) { + this.message = msg; + } + + @Override + public void open(int taskNumber, int numTasks) { + // get the target stream + this.stream = System.out; + + // set the prefix to message + this.prefix = message + ": "; + } + + @Override + public void writeRecord(T record) { + if (this.prefix != null) { + this.stream.println(this.prefix + record.toString()); + } + else { + this.stream.println(record.toString()); + } + } + + @Override + public void close() { + this.stream = null; + this.prefix = null; + } + + @Override + public String toString() { + return "Print to System.out"; + } + + @Override + public void configure(Configuration parameters) {} + } +} From 96fba40648f1048cfeebd780e947ce89fb1e7c99 Mon Sep 17 00:00:00 2001 From: andralungu Date: Tue, 9 Dec 2014 22:28:54 +0100 Subject: [PATCH 061/112] validate vertex ids implemented and tested --- src/main/java/flink/graphs/Graph.java | 10 + .../java/flink/graphs/GraphValidator.java | 17 ++ .../graphs/InvalidVertexIdsValidator.java | 72 +++++ .../java/flink/graphs/TestGraphCreation.java | 266 ++++++++++-------- .../java/flink/graphs/TestGraphUtils.java | 15 + 5 files changed, 262 insertions(+), 118 deletions(-) create mode 100644 src/main/java/flink/graphs/GraphValidator.java create mode 100644 src/main/java/flink/graphs/InvalidVertexIdsValidator.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 6f79e7913d6e5..3fa5b826453bb 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -85,6 +85,16 @@ public Graph(DataSet> vertices, DataSet> edges, Execut Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } + /** + * Function that checks whether a graph's ids are valid + * @return + */ + public & Serializable, VV extends Serializable, EV extends Serializable> DataSet + validate(GraphValidator validator) throws Exception { + + return validator.validate((Graph) this); + } + public DataSet> getVertices() { return vertices; } diff --git a/src/main/java/flink/graphs/GraphValidator.java b/src/main/java/flink/graphs/GraphValidator.java new file mode 100644 index 0000000000000..74499c6278dae --- /dev/null +++ b/src/main/java/flink/graphs/GraphValidator.java @@ -0,0 +1,17 @@ +package flink.graphs; +import java.io.Serializable; +import org.apache.flink.api.java.DataSet; + +/** + * A validation method for different types of Graphs + * + * @param + * @param + * @param + */ +public abstract class GraphValidator & Serializable, VV extends Serializable, + EV extends Serializable> implements Serializable{ + + public abstract DataSet validate(Graph graph) throws Exception; + +} \ No newline at end of file diff --git a/src/main/java/flink/graphs/InvalidVertexIdsValidator.java b/src/main/java/flink/graphs/InvalidVertexIdsValidator.java new file mode 100644 index 0000000000000..b5c3e2536a766 --- /dev/null +++ b/src/main/java/flink/graphs/InvalidVertexIdsValidator.java @@ -0,0 +1,72 @@ +package flink.graphs; + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +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.tuple.Tuple1; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +public class InvalidVertexIdsValidator & Serializable, VV extends Serializable, + EV extends Serializable> extends GraphValidator { + + /** + * Checks that the edge set input contains valid vertex Ids, + * i.e. that they also exist in the vertex input set. + * @return a singleton DataSet stating whether a graph is valid + * with respect to its vertex ids. + */ + @Override + public DataSet validate(Graph graph) throws Exception { + DataSet> edgeIds = graph.getEdges().flatMap(new MapEdgeIds()).distinct(); + DataSet invalidIds = graph.getVertices().coGroup(edgeIds).where(0).equalTo(0) + .with(new GroupInvalidIds()).first(1); + + return GraphUtils.count(invalidIds.map(new KToTupleMap()), ExecutionEnvironment.getExecutionEnvironment()) + .map(new InvalidIdsMap()); + } + + private static final class MapEdgeIds & Serializable, + EV extends Serializable> implements FlatMapFunction, + Tuple1> { + + @Override + public void flatMap(Edge edge, Collector> out) { + out.collect(new Tuple1(edge.f0)); + out.collect(new Tuple1(edge.f1)); + } + } + + private static final class GroupInvalidIds & Serializable, + VV extends Serializable> implements CoGroupFunction, Tuple1, K> { + + @Override + public void coGroup(Iterable> vertexId, + Iterable> edgeId, Collector out) { + if (!(vertexId.iterator().hasNext())) { + // found an id that doesn't exist in the vertex set + out.collect(edgeId.iterator().next().f0); + } + } + } + + private static final class KToTupleMap implements MapFunction> { + + @Override + public Tuple1 map (K key)throws Exception { + return new Tuple1<>(key); + } + } + + private static final class InvalidIdsMap implements MapFunction { + + @Override + public Boolean map (Integer numberOfInvalidIds)throws Exception { + return numberOfInvalidIds == 0; + } + } + +} \ No newline at end of file diff --git a/src/test/java/flink/graphs/TestGraphCreation.java b/src/test/java/flink/graphs/TestGraphCreation.java index a2e8c08517cd4..fc1305193b6a0 100644 --- a/src/test/java/flink/graphs/TestGraphCreation.java +++ b/src/test/java/flink/graphs/TestGraphCreation.java @@ -6,6 +6,7 @@ import java.util.LinkedList; 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.tuple.Tuple2; import org.apache.flink.configuration.Configuration; @@ -21,16 +22,16 @@ @RunWith(Parameterized.class) public class TestGraphCreation extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 6; - + private static int NUM_PROGRAMS = 8; + private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; private String expectedResult; - + public TestGraphCreation(Configuration config) { super(config); } - + @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); @@ -40,12 +41,12 @@ protected void preSubmit() throws Exception { protected void testProgram() throws Exception { expectedResult = GraphProgs.runProgram(curProgId, resultPath); } - + @Override protected void postSubmit() throws Exception { compareResultsByLinesInMemory(expectedResult, resultPath); } - + @Parameters public static Collection getConfigurations() throws FileNotFoundException, IOException { @@ -56,142 +57,171 @@ public static Collection getConfigurations() throws FileNotFoundExcept config.setInteger("ProgramId", i); tConfigs.add(config); } - + return toParameterList(tConfigs); } - + private static class GraphProgs { - + @SuppressWarnings("serial") public static String runProgram(int progId, String resultPath) throws Exception { - + switch(progId) { - case 1: { + case 1: { /* * Test create() with edge dataset and no vertex values */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(null)\n" + - "2,(null)\n" + - "3,(null)\n" + - "4,(null)\n" + - "5,(null)\n"; - } - case 2: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(null)\n" + + "2,(null)\n" + + "3,(null)\n" + + "4,(null)\n" + + "5,(null)\n"; + } + case 2: { /* * Test create() with edge dataset and a mapper that assigns the id as value */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), - new MapFunction() { - public Long map(Long vertexId) { - return vertexId; - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,1\n" + - "2,2\n" + - "3,3\n" + - "4,4\n" + - "5,5\n"; - } - case 3: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), + new MapFunction() { + public Long map(Long vertexId) { + return vertexId; + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,1\n" + + "2,2\n" + + "3,3\n" + + "4,4\n" + + "5,5\n"; + } + case 3: { /* * Test create() with edge dataset and a mapper that assigns a double constant as value */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), - new MapFunction() { - public Double map(Long value) { - return 0.1d; - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,0.1\n" + - "2,0.1\n" + - "3,0.1\n" + - "4,0.1\n" + - "5,0.1\n"; - } - case 4: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), + new MapFunction() { + public Double map(Long value) { + return 0.1d; + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,0.1\n" + + "2,0.1\n" + + "3,0.1\n" + + "4,0.1\n" + + "5,0.1\n"; + } + case 4: { /* * Test create() with edge dataset and a mapper that assigns a Tuple2 as value */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph, Long> graph = Graph.create( - TestGraphUtils.getLongLongEdgeData(env), new MapFunction>() { - public Tuple2 map(Long vertexId) { - return new Tuple2(vertexId*2, 42l); - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(2,42)\n" + - "2,(4,42)\n" + - "3,(6,42)\n" + - "4,(8,42)\n" + - "5,(10,42)\n"; - } - case 5: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph, Long> graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction>() { + public Tuple2 map(Long vertexId) { + return new Tuple2(vertexId*2, 42l); + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(2,42)\n" + + "2,(4,42)\n" + + "3,(6,42)\n" + + "4,(8,42)\n" + + "5,(10,42)\n"; + } + case 5: { /* * Test create() with edge dataset and a mapper that assigns a custom vertex value */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create( - TestGraphUtils.getLongLongEdgeData(env), new MapFunction() { - public DummyCustomType map(Long vertexId) { - return new DummyCustomType(vertexId.intValue()-1, false); - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(F,0)\n" + - "2,(F,1)\n" + - "3,(F,2)\n" + - "4,(F,3)\n" + - "5,(F,4)\n"; - } - case 6: { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction() { + public DummyCustomType map(Long vertexId) { + return new DummyCustomType(vertexId.intValue()-1, false); + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(F,0)\n" + + "2,(F,1)\n" + + "3,(F,2)\n" + + "4,(F,3)\n" + + "5,(F,4)\n"; + } + case 6: { /* * Test create() with edge dataset and a mapper that assigns a parametrized custom vertex value */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph, Long> graph = Graph.create( - TestGraphUtils.getLongLongEdgeData(env), - new MapFunction>() { - - DummyCustomParameterizedType dummyValue = - new DummyCustomParameterizedType(); - - public DummyCustomParameterizedType map(Long vertexId) { - dummyValue.setIntField(vertexId.intValue()-1); - dummyValue.setTField(vertexId*2.0); - return dummyValue; - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(2.0,0)\n" + - "2,(4.0,1)\n" + - "3,(6.0,2)\n" + - "4,(8.0,3)\n" + - "5,(10.0,4)\n"; - } - default: - throw new IllegalArgumentException("Invalid program id"); + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph, Long> graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), + new MapFunction>() { + + DummyCustomParameterizedType dummyValue = + new DummyCustomParameterizedType(); + + public DummyCustomParameterizedType map(Long vertexId) { + dummyValue.setIntField(vertexId.intValue()-1); + dummyValue.setTField(vertexId*2.0); + return dummyValue; + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(2.0,0)\n" + + "2,(4.0,1)\n" + + "3,(6.0,2)\n" + + "4,(8.0,3)\n" + + "5,(10.0,4)\n"; + } + case 7: { + /* + * Test validate(): + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet> vertices = TestGraphUtils.getLongLongVertexData(env); + DataSet> edges = TestGraphUtils.getLongLongEdgeData(env); + Graph graph = new Graph(vertices, edges, env); + DataSet result = graph.validate(new InvalidVertexIdsValidator()); + result.writeAsText(resultPath); + env.execute(); + + return "true\n"; + } + case 8: { + /* + * Test validate() - invalid vertex ids + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet> vertices = TestGraphUtils.getLongLongInvalidVertexData(env); + DataSet> edges = TestGraphUtils.getLongLongEdgeData(env); + + Graph graph = new Graph(vertices, edges, env); + DataSet result = graph.validate(new InvalidVertexIdsValidator()); + result.writeAsText(resultPath); + env.execute(); + + return "false\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } } } + } - -} diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 9e5ebb20610f7..ac2adc6a436ba 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -35,6 +35,21 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } + /** + * A graph with invalid vertex ids + */ + public static final DataSet> getLongLongInvalidVertexData( + ExecutionEnvironment env) { + List> vertices = new ArrayList>(); + vertices.add(new Vertex(15L, 1L)); + vertices.add(new Vertex(2L, 2L)); + vertices.add(new Vertex(3L, 3L)); + vertices.add(new Vertex(4L, 4L)); + vertices.add(new Vertex(5L, 5L)); + + return env.fromCollection(vertices); + } + /** * A graph that has at least one vertex with no ingoing/outgoing edges */ From 3afe2932663e8705314dc57a03aba6106e5ba999 Mon Sep 17 00:00:00 2001 From: andralungu Date: Tue, 9 Dec 2014 23:25:40 +0100 Subject: [PATCH 062/112] integrated inline suggestions --- src/main/java/flink/graphs/Graph.java | 7 +++---- src/main/java/flink/graphs/GraphValidator.java | 3 ++- src/main/java/flink/graphs/InvalidVertexIdsValidator.java | 5 +++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 3fa5b826453bb..a611b8c6b372a 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -88,11 +88,10 @@ public Graph(DataSet> vertices, DataSet> edges, Execut /** * Function that checks whether a graph's ids are valid * @return - */ - public & Serializable, VV extends Serializable, EV extends Serializable> DataSet - validate(GraphValidator validator) throws Exception { + */ + public DataSet validate(GraphValidator validator) { - return validator.validate((Graph) this); + return validator.validate(this); } public DataSet> getVertices() { diff --git a/src/main/java/flink/graphs/GraphValidator.java b/src/main/java/flink/graphs/GraphValidator.java index 74499c6278dae..5554787bed605 100644 --- a/src/main/java/flink/graphs/GraphValidator.java +++ b/src/main/java/flink/graphs/GraphValidator.java @@ -9,9 +9,10 @@ * @param * @param */ +@SuppressWarnings("serial") public abstract class GraphValidator & Serializable, VV extends Serializable, EV extends Serializable> implements Serializable{ - public abstract DataSet validate(Graph graph) throws Exception; + public abstract DataSet validate(Graph graph); } \ No newline at end of file diff --git a/src/main/java/flink/graphs/InvalidVertexIdsValidator.java b/src/main/java/flink/graphs/InvalidVertexIdsValidator.java index b5c3e2536a766..1a7a5aad81611 100644 --- a/src/main/java/flink/graphs/InvalidVertexIdsValidator.java +++ b/src/main/java/flink/graphs/InvalidVertexIdsValidator.java @@ -10,6 +10,7 @@ import java.io.Serializable; +@SuppressWarnings("serial") public class InvalidVertexIdsValidator & Serializable, VV extends Serializable, EV extends Serializable> extends GraphValidator { @@ -20,12 +21,12 @@ public class InvalidVertexIdsValidator & Serializable, V * with respect to its vertex ids. */ @Override - public DataSet validate(Graph graph) throws Exception { + public DataSet validate(Graph graph) { DataSet> edgeIds = graph.getEdges().flatMap(new MapEdgeIds()).distinct(); DataSet invalidIds = graph.getVertices().coGroup(edgeIds).where(0).equalTo(0) .with(new GroupInvalidIds()).first(1); - return GraphUtils.count(invalidIds.map(new KToTupleMap()), ExecutionEnvironment.getExecutionEnvironment()) + return GraphUtils.count(invalidIds.map(new KToTupleMap()), ExecutionEnvironment.getExecutionEnvironment()) .map(new InvalidIdsMap()); } From 7d11c20e685c0a01049dd1d3d48ae3bc772ce3c7 Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 5 Dec 2014 15:52:42 +0100 Subject: [PATCH 063/112] Added PageRank example --- .../java/flink/graphs/example/PageRank.java | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 src/main/java/flink/graphs/example/PageRank.java diff --git a/src/main/java/flink/graphs/example/PageRank.java b/src/main/java/flink/graphs/example/PageRank.java new file mode 100644 index 0000000000000..544408ea97935 --- /dev/null +++ b/src/main/java/flink/graphs/example/PageRank.java @@ -0,0 +1,103 @@ +package flink.graphs.example; + + +import flink.graphs.*; +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.FlatMapFunction; +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.spargel.java.MessageIterator; +import org.apache.flink.spargel.java.MessagingFunction; +import org.apache.flink.spargel.java.OutgoingEdge; +import org.apache.flink.spargel.java.VertexUpdateFunction; +import org.apache.flink.util.Collector; + + +public class PageRank implements ProgramDescription { + + private static final double BETA = 0.85; + + + public static void main (String [] args) throws Exception { + + final int numVertices = 10; + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> pages = env.generateSequence(1, numVertices) + .map(new MapFunction>() { + @Override + public Vertex map(Long l) throws Exception { + return new Vertex(l, 1.0 / numVertices); + } + }); + + DataSet> links = env.generateSequence(1, numVertices) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(Long key, Collector> out) throws Exception { + int numOutEdges = (int) (Math.random() * (numVertices / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numVertices) + 1; + out.collect(new Edge(key, target, 1.0/numOutEdges)); + } + } + }); + + Graph network = new Graph(pages, links, env); + + network.runVertexCentricIteration(new VertexRankUpdater(numVertices, BETA), new RankMessenger(), 60) + .getVertices() + .print(); + + env.execute(); + } + + /** + * Function that updates the rank of a vertex by summing up the partial ranks from all incoming messages + * and then applying the dampening formula. + */ + public static final class VertexRankUpdater extends VertexUpdateFunction { + + private final long numVertices; + private final double beta; + + public VertexRankUpdater(long numVertices, double beta) { + this.numVertices = numVertices; + this.beta = beta; + } + + @Override + public void updateVertex(Long vertexKey, Double vertexValue, MessageIterator inMessages) { + double rankSum = 0.0; + for (double msg : inMessages) { + rankSum += msg; + } + + // apply the dampening factor / random jump + double newRank = (beta * rankSum) + (1-BETA)/numVertices; + setNewVertexValue(newRank); + } + } + + /** + * Distributes the rank of a vertex among all target vertices according to the transition probability, + * which is associated with an edge as the edge value. + */ + public static final class RankMessenger extends MessagingFunction { + + @Override + public void sendMessages(Long vertexId, Double newRank) { + for (OutgoingEdge edge : getOutgoingEdges()) { + sendMessageTo(edge.target(), newRank * edge.edgeValue()); + } + } + } + + + @Override + public String getDescription() { + return "PageRank"; + } +} From 16ad8c051ca1f417b08b2e0df73aa552cce8aaee Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Fri, 5 Dec 2014 16:28:45 +0100 Subject: [PATCH 064/112] Added PageRank as a library method and example --- src/main/java/flink/graphs/GraphLib.java | 67 +++++++++++++++++++ .../java/flink/graphs/example/PageRank.java | 48 +------------ 2 files changed, 70 insertions(+), 45 deletions(-) create mode 100644 src/main/java/flink/graphs/GraphLib.java diff --git a/src/main/java/flink/graphs/GraphLib.java b/src/main/java/flink/graphs/GraphLib.java new file mode 100644 index 0000000000000..6560dcf713d42 --- /dev/null +++ b/src/main/java/flink/graphs/GraphLib.java @@ -0,0 +1,67 @@ +package flink.graphs; + + +import flink.graphs.Graph; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.spargel.java.MessageIterator; +import org.apache.flink.spargel.java.MessagingFunction; +import org.apache.flink.spargel.java.OutgoingEdge; +import org.apache.flink.spargel.java.VertexUpdateFunction; + +import java.io.Serializable; + +public class GraphLib { + + + /** + * Function that updates the rank of a vertex by summing up the partial ranks from all incoming messages + * and then applying the dampening formula. + */ + public static final class VertexRankUpdater & Serializable> extends VertexUpdateFunction { + + private final long numVertices; + private final double beta; + + public VertexRankUpdater(long numVertices, double beta) { + this.numVertices = numVertices; + this.beta = beta; + } + + @Override + public void updateVertex(K vertexKey, Double vertexValue, MessageIterator inMessages) { + double rankSum = 0.0; + for (double msg : inMessages) { + rankSum += msg; + } + + // apply the dampening factor / random jump + double newRank = (beta * rankSum) + (1-beta)/numVertices; + setNewVertexValue(newRank); + } + } + + /** + * Distributes the rank of a vertex among all target vertices according to the transition probability, + * which is associated with an edge as the edge value. + */ + public static final class RankMessenger & Serializable> extends MessagingFunction { + + @Override + public void sendMessages(K vertexId, Double newRank) { + for (OutgoingEdge edge : getOutgoingEdges()) { + sendMessageTo(edge.target(), newRank * edge.edgeValue()); + } + } + } + + //TODO Get numVertices from graph as long when this is possible (https://github.com/apache/incubator-flink/pull/210) + public static & Serializable> Graph pageRank (Graph network, + long numVertices, double beta, int maxIterations) { + + return network.runVertexCentricIteration( + new VertexRankUpdater(numVertices, beta), + new RankMessenger(), + maxIterations + ); + } +} diff --git a/src/main/java/flink/graphs/example/PageRank.java b/src/main/java/flink/graphs/example/PageRank.java index 544408ea97935..56e91293385a8 100644 --- a/src/main/java/flink/graphs/example/PageRank.java +++ b/src/main/java/flink/graphs/example/PageRank.java @@ -47,55 +47,13 @@ public void flatMap(Long key, Collector> out) throws Exceptio Graph network = new Graph(pages, links, env); - network.runVertexCentricIteration(new VertexRankUpdater(numVertices, BETA), new RankMessenger(), 60) - .getVertices() - .print(); + DataSet> verticesWithRanks = GraphLib.pageRank(network, numVertices, BETA, 60).getVertices(); - env.execute(); - } - - /** - * Function that updates the rank of a vertex by summing up the partial ranks from all incoming messages - * and then applying the dampening formula. - */ - public static final class VertexRankUpdater extends VertexUpdateFunction { - - private final long numVertices; - private final double beta; - - public VertexRankUpdater(long numVertices, double beta) { - this.numVertices = numVertices; - this.beta = beta; - } - - @Override - public void updateVertex(Long vertexKey, Double vertexValue, MessageIterator inMessages) { - double rankSum = 0.0; - for (double msg : inMessages) { - rankSum += msg; - } - - // apply the dampening factor / random jump - double newRank = (beta * rankSum) + (1-BETA)/numVertices; - setNewVertexValue(newRank); - } - } + verticesWithRanks.print(); - /** - * Distributes the rank of a vertex among all target vertices according to the transition probability, - * which is associated with an edge as the edge value. - */ - public static final class RankMessenger extends MessagingFunction { - - @Override - public void sendMessages(Long vertexId, Double newRank) { - for (OutgoingEdge edge : getOutgoingEdges()) { - sendMessageTo(edge.target(), newRank * edge.edgeValue()); - } - } + env.execute(); } - @Override public String getDescription() { return "PageRank"; From 8022ae86c486a1998d94fb64f8d20455507b567d Mon Sep 17 00:00:00 2001 From: Kostas Tzoumas Date: Wed, 10 Dec 2014 11:39:31 +0100 Subject: [PATCH 065/112] PageRank example implementing GraphAlgorithm --- src/main/java/flink/graphs/Graph.java | 12 ++- .../java/flink/graphs/GraphAlgorithm.java | 11 +++ .../java/flink/graphs/example/PageRank.java | 61 --------------- .../flink/graphs/example/PageRankExample.java | 74 +++++++++++++++++++ .../{GraphLib.java => library/PageRank.java} | 36 +++++---- 5 files changed, 115 insertions(+), 79 deletions(-) create mode 100644 src/main/java/flink/graphs/GraphAlgorithm.java delete mode 100644 src/main/java/flink/graphs/example/PageRank.java create mode 100644 src/main/java/flink/graphs/example/PageRankExample.java rename src/main/java/flink/graphs/{GraphLib.java => library/PageRank.java} (78%) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index a611b8c6b372a..213c6ab8cace4 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -622,8 +622,8 @@ public Graph addVertex (final Vertex vertex, List> * @return */ public Graph addEdge (Vertex source, Vertex target, EV edgeValue) { - Graph partialGraph = this.fromCollection(Arrays.asList(source, target), - Arrays.asList(new Edge(source.f0, target.f0, edgeValue))); + Graph partialGraph = this.fromCollection(Arrays.asList(source, target), + Arrays.asList(new Edge(source.f0, target.f0, edgeValue))); return this.union(partialGraph); } @@ -725,8 +725,8 @@ public Graph union (Graph graph) { public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, MessagingFunction messagingFunction, int maximumNumberOfIterations) { DataSet> newVertices = vertices.map(new VertexToTuple2Map()).runOperation( - VertexCentricIteration.withValuedEdges(edges.map(new EdgeToTuple3Map()), - vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); + VertexCentricIteration.withValuedEdges(edges.map(new EdgeToTuple3Map()), + vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); return new Graph(newVertices.map(new Tuple2ToVertexMap()), edges, context); } @@ -776,4 +776,8 @@ EV extends Serializable> Graph fromCollection(ExecutionEnvironment en return Graph.create(edges, mapper, env); } + public Graph run (GraphAlgorithm algorithm) { + return algorithm.run(this); + } + } diff --git a/src/main/java/flink/graphs/GraphAlgorithm.java b/src/main/java/flink/graphs/GraphAlgorithm.java new file mode 100644 index 0000000000000..c2cbd71793c48 --- /dev/null +++ b/src/main/java/flink/graphs/GraphAlgorithm.java @@ -0,0 +1,11 @@ +package flink.graphs; + + +import java.io.Serializable; + +public interface GraphAlgorithm & Serializable, VV extends Serializable, + EV extends Serializable> { + + public Graph run (Graph input); + +} diff --git a/src/main/java/flink/graphs/example/PageRank.java b/src/main/java/flink/graphs/example/PageRank.java deleted file mode 100644 index 56e91293385a8..0000000000000 --- a/src/main/java/flink/graphs/example/PageRank.java +++ /dev/null @@ -1,61 +0,0 @@ -package flink.graphs.example; - - -import flink.graphs.*; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.FlatMapFunction; -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.spargel.java.MessageIterator; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.OutgoingEdge; -import org.apache.flink.spargel.java.VertexUpdateFunction; -import org.apache.flink.util.Collector; - - -public class PageRank implements ProgramDescription { - - private static final double BETA = 0.85; - - - public static void main (String [] args) throws Exception { - - final int numVertices = 10; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> pages = env.generateSequence(1, numVertices) - .map(new MapFunction>() { - @Override - public Vertex map(Long l) throws Exception { - return new Vertex(l, 1.0 / numVertices); - } - }); - - DataSet> links = env.generateSequence(1, numVertices) - .flatMap(new FlatMapFunction>() { - @Override - public void flatMap(Long key, Collector> out) throws Exception { - int numOutEdges = (int) (Math.random() * (numVertices / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numVertices) + 1; - out.collect(new Edge(key, target, 1.0/numOutEdges)); - } - } - }); - - Graph network = new Graph(pages, links, env); - - DataSet> verticesWithRanks = GraphLib.pageRank(network, numVertices, BETA, 60).getVertices(); - - verticesWithRanks.print(); - - env.execute(); - } - - @Override - public String getDescription() { - return "PageRank"; - } -} diff --git a/src/main/java/flink/graphs/example/PageRankExample.java b/src/main/java/flink/graphs/example/PageRankExample.java new file mode 100644 index 0000000000000..48a7b153e9fad --- /dev/null +++ b/src/main/java/flink/graphs/example/PageRankExample.java @@ -0,0 +1,74 @@ +package flink.graphs.example; + + +import flink.graphs.*; +import flink.graphs.library.PageRank; +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.*; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; + +import java.util.Collection; +import java.util.List; + + +public class PageRankExample implements ProgramDescription { + + public static void main (String [] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> pages = getPagesDataSet(env); + + DataSet> links = getLinksDataSet(env); + + Graph network = new Graph(pages, links, env); + + DataSet> pageRanks = + network.run(new PageRank(numPages, DAMPENING_FACTOR, maxIterations)).getVertices(); + + pageRanks.print(); + + env.execute(); + } + + @Override + public String getDescription() { + return "PageRank"; + } + + private static final double DAMPENING_FACTOR = 0.85; + private static long numPages = 10; + private static int maxIterations = 10; + + + private static DataSet> getPagesDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numPages) + .map(new MapFunction>() { + @Override + public Vertex map(Long l) throws Exception { + return new Vertex(l, 1.0 / numPages); + } + }); + + } + + private static DataSet> getLinksDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numPages) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(Long key, Collector> out) throws Exception { + int numOutEdges = (int) (Math.random() * (numPages / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numPages) + 1; + out.collect(new Edge(key, target, 1.0 / numOutEdges)); + } + } + }); + } +} diff --git a/src/main/java/flink/graphs/GraphLib.java b/src/main/java/flink/graphs/library/PageRank.java similarity index 78% rename from src/main/java/flink/graphs/GraphLib.java rename to src/main/java/flink/graphs/library/PageRank.java index 6560dcf713d42..5d711e6d2b746 100644 --- a/src/main/java/flink/graphs/GraphLib.java +++ b/src/main/java/flink/graphs/library/PageRank.java @@ -1,8 +1,8 @@ -package flink.graphs; +package flink.graphs.library; import flink.graphs.Graph; -import org.apache.flink.api.java.DataSet; +import flink.graphs.GraphAlgorithm; import org.apache.flink.spargel.java.MessageIterator; import org.apache.flink.spargel.java.MessagingFunction; import org.apache.flink.spargel.java.OutgoingEdge; @@ -10,7 +10,26 @@ import java.io.Serializable; -public class GraphLib { +public class PageRank & Serializable> implements GraphAlgorithm { + + private long numVertices; + private double beta; + private int maxIterations; + + public PageRank(long numVertices, double beta, int maxIterations) { + this.numVertices = numVertices; + this.beta = beta; + this.maxIterations = maxIterations; + } + + @Override + public Graph run(Graph network) { + return network.runVertexCentricIteration( + new VertexRankUpdater(numVertices, beta), + new RankMessenger(), + maxIterations + ); + } /** @@ -53,15 +72,4 @@ public void sendMessages(K vertexId, Double newRank) { } } } - - //TODO Get numVertices from graph as long when this is possible (https://github.com/apache/incubator-flink/pull/210) - public static & Serializable> Graph pageRank (Graph network, - long numVertices, double beta, int maxIterations) { - - return network.runVertexCentricIteration( - new VertexRankUpdater(numVertices, beta), - new RankMessenger(), - maxIterations - ); - } } From f31e7b837664888209bd8e302aaf6f7dcd6e4294 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 27 Nov 2014 00:11:59 +0100 Subject: [PATCH 066/112] first take on neighborhood methods: OutEdges --- src/main/java/flink/graphs/Graph.java | 39 +++++++ .../java/flink/graphs/OutEdgesFunction.java | 13 +++ .../java/flink/graphs/TestForeachEdge.java | 103 ++++++++++++++++++ 3 files changed, 155 insertions(+) create mode 100644 src/main/java/flink/graphs/OutEdgesFunction.java create mode 100644 src/test/java/flink/graphs/TestForeachEdge.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 213c6ab8cace4..5c1c817f6ca28 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -323,6 +323,45 @@ public Graph getUndirected() throws UnsupportedOperationException { return new Graph(vertices, undirectedEdges, this.context, true); } } + + /** + * Utility function that allows each vertex of the graph + * to access its out-neighbors + * @param edgesFunction the function to apply to the neighborhood + * @return + */ + public DataSet foreachEdge(OutEdgesFunction edgesFunction) { + return vertices.coGroup(edges).where(0).equalTo(0).with( + new ApplyCoGroupFunction(edgesFunction)); + } + + /** + * + * @param + * @param + * @param + * @param + */ + private static final class ApplyCoGroupFunction & Serializable, + VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple3, T>, + ResultTypeQueryable { + + private OutEdgesFunction function; + + public ApplyCoGroupFunction (OutEdgesFunction fun) { + this.function = fun; + } + public void coGroup(Iterable> vertex, + Iterable> outEdges, Collector out) throws Exception { + out.collect(function.iterateOutEdges(vertex.iterator().next(), outEdges)); + } + @Override + public TypeInformation getProducedType() { + // TODO Auto-generated method stub + return null; + } + } @ConstantFields("0->1;1->0;2->2") private static final class ReverseEdgesMap & Serializable, diff --git a/src/main/java/flink/graphs/OutEdgesFunction.java b/src/main/java/flink/graphs/OutEdgesFunction.java new file mode 100644 index 0000000000000..a7e4d6e76e6af --- /dev/null +++ b/src/main/java/flink/graphs/OutEdgesFunction.java @@ -0,0 +1,13 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; + +public interface OutEdgesFunction & Serializable, + VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { + + O iterateOutEdges(Tuple2 v, Iterable> outEdges) throws Exception; +} diff --git a/src/test/java/flink/graphs/TestForeachEdge.java b/src/test/java/flink/graphs/TestForeachEdge.java new file mode 100644 index 0000000000000..8f84ace7efdae --- /dev/null +++ b/src/test/java/flink/graphs/TestForeachEdge.java @@ -0,0 +1,103 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestForeachEdge extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 1; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestForeachEdge(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Get the lowest-weight out-neighbor + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env)); + + DataSet> verticesWithLowestOutNeighbor = + graph.foreachEdge(new OutEdgesFunction>() { + + public Tuple2 iterateOutEdges( + Tuple2 v, + Iterable> outEdges) { + + long weight = Long.MAX_VALUE; + long minNeighorId = 0; + + for (Tuple3 edge: outEdges) { + if (edge.f2 < weight) { + weight = edge.f2; + minNeighorId = edge.f1; + } + } + return new Tuple2(v.f0, minNeighorId); + } + }); + env.execute(); + return "something"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + +} From 66ffa62067a7cd71bceacc930ea0de29cce3fb93 Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 9 Dec 2014 20:09:21 +0100 Subject: [PATCH 067/112] fixed type info for the outEdgesFunction --- src/main/java/flink/graphs/Graph.java | 22 +++++++++---------- .../java/flink/graphs/OutEdgesFunction.java | 3 +-- .../java/flink/graphs/TestForeachEdge.java | 15 ++++++++----- 3 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 5c1c817f6ca28..2cb50f57f5dc6 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -326,11 +326,11 @@ public Graph getUndirected() throws UnsupportedOperationException { /** * Utility function that allows each vertex of the graph - * to access its out-neighbors + * to access its out-edges * @param edgesFunction the function to apply to the neighborhood - * @return + * @return a dataset of a Tuple2 with the vertex id and the computed value */ - public DataSet foreachEdge(OutEdgesFunction edgesFunction) { + public DataSet> foreachEdge(OutEdgesFunction edgesFunction) { return vertices.coGroup(edges).where(0).equalTo(0).with( new ApplyCoGroupFunction(edgesFunction)); } @@ -344,23 +344,23 @@ public DataSet foreachEdge(OutEdgesFunction edgesFunction) */ private static final class ApplyCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple3, T>, - ResultTypeQueryable { + implements CoGroupFunction, Edge, Tuple2>, + ResultTypeQueryable> { private OutEdgesFunction function; public ApplyCoGroupFunction (OutEdgesFunction fun) { this.function = fun; } - public void coGroup(Iterable> vertex, - Iterable> outEdges, Collector out) throws Exception { + public void coGroup(Iterable> vertex, + Iterable> outEdges, Collector> out) throws Exception { out.collect(function.iterateOutEdges(vertex.iterator().next(), outEdges)); } @Override - public TypeInformation getProducedType() { - // TODO Auto-generated method stub - return null; - } + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(OutEdgesFunction.class, function.getClass(), 3, null, null)); + } } @ConstantFields("0->1;1->0;2->2") diff --git a/src/main/java/flink/graphs/OutEdgesFunction.java b/src/main/java/flink/graphs/OutEdgesFunction.java index a7e4d6e76e6af..9784b1924a410 100644 --- a/src/main/java/flink/graphs/OutEdgesFunction.java +++ b/src/main/java/flink/graphs/OutEdgesFunction.java @@ -4,10 +4,9 @@ import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; public interface OutEdgesFunction & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - O iterateOutEdges(Tuple2 v, Iterable> outEdges) throws Exception; + Tuple2 iterateOutEdges(Vertex v, Iterable> outEdges) throws Exception; } diff --git a/src/test/java/flink/graphs/TestForeachEdge.java b/src/test/java/flink/graphs/TestForeachEdge.java index 8f84ace7efdae..4e17cc600081c 100644 --- a/src/test/java/flink/graphs/TestForeachEdge.java +++ b/src/test/java/flink/graphs/TestForeachEdge.java @@ -70,14 +70,14 @@ public static String runProgram(int progId, String resultPath) throws Exception */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), - TestGraphUtils.getLongLongEdgeData(env)); + TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.foreachEdge(new OutEdgesFunction>() { + graph.foreachEdge(new OutEdgesFunction() { public Tuple2 iterateOutEdges( - Tuple2 v, - Iterable> outEdges) { + Vertex v, + Iterable> outEdges) { long weight = Long.MAX_VALUE; long minNeighorId = 0; @@ -91,8 +91,13 @@ public Tuple2 iterateOutEdges( return new Tuple2(v.f0, minNeighorId); } }); + verticesWithLowestOutNeighbor.writeAsCsv(resultPath); env.execute(); - return "something"; + return "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,1\n"; } default: throw new IllegalArgumentException("Invalid program id"); From f2082f8c1bd4ad8e6930c456695a8c05a569720b Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 9 Dec 2014 20:18:52 +0100 Subject: [PATCH 068/112] use vertex/edge instead of tuples in the test --- src/test/java/flink/graphs/TestForeachEdge.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/test/java/flink/graphs/TestForeachEdge.java b/src/test/java/flink/graphs/TestForeachEdge.java index 4e17cc600081c..34f121fb0beaa 100644 --- a/src/test/java/flink/graphs/TestForeachEdge.java +++ b/src/test/java/flink/graphs/TestForeachEdge.java @@ -8,7 +8,6 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -82,13 +81,13 @@ public Tuple2 iterateOutEdges( long weight = Long.MAX_VALUE; long minNeighorId = 0; - for (Tuple3 edge: outEdges) { - if (edge.f2 < weight) { - weight = edge.f2; - minNeighorId = edge.f1; + for (Edge edge: outEdges) { + if (edge.getValue() < weight) { + weight = edge.getValue(); + minNeighorId = edge.getTarget(); } } - return new Tuple2(v.f0, minNeighorId); + return new Tuple2(v.getId(), minNeighorId); } }); verticesWithLowestOutNeighbor.writeAsCsv(resultPath); From 9a25461e7b5c91b81cb383e2ad5643fd5d0f81aa Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 13 Dec 2014 14:28:20 +0100 Subject: [PATCH 069/112] fixed a few warnings in pagerank example --- src/main/java/flink/graphs/Graph.java | 5 ++--- .../flink/graphs/example/PageRankExample.java | 16 +++++----------- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 2cb50f57f5dc6..9433eb22c2830 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -10,8 +10,7 @@ * 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,K extends Serializablr & Comparable, - VV implements Serializable, + * 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. @@ -815,7 +814,7 @@ EV extends Serializable> Graph fromCollection(ExecutionEnvironment en return Graph.create(edges, mapper, env); } - public Graph run (GraphAlgorithm algorithm) { + public Graph run (GraphAlgorithm algorithm) { return algorithm.run(this); } diff --git a/src/main/java/flink/graphs/example/PageRankExample.java b/src/main/java/flink/graphs/example/PageRankExample.java index 48a7b153e9fad..0fc80849b464c 100644 --- a/src/main/java/flink/graphs/example/PageRankExample.java +++ b/src/main/java/flink/graphs/example/PageRankExample.java @@ -3,20 +3,13 @@ import flink.graphs.*; import flink.graphs.library.PageRank; + import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import java.util.Collection; -import java.util.List; - - public class PageRankExample implements ProgramDescription { public static void main (String [] args) throws Exception { @@ -27,7 +20,7 @@ public static void main (String [] args) throws Exception { DataSet> links = getLinksDataSet(env); - Graph network = new Graph(pages, links, env); + Graph network = new Graph(pages, links, env); DataSet> pageRanks = network.run(new PageRank(numPages, DAMPENING_FACTOR, maxIterations)).getVertices(); @@ -46,8 +39,8 @@ public String getDescription() { private static long numPages = 10; private static int maxIterations = 10; - - private static DataSet> getPagesDataSet(ExecutionEnvironment env) { + @SuppressWarnings("serial") + private static DataSet> getPagesDataSet(ExecutionEnvironment env) { return env.generateSequence(1, numPages) .map(new MapFunction>() { @Override @@ -58,6 +51,7 @@ public Vertex map(Long l) throws Exception { } + @SuppressWarnings("serial") private static DataSet> getLinksDataSet(ExecutionEnvironment env) { return env.generateSequence(1, numPages) .flatMap(new FlatMapFunction>() { From fd58606d2dbb8bff363bdd8af7198f6aa956ce48 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 13 Dec 2014 15:19:56 +0100 Subject: [PATCH 070/112] some refactoring and cleanup --- src/main/java/flink/graphs/Graph.java | 6 ++++ .../flink/graphs/example/GraphMetrics.java | 27 +++++--------- .../graphs/example/utils/ExampleUtils.java | 35 +++++++++++++++++++ .../java/flink/graphs/library/PageRank.java | 7 ++-- .../graphs/{ => utils}/EdgeToTuple3Map.java | 4 ++- .../flink/graphs/{ => utils}/GraphUtils.java | 2 +- .../graphs/{ => utils}/Tuple2ToVertexMap.java | 4 ++- .../graphs/{ => utils}/VertexToTuple2Map.java | 4 ++- .../{ => validation}/GraphValidator.java | 5 ++- .../InvalidVertexIdsValidator.java | 7 +++- .../java/flink/graphs/TestGraphCreation.java | 1 + 11 files changed, 76 insertions(+), 26 deletions(-) rename src/main/java/flink/graphs/{ => utils}/EdgeToTuple3Map.java (89%) rename src/main/java/flink/graphs/{ => utils}/GraphUtils.java (97%) rename src/main/java/flink/graphs/{ => utils}/Tuple2ToVertexMap.java (88%) rename src/main/java/flink/graphs/{ => utils}/VertexToTuple2Map.java (88%) rename src/main/java/flink/graphs/{ => validation}/GraphValidator.java (87%) rename src/main/java/flink/graphs/{ => validation}/InvalidVertexIdsValidator.java (94%) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 9433eb22c2830..b1d41e6028a49 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -49,6 +49,12 @@ import org.apache.flink.util.Collector; import org.apache.flink.types.NullValue; +import flink.graphs.utils.EdgeToTuple3Map; +import flink.graphs.utils.GraphUtils; +import flink.graphs.utils.Tuple2ToVertexMap; +import flink.graphs.utils.VertexToTuple2Map; +import flink.graphs.validation.GraphValidator; + @SuppressWarnings("serial") public class Graph & Serializable, VV extends Serializable, EV extends Serializable> implements Serializable { diff --git a/src/main/java/flink/graphs/example/GraphMetrics.java b/src/main/java/flink/graphs/example/GraphMetrics.java index b1c0a77429416..68ca82aa76099 100644 --- a/src/main/java/flink/graphs/example/GraphMetrics.java +++ b/src/main/java/flink/graphs/example/GraphMetrics.java @@ -8,15 +8,11 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.testdata.ConnectedComponentsData; import org.apache.flink.types.NullValue; -import flink.graphs.Edge; import flink.graphs.Graph; -import flink.graphs.example.utils.EdgeWithLongIdNullValueParser; import flink.graphs.example.utils.ExampleUtils; /** @@ -31,8 +27,7 @@ * */ public class GraphMetrics implements ProgramDescription { - - static final int NUM_EDGES = 1000; + static final int NUM_VERTICES = 100; static final long SEED = 9876; @@ -46,13 +41,9 @@ public static void main(String[] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - /** create random input edges **/ - DataSet edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED).split("\n")); - DataSet> edges = edgeString.map(new EdgeWithLongIdNullValueParser()); - - /** create the graph from the edges **/ - Graph graph = Graph.create(edges, env); + /** create a random graph **/ + Graph graph = Graph.create(ExampleUtils + .getRandomEdges(env, NUM_VERTICES), env); /** get the number of vertices **/ DataSet numVertices = graph.numberOfVertices(); @@ -63,8 +54,8 @@ public static void main(String[] args) throws Exception { /** compute the average node degree **/ DataSet> verticesWithDegrees = graph.getDegrees(); - DataSet avgNodeDegree = verticesWithDegrees.project(1).types(Long.class) - .aggregate(Aggregations.SUM, 0).map(new AvgNodeDegreeMapper()) + DataSet avgNodeDegree = verticesWithDegrees + .aggregate(Aggregations.SUM, 1).map(new AvgNodeDegreeMapper()) .withBroadcastSet(numVertices, "numberOfVertices"); /** find the vertex with the maximum in-degree **/ @@ -92,7 +83,7 @@ public static void main(String[] args) throws Exception { } @SuppressWarnings("serial") - private static final class AvgNodeDegreeMapper extends RichMapFunction, Double> { + private static final class AvgNodeDegreeMapper extends RichMapFunction, Double> { private int numberOfVertices; @@ -103,8 +94,8 @@ public void open(Configuration parameters) throws Exception { numberOfVertices = bCastSet.iterator().next(); } - public Double map(Tuple1 sum) { - return (double) (sum.f0 / numberOfVertices) ; + public Double map(Tuple2 sumTuple) { + return (double) (sumTuple.f1 / numberOfVertices) ; } } diff --git a/src/main/java/flink/graphs/example/utils/ExampleUtils.java b/src/main/java/flink/graphs/example/utils/ExampleUtils.java index 9d3b44b1b020b..c4ff43edcf75d 100644 --- a/src/main/java/flink/graphs/example/utils/ExampleUtils.java +++ b/src/main/java/flink/graphs/example/utils/ExampleUtils.java @@ -2,10 +2,17 @@ import java.io.PrintStream; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.NullValue; +import org.apache.flink.util.Collector; + +import flink.graphs.Edge; +import flink.graphs.Vertex; public class ExampleUtils { @@ -69,4 +76,32 @@ public String toString() { @Override public void configure(Configuration parameters) {} } + + @SuppressWarnings("serial") + public static DataSet> getVertexIds(ExecutionEnvironment env, + final long numVertices) { + return env.generateSequence(1, numVertices) + .map(new MapFunction>() { + public Vertex map(Long l) { + return new Vertex(l, NullValue.getInstance()); + } + }); + } + + @SuppressWarnings("serial") + public static DataSet> getRandomEdges(ExecutionEnvironment env, + final long numVertices) { + return env.generateSequence(1, numVertices) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(Long key, Collector> out) throws Exception { + int numOutEdges = (int) (Math.random() * (numVertices / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numVertices) + 1; + out.collect(new Edge(key, target, NullValue.getInstance())); + } + } + }); + } } + diff --git a/src/main/java/flink/graphs/library/PageRank.java b/src/main/java/flink/graphs/library/PageRank.java index 5d711e6d2b746..7ab9964bb5dbc 100644 --- a/src/main/java/flink/graphs/library/PageRank.java +++ b/src/main/java/flink/graphs/library/PageRank.java @@ -3,6 +3,7 @@ import flink.graphs.Graph; import flink.graphs.GraphAlgorithm; + import org.apache.flink.spargel.java.MessageIterator; import org.apache.flink.spargel.java.MessagingFunction; import org.apache.flink.spargel.java.OutgoingEdge; @@ -36,7 +37,8 @@ public Graph run(Graph network) { * Function that updates the rank of a vertex by summing up the partial ranks from all incoming messages * and then applying the dampening formula. */ - public static final class VertexRankUpdater & Serializable> extends VertexUpdateFunction { + @SuppressWarnings("serial") + public static final class VertexRankUpdater & Serializable> extends VertexUpdateFunction { private final long numVertices; private final double beta; @@ -63,7 +65,8 @@ public void updateVertex(K vertexKey, Double vertexValue, MessageIterator & Serializable> extends MessagingFunction { + @SuppressWarnings("serial") + public static final class RankMessenger & Serializable> extends MessagingFunction { @Override public void sendMessages(K vertexId, Double newRank) { diff --git a/src/main/java/flink/graphs/EdgeToTuple3Map.java b/src/main/java/flink/graphs/utils/EdgeToTuple3Map.java similarity index 89% rename from src/main/java/flink/graphs/EdgeToTuple3Map.java rename to src/main/java/flink/graphs/utils/EdgeToTuple3Map.java index 2ec75c29ecdcc..86103a6aee705 100644 --- a/src/main/java/flink/graphs/EdgeToTuple3Map.java +++ b/src/main/java/flink/graphs/utils/EdgeToTuple3Map.java @@ -1,10 +1,12 @@ -package flink.graphs; +package flink.graphs.utils; import java.io.Serializable; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple3; +import flink.graphs.Edge; + public class EdgeToTuple3Map & Serializable, EV extends Serializable> implements MapFunction, Tuple3> { diff --git a/src/main/java/flink/graphs/GraphUtils.java b/src/main/java/flink/graphs/utils/GraphUtils.java similarity index 97% rename from src/main/java/flink/graphs/GraphUtils.java rename to src/main/java/flink/graphs/utils/GraphUtils.java index 1790b4e469bcd..0b0dc18f25597 100644 --- a/src/main/java/flink/graphs/GraphUtils.java +++ b/src/main/java/flink/graphs/utils/GraphUtils.java @@ -1,4 +1,4 @@ -package flink.graphs; +package flink.graphs.utils; import java.util.ArrayList; import java.util.List; diff --git a/src/main/java/flink/graphs/Tuple2ToVertexMap.java b/src/main/java/flink/graphs/utils/Tuple2ToVertexMap.java similarity index 88% rename from src/main/java/flink/graphs/Tuple2ToVertexMap.java rename to src/main/java/flink/graphs/utils/Tuple2ToVertexMap.java index 7cc7c2477d856..893ae9598bcc1 100644 --- a/src/main/java/flink/graphs/Tuple2ToVertexMap.java +++ b/src/main/java/flink/graphs/utils/Tuple2ToVertexMap.java @@ -1,10 +1,12 @@ -package flink.graphs; +package flink.graphs.utils; import java.io.Serializable; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; +import flink.graphs.Vertex; + public class Tuple2ToVertexMap & Serializable, VV extends Serializable> implements MapFunction, Vertex> { diff --git a/src/main/java/flink/graphs/VertexToTuple2Map.java b/src/main/java/flink/graphs/utils/VertexToTuple2Map.java similarity index 88% rename from src/main/java/flink/graphs/VertexToTuple2Map.java rename to src/main/java/flink/graphs/utils/VertexToTuple2Map.java index 717e13c7953bf..30f867d8b5b25 100644 --- a/src/main/java/flink/graphs/VertexToTuple2Map.java +++ b/src/main/java/flink/graphs/utils/VertexToTuple2Map.java @@ -1,10 +1,12 @@ -package flink.graphs; +package flink.graphs.utils; import java.io.Serializable; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; +import flink.graphs.Vertex; + public class VertexToTuple2Map & Serializable, VV extends Serializable> implements MapFunction, Tuple2> { diff --git a/src/main/java/flink/graphs/GraphValidator.java b/src/main/java/flink/graphs/validation/GraphValidator.java similarity index 87% rename from src/main/java/flink/graphs/GraphValidator.java rename to src/main/java/flink/graphs/validation/GraphValidator.java index 5554787bed605..88dfcde08f4b7 100644 --- a/src/main/java/flink/graphs/GraphValidator.java +++ b/src/main/java/flink/graphs/validation/GraphValidator.java @@ -1,7 +1,10 @@ -package flink.graphs; +package flink.graphs.validation; import java.io.Serializable; + import org.apache.flink.api.java.DataSet; +import flink.graphs.Graph; + /** * A validation method for different types of Graphs * diff --git a/src/main/java/flink/graphs/InvalidVertexIdsValidator.java b/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java similarity index 94% rename from src/main/java/flink/graphs/InvalidVertexIdsValidator.java rename to src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java index 1a7a5aad81611..1b68abe71f6ee 100644 --- a/src/main/java/flink/graphs/InvalidVertexIdsValidator.java +++ b/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java @@ -1,4 +1,4 @@ -package flink.graphs; +package flink.graphs.validation; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.FlatMapFunction; @@ -8,6 +8,11 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.util.Collector; +import flink.graphs.Edge; +import flink.graphs.Graph; +import flink.graphs.Vertex; +import flink.graphs.utils.GraphUtils; + import java.io.Serializable; @SuppressWarnings("serial") diff --git a/src/test/java/flink/graphs/TestGraphCreation.java b/src/test/java/flink/graphs/TestGraphCreation.java index fc1305193b6a0..3bec4af170d87 100644 --- a/src/test/java/flink/graphs/TestGraphCreation.java +++ b/src/test/java/flink/graphs/TestGraphCreation.java @@ -18,6 +18,7 @@ import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; import flink.graphs.TestGraphUtils.DummyCustomType; +import flink.graphs.validation.InvalidVertexIdsValidator; @RunWith(Parameterized.class) public class TestGraphCreation extends JavaProgramTestBase { From 459be2b9d96f5ed53511bb7187246592bd7fa9ab Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 13 Dec 2014 16:37:50 +0100 Subject: [PATCH 071/112] reduceOnEdges method for in-/out- and all edges --- src/main/java/flink/graphs/EdgeDirection.java | 7 ++ ...tEdgesFunction.java => EdgesFunction.java} | 4 +- src/main/java/flink/graphs/Graph.java | 104 +++++++++++++++--- .../java/flink/graphs/TestGraphCreation.java | 8 +- ...eachEdge.java => TestNeighborMethods.java} | 87 +++++++++++++-- 5 files changed, 177 insertions(+), 33 deletions(-) create mode 100644 src/main/java/flink/graphs/EdgeDirection.java rename src/main/java/flink/graphs/{OutEdgesFunction.java => EdgesFunction.java} (58%) rename src/test/java/flink/graphs/{TestForeachEdge.java => TestNeighborMethods.java} (50%) diff --git a/src/main/java/flink/graphs/EdgeDirection.java b/src/main/java/flink/graphs/EdgeDirection.java new file mode 100644 index 0000000000000..5aa161587aeb8 --- /dev/null +++ b/src/main/java/flink/graphs/EdgeDirection.java @@ -0,0 +1,7 @@ +package flink.graphs; + +public enum EdgeDirection { + IN, + OUT, + ALL +} diff --git a/src/main/java/flink/graphs/OutEdgesFunction.java b/src/main/java/flink/graphs/EdgesFunction.java similarity index 58% rename from src/main/java/flink/graphs/OutEdgesFunction.java rename to src/main/java/flink/graphs/EdgesFunction.java index 9784b1924a410..1b44fa717b3f6 100644 --- a/src/main/java/flink/graphs/OutEdgesFunction.java +++ b/src/main/java/flink/graphs/EdgesFunction.java @@ -5,8 +5,8 @@ import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.java.tuple.Tuple2; -public interface OutEdgesFunction & Serializable, +public interface EdgesFunction & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateOutEdges(Vertex v, Iterable> outEdges) throws Exception; + Tuple2 iterateEdges(Vertex v, Iterable> edges) throws Exception; } diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index b1d41e6028a49..48b16026bdf1e 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.functions.CoGroupFunction; @@ -330,44 +331,111 @@ public Graph getUndirected() throws UnsupportedOperationException { } /** - * Utility function that allows each vertex of the graph - * to access its out-edges - * @param edgesFunction the function to apply to the neighborhood + * Compute an aggregate over the edges of each vertex. + * @param edgesFunction edgesFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) * @return a dataset of a Tuple2 with the vertex id and the computed value + * @throws IllegalArgumentException */ - public DataSet> foreachEdge(OutEdgesFunction edgesFunction) { - return vertices.coGroup(edges).where(0).equalTo(0).with( - new ApplyCoGroupFunction(edgesFunction)); + public DataSet> reduceOnEdges(EdgesFunction edgesFunction, + EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { + case IN: + return vertices.coGroup(edges).where(0).equalTo(1).with( + new ApplyCoGroupFunction(edgesFunction)); + case OUT: + return vertices.coGroup(edges).where(0).equalTo(0).with( + new ApplyCoGroupFunction(edgesFunction)); + case ALL: + return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode())) + .where(0).equalTo(0) + .with(new ApplyCoGroupFunctionOnAllEdges(edgesFunction)); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + + private static final class EmitOneEdgePerNode & Serializable, + VV extends Serializable, EV extends Serializable> implements FlatMapFunction< + Edge, Tuple2>> { + public void flatMap(Edge edge, Collector>> out) { + out.collect(new Tuple2>(edge.getSource(), edge)); + out.collect(new Tuple2>(edge.getTarget(), edge)); + } } - /** - * - * @param - * @param - * @param - * @param - */ private static final class ApplyCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> implements CoGroupFunction, Edge, Tuple2>, ResultTypeQueryable> { - private OutEdgesFunction function; + private EdgesFunction function; - public ApplyCoGroupFunction (OutEdgesFunction fun) { + public ApplyCoGroupFunction (EdgesFunction fun) { this.function = fun; } public void coGroup(Iterable> vertex, - Iterable> outEdges, Collector> out) throws Exception { - out.collect(function.iterateOutEdges(vertex.iterator().next(), outEdges)); + Iterable> edges, Collector> out) throws Exception { + out.collect(function.iterateEdges(vertex.iterator().next(), edges)); } @Override public TypeInformation> getProducedType() { return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(OutEdgesFunction.class, function.getClass(), 3, null, null)); + TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 3, null, null)); } } + private static final class ApplyCoGroupFunctionOnAllEdges & Serializable, + VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple2>, Tuple2>, + ResultTypeQueryable> { + + private EdgesFunction function; + + public ApplyCoGroupFunctionOnAllEdges (EdgesFunction fun) { + this.function = fun; + } + + public void coGroup(Iterable> vertex, + final Iterable>> keysWithEdges, Collector> out) + throws Exception { + + final Iterator> edgesIterator = new Iterator>() { + + final Iterator>> keysWithEdgesIterator = keysWithEdges.iterator(); + + @Override + public boolean hasNext() { + return keysWithEdgesIterator.hasNext(); + } + + @Override + public Edge next() { + return keysWithEdgesIterator.next().f1; + } + + @Override + public void remove() { + keysWithEdgesIterator.remove(); + } + }; + + Iterable> edgesIterable = new Iterable>() { + public Iterator> iterator() { + return edgesIterator; + } + }; + + out.collect(function.iterateEdges(vertex.iterator().next(), edgesIterable)); + } + + @Override + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 3, null, null)); + } +} + @ConstantFields("0->1;1->0;2->2") private static final class ReverseEdgesMap & Serializable, EV extends Serializable> implements MapFunction, diff --git a/src/test/java/flink/graphs/TestGraphCreation.java b/src/test/java/flink/graphs/TestGraphCreation.java index 3bec4af170d87..b1b4d133142cb 100644 --- a/src/test/java/flink/graphs/TestGraphCreation.java +++ b/src/test/java/flink/graphs/TestGraphCreation.java @@ -197,8 +197,8 @@ public DummyCustomParameterizedType map(Long vertexId) { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> vertices = TestGraphUtils.getLongLongVertexData(env); DataSet> edges = TestGraphUtils.getLongLongEdgeData(env); - Graph graph = new Graph(vertices, edges, env); - DataSet result = graph.validate(new InvalidVertexIdsValidator()); + Graph graph = new Graph(vertices, edges, env); + DataSet result = graph.validate(new InvalidVertexIdsValidator()); result.writeAsText(resultPath); env.execute(); @@ -212,8 +212,8 @@ public DummyCustomParameterizedType map(Long vertexId) { DataSet> vertices = TestGraphUtils.getLongLongInvalidVertexData(env); DataSet> edges = TestGraphUtils.getLongLongEdgeData(env); - Graph graph = new Graph(vertices, edges, env); - DataSet result = graph.validate(new InvalidVertexIdsValidator()); + Graph graph = new Graph(vertices, edges, env); + DataSet result = graph.validate(new InvalidVertexIdsValidator()); result.writeAsText(resultPath); env.execute(); diff --git a/src/test/java/flink/graphs/TestForeachEdge.java b/src/test/java/flink/graphs/TestNeighborMethods.java similarity index 50% rename from src/test/java/flink/graphs/TestForeachEdge.java rename to src/test/java/flink/graphs/TestNeighborMethods.java index 34f121fb0beaa..095f652853773 100644 --- a/src/test/java/flink/graphs/TestForeachEdge.java +++ b/src/test/java/flink/graphs/TestNeighborMethods.java @@ -15,15 +15,15 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) -public class TestForeachEdge extends JavaProgramTestBase { +public class TestNeighborMethods extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 1; + private static int NUM_PROGRAMS = 3; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; private String expectedResult; - public TestForeachEdge(Configuration config) { + public TestNeighborMethods(Configuration config) { super(config); } @@ -70,18 +70,18 @@ public static String runProgram(int progId, String resultPath) throws Exception final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - + DataSet> verticesWithLowestOutNeighbor = - graph.foreachEdge(new OutEdgesFunction() { + graph.reduceOnEdges(new EdgesFunction() { - public Tuple2 iterateOutEdges( + public Tuple2 iterateEdges( Vertex v, - Iterable> outEdges) { + Iterable> edges) { long weight = Long.MAX_VALUE; long minNeighorId = 0; - for (Edge edge: outEdges) { + for (Edge edge: edges) { if (edge.getValue() < weight) { weight = edge.getValue(); minNeighorId = edge.getTarget(); @@ -89,7 +89,7 @@ public Tuple2 iterateOutEdges( } return new Tuple2(v.getId(), minNeighorId); } - }); + }, EdgeDirection.OUT); verticesWithLowestOutNeighbor.writeAsCsv(resultPath); env.execute(); return "1,2\n" + @@ -98,6 +98,75 @@ public Tuple2 iterateOutEdges( "4,5\n" + "5,1\n"; } + case 2: { + /* + * Get the lowest-weight in-neighbor + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithLowestOutNeighbor = + graph.reduceOnEdges(new EdgesFunction() { + + public Tuple2 iterateEdges( + Vertex v, + Iterable> edges) { + + long weight = Long.MAX_VALUE; + long minNeighorId = 0; + + for (Edge edge: edges) { + if (edge.getValue() < weight) { + weight = edge.getValue(); + minNeighorId = edge.getSource(); + } + } + return new Tuple2(v.getId(), minNeighorId); + } + }, EdgeDirection.IN); + verticesWithLowestOutNeighbor.writeAsCsv(resultPath); + env.execute(); + return "1,5\n" + + "2,1\n" + + "3,1\n" + + "4,3\n" + + "5,3\n"; + } + case 3: { + /* + * Get the maximum weight among all edges + * of a vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithMaxEdgeWeight = + graph.reduceOnEdges(new EdgesFunction() { + + public Tuple2 iterateEdges(Vertex v, + Iterable> edges) { + + long weight = Long.MIN_VALUE; + + for (Edge edge: edges) { + if (edge.getValue() > weight) { + weight = edge.getValue(); + } + } + return new Tuple2(v.getId(), weight); + } + }, EdgeDirection.ALL); + verticesWithMaxEdgeWeight.writeAsCsv(resultPath); + env.execute(); + return "1,51\n" + + "2,23\n" + + "3,35\n" + + "4,45\n" + + "5,51\n"; + } default: throw new IllegalArgumentException("Invalid program id"); } From 88e883b29db7cb543b2b8ad7acfaaa109232a1a5 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 14 Dec 2014 19:27:33 +0100 Subject: [PATCH 072/112] added simplified reduceOnEdges without the vertex value --- src/main/java/flink/graphs/EdgesFunction.java | 4 +- .../graphs/EdgesFunctionWithVertexValue.java | 12 ++ src/main/java/flink/graphs/Graph.java | 108 +++++++++++++-- .../flink/graphs/TestNeighborMethods.java | 124 +++++++++++++++++- 4 files changed, 230 insertions(+), 18 deletions(-) create mode 100644 src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java diff --git a/src/main/java/flink/graphs/EdgesFunction.java b/src/main/java/flink/graphs/EdgesFunction.java index 1b44fa717b3f6..4bfaff2ac43cc 100644 --- a/src/main/java/flink/graphs/EdgesFunction.java +++ b/src/main/java/flink/graphs/EdgesFunction.java @@ -6,7 +6,7 @@ import org.apache.flink.api.java.tuple.Tuple2; public interface EdgesFunction & Serializable, - VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { + EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateEdges(Vertex v, Iterable> edges) throws Exception; + Tuple2 iterateEdges(Iterable>> edges) throws Exception; } diff --git a/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java b/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java new file mode 100644 index 0000000000000..366c0b17120a1 --- /dev/null +++ b/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java @@ -0,0 +1,12 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.java.tuple.Tuple2; + +public interface EdgesFunctionWithVertexValue & Serializable, + VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { + + Tuple2 iterateEdges(Vertex v, Iterable> edges) throws Exception; +} diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 48b16026bdf1e..c1bfeb3af8ee1 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -332,12 +332,13 @@ public Graph getUndirected() throws UnsupportedOperationException { /** * Compute an aggregate over the edges of each vertex. - * @param edgesFunction edgesFunction the function to apply to the neighborhood + * The function applied on the edges has access to the vertex value. + * @param edgesFunction the function to apply to the neighborhood * @param direction the edge direction (in-, out-, all-) * @return a dataset of a Tuple2 with the vertex id and the computed value * @throws IllegalArgumentException */ - public DataSet> reduceOnEdges(EdgesFunction edgesFunction, + public DataSet> reduceOnEdges(EdgesFunctionWithVertexValue edgesFunction, EdgeDirection direction) throws IllegalArgumentException { switch (direction) { case IN: @@ -355,6 +356,92 @@ public DataSet> reduceOnEdges(EdgesFunction edges } } + /** + * Compute an aggregate over the edges of each vertex. + * The function applied on the edges only has access to the vertex id + * (not the vertex value). + * @param edgesFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @return a dataset of a Tuple2 with the vertex id and the computed value + * @throws IllegalArgumentException + */ + public DataSet> reduceOnEdges(EdgesFunction edgesFunction, + EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { + case IN: + return edges.map(new ProjectVertexIdMap(1)) + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); + case OUT: + return edges.map(new ProjectVertexIdMap(0)) + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); + case ALL: + return edges.flatMap(new EmitOneEdgePerNode()).groupBy(0) + .reduceGroup(new ApplyGroupReduceFunctionOnAllEdges(edgesFunction)); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + + private static final class ProjectVertexIdMap & Serializable, + EV extends Serializable> implements MapFunction, Tuple2>> { + + private int fieldPosition; + + public ProjectVertexIdMap(int position) { + this.fieldPosition = position; + } + + @SuppressWarnings("unchecked") + public Tuple2> map(Edge edge) { + return new Tuple2>((K) edge.getField(fieldPosition), edge); + } + } + + private static final class ApplyGroupReduceFunctionOnAllEdges & Serializable, + EV extends Serializable, T> implements GroupReduceFunction>, Tuple2>, + ResultTypeQueryable> { + + private EdgesFunction function; + + public ApplyGroupReduceFunctionOnAllEdges(EdgesFunction fun) { + this.function = fun; + } + + public void reduce(final Iterable>> keysWithEdges, + Collector> out) throws Exception { + out.collect(function.iterateEdges(keysWithEdges)); + } + + @Override + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null)); + } + } + + private static final class ApplyGroupReduceFunction & Serializable, + EV extends Serializable, T> implements GroupReduceFunction>, Tuple2>, + ResultTypeQueryable> { + + private EdgesFunction function; + + public ApplyGroupReduceFunction(EdgesFunction fun) { + this.function = fun; + } + + public void reduce(Iterable>> edges, + Collector> out) throws Exception { + out.collect(function.iterateEdges(edges)); + + } + + @Override + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null)); + } + } + private static final class EmitOneEdgePerNode & Serializable, VV extends Serializable, EV extends Serializable> implements FlatMapFunction< Edge, Tuple2>> { @@ -369,9 +456,9 @@ private static final class ApplyCoGroupFunction & Serial implements CoGroupFunction, Edge, Tuple2>, ResultTypeQueryable> { - private EdgesFunction function; + private EdgesFunctionWithVertexValue function; - public ApplyCoGroupFunction (EdgesFunction fun) { + public ApplyCoGroupFunction (EdgesFunctionWithVertexValue fun) { this.function = fun; } public void coGroup(Iterable> vertex, @@ -381,7 +468,7 @@ public void coGroup(Iterable> vertex, @Override public TypeInformation> getProducedType() { return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 3, null, null)); + TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null)); } } @@ -390,15 +477,14 @@ private static final class ApplyCoGroupFunctionOnAllEdges, Tuple2>, Tuple2>, ResultTypeQueryable> { - private EdgesFunction function; + private EdgesFunctionWithVertexValue function; - public ApplyCoGroupFunctionOnAllEdges (EdgesFunction fun) { + public ApplyCoGroupFunctionOnAllEdges (EdgesFunctionWithVertexValue fun) { this.function = fun; } - public void coGroup(Iterable> vertex, - final Iterable>> keysWithEdges, Collector> out) - throws Exception { + public void coGroup(Iterable> vertex, final Iterable>> keysWithEdges, + Collector> out) throws Exception { final Iterator> edgesIterator = new Iterator>() { @@ -432,7 +518,7 @@ public Iterator> iterator() { @Override public TypeInformation> getProducedType() { return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 3, null, null)); + TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null)); } } diff --git a/src/test/java/flink/graphs/TestNeighborMethods.java b/src/test/java/flink/graphs/TestNeighborMethods.java index 095f652853773..6a97767c33397 100644 --- a/src/test/java/flink/graphs/TestNeighborMethods.java +++ b/src/test/java/flink/graphs/TestNeighborMethods.java @@ -17,7 +17,7 @@ @RunWith(Parameterized.class) public class TestNeighborMethods extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 3; + private static int NUM_PROGRAMS = 6; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -72,7 +72,7 @@ public static String runProgram(int progId, String resultPath) throws Exception TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { public Tuple2 iterateEdges( Vertex v, @@ -108,7 +108,7 @@ public Tuple2 iterateEdges( TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { public Tuple2 iterateEdges( Vertex v, @@ -144,7 +144,7 @@ public Tuple2 iterateEdges( TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithMaxEdgeWeight = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { public Tuple2 iterateEdges(Vertex v, Iterable> edges) { @@ -167,10 +167,124 @@ public Tuple2 iterateEdges(Vertex v, "4,45\n" + "5,51\n"; } + case 4: { + /* + * Get the lowest-weight out-neighbor + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithLowestOutNeighbor = + graph.reduceOnEdges(new EdgesFunction() { + + public Tuple2 iterateEdges(Iterable>> edges) { + + long weight = Long.MAX_VALUE; + long minNeighorId = 0; + long vertexId = -1; + long i=0; + + for (Tuple2> edge: edges) { + if (edge.f1.getValue() < weight) { + weight = edge.f1.getValue(); + minNeighorId = edge.f1.getTarget(); + } + if (i==0) { + vertexId = edge.f0; + } i++; + } + return new Tuple2(vertexId, minNeighorId); + } + }, EdgeDirection.OUT); + verticesWithLowestOutNeighbor.writeAsCsv(resultPath); + env.execute(); + return "1,2\n" + + "2,3\n" + + "3,4\n" + + "4,5\n" + + "5,1\n"; + } + case 5: { + /* + * Get the lowest-weight in-neighbor + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithLowestOutNeighbor = + graph.reduceOnEdges(new EdgesFunction() { + + public Tuple2 iterateEdges(Iterable>> edges) { + + long weight = Long.MAX_VALUE; + long minNeighorId = 0; + long vertexId = -1; + long i=0; + + for (Tuple2> edge: edges) { + if (edge.f1.getValue() < weight) { + weight = edge.f1.getValue(); + minNeighorId = edge.f1.getSource(); + } + if (i==0) { + vertexId = edge.f0; + } i++; + } + return new Tuple2(vertexId, minNeighorId); + } + }, EdgeDirection.IN); + verticesWithLowestOutNeighbor.writeAsCsv(resultPath); + env.execute(); + return "1,5\n" + + "2,1\n" + + "3,1\n" + + "4,3\n" + + "5,3\n"; + } + case 6: { + /* + * Get the maximum weight among all edges + * of a vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithMaxEdgeWeight = + graph.reduceOnEdges(new EdgesFunction() { + + public Tuple2 iterateEdges(Iterable>> edges) { + + long weight = Long.MIN_VALUE; + long vertexId = -1; + long i=0; + + for (Tuple2> edge: edges) { + if (edge.f1.getValue() > weight) { + weight = edge.f1.getValue(); + } + if (i==0) { + vertexId = edge.f0; + } i++; + } + return new Tuple2(vertexId, weight); + } + }, EdgeDirection.ALL); + verticesWithMaxEdgeWeight.writeAsCsv(resultPath); + env.execute(); + return "1,51\n" + + "2,23\n" + + "3,35\n" + + "4,45\n" + + "5,51\n"; + } default: throw new IllegalArgumentException("Invalid program id"); } } } - } From 2087311558f7e79edd3a461d7caf5dc7cc70deaf Mon Sep 17 00:00:00 2001 From: andralungu Date: Wed, 17 Dec 2014 22:59:43 +0100 Subject: [PATCH 073/112] SingleSourceShortestPaths example --- .../SingleSourceShortestPathsExample.java | 70 ++++++++++++ .../library/SingleSourceShortestPaths.java | 104 ++++++++++++++++++ 2 files changed, 174 insertions(+) create mode 100644 src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java create mode 100644 src/main/java/flink/graphs/library/SingleSourceShortestPaths.java diff --git a/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java new file mode 100644 index 0000000000000..286551f60bd5f --- /dev/null +++ b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java @@ -0,0 +1,70 @@ +package flink.graphs.example; + +import flink.graphs.Edge; +import flink.graphs.Graph; +import flink.graphs.Vertex; +import flink.graphs.library.PageRank; +import flink.graphs.library.SingleSourceShortestPaths; +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; + +import java.util.ArrayList; +import java.util.List; + +public class SingleSourceShortestPathsExample implements ProgramDescription { + + private static int maxIterations = 5; + + public static void main (String [] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> vertices = getLongDoubleVertexData(env); + + DataSet> edges = getLongDoubleEdgeData(env); + + Graph graph = Graph.create(vertices, edges, env); + + Long srcVertexId = 1L; + + DataSet> singleSourceShortestPaths = + graph.run(new SingleSourceShortestPaths(srcVertexId, maxIterations)).getVertices(); + + singleSourceShortestPaths.print(); + + env.execute(); + } + + @Override + public String getDescription() { + return "Single Source Shortest Paths"; + } + + public static final DataSet> getLongDoubleVertexData( + ExecutionEnvironment env) { + List> vertices = new ArrayList>(); + vertices.add(new Vertex(1L, 1.0)); + vertices.add(new Vertex(2L, 2.0)); + vertices.add(new Vertex(3L, 3.0)); + vertices.add(new Vertex(4L, 4.0)); + vertices.add(new Vertex(5L, 5.0)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getLongDoubleEdgeData( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12.0)); + edges.add(new Edge(1L, 3L, 13.0)); + edges.add(new Edge(2L, 3L, 23.0)); + edges.add(new Edge(3L, 4L, 34.0)); + edges.add(new Edge(3L, 5L, 35.0)); + edges.add(new Edge(4L, 5L, 45.0)); + edges.add(new Edge(5L, 1L, 51.0)); + + return env.fromCollection(edges); + } + +} diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java new file mode 100644 index 0000000000000..ec04f772bda36 --- /dev/null +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -0,0 +1,104 @@ +package flink.graphs.library; + +import flink.graphs.*; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.aggregation.Aggregations; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.Collector; + +public class SingleSourceShortestPaths implements GraphAlgorithm { + + private final Long srcVertexId; + private final Integer maxIterations; + + public SingleSourceShortestPaths(Long srcVertexId, Integer maxIterations) { + this.srcVertexId = srcVertexId; + this.maxIterations = maxIterations; + } + + private Graph computeDeltaShortestPath(Graph graph) { + DataSet> sourceVertex = graph.getVertices().filter( + new SelectVertex(srcVertexId)); + //assign the initial distance as a value for each vertex + DataSet> verticesWithInitialDistance = sourceVertex.cross(graph.getVertices()) + .map(new InitSrcVertex()); + + //open a delta iteration + DeltaIteration, Vertex> iteration = + verticesWithInitialDistance.iterateDelta(verticesWithInitialDistance, maxIterations, 0); + + //apply the step logic: join with the edges, choose the minimum distance and update if + //the new distance proposed is smaller + DataSet> changes = iteration.getWorkset() + .join(graph.getEdges()).where(0).equalTo(0).with(new NeighborWithDistanceJoin()) + .groupBy(0).aggregate(Aggregations.MIN, 1) + .join(iteration.getSolutionSet()).where(0).equalTo(0) + .flatMap(new DistanceFilter()); + + // close the delta iteration (delta and new workset are identical) + DataSet> result = iteration.closeWith(changes, changes); + + return Graph.create(result, graph.getEdges(), ExecutionEnvironment.getExecutionEnvironment()); + } + + private static final class SelectVertex implements FilterFunction> { + + private Long id; + + public SelectVertex(Long id) { + this.id = id; + } + + @Override + public boolean filter(Vertex kvvVertex) throws Exception { + return kvvVertex.getId().equals(id); + } + } + + private static final class InitSrcVertex + implements MapFunction, Vertex>, Vertex> { + + @Override + public Vertex map(Tuple2, Vertex> vertexVertexTuple2) throws Exception { + if(vertexVertexTuple2.f0.f0.equals(vertexVertexTuple2.f1.f0)) { + return new Vertex<>(vertexVertexTuple2.f0.f0, 0.0); + } else { + return new Vertex<>(vertexVertexTuple2.f1.f0, Double.MAX_VALUE); + } + } + } + + public static final class NeighborWithDistanceJoin + implements JoinFunction, Edge, Vertex> { + + @Override + public Vertex join(Vertex kvvVertex, Edge kevEdge) throws Exception { + return new Vertex<>(kevEdge.f1, kvvVertex.f1 + kevEdge.f2); + } + } + + public static final class DistanceFilter + implements FlatMapFunction, Vertex>, Vertex> { + + @Override + public void flatMap(Tuple2, Vertex> value, + Collector> collector) throws Exception { + if (value.f0.f1 < value.f1.f1) { + collector.collect(value.f0); + } + } + } + + @Override + public Graph run(Graph input) { + + return computeDeltaShortestPath(input); + } + +} From 343475a9987f2a99d87b74de7b5536899940cad6 Mon Sep 17 00:00:00 2001 From: andralungu Date: Thu, 18 Dec 2014 20:27:11 +0100 Subject: [PATCH 074/112] SSSP with runVertexCentricIteration --- .../SingleSourceShortestPathsExample.java | 8 +- .../library/SingleSourceShortestPaths.java | 140 ++++++++++-------- 2 files changed, 82 insertions(+), 66 deletions(-) diff --git a/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java index 286551f60bd5f..4b1550832c9df 100644 --- a/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java +++ b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java @@ -3,7 +3,6 @@ import flink.graphs.Edge; import flink.graphs.Graph; import flink.graphs.Vertex; -import flink.graphs.library.PageRank; import flink.graphs.library.SingleSourceShortestPaths; import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.java.DataSet; @@ -24,10 +23,10 @@ public static void main (String [] args) throws Exception { DataSet> edges = getLongDoubleEdgeData(env); - Graph graph = Graph.create(vertices, edges, env); - Long srcVertexId = 1L; + Graph graph = Graph.create(vertices, edges, env); + DataSet> singleSourceShortestPaths = graph.run(new SingleSourceShortestPaths(srcVertexId, maxIterations)).getVertices(); @@ -36,11 +35,13 @@ public static void main (String [] args) throws Exception { env.execute(); } + @Override public String getDescription() { return "Single Source Shortest Paths"; } + @SuppressWarnings("serial") public static final DataSet> getLongDoubleVertexData( ExecutionEnvironment env) { List> vertices = new ArrayList>(); @@ -53,6 +54,7 @@ public static final DataSet> getLongDoubleVertexData( return env.fromCollection(vertices); } + @SuppressWarnings("serial") public static final DataSet> getLongDoubleEdgeData( ExecutionEnvironment env) { List> edges = new ArrayList>(); diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java index ec04f772bda36..624509103b320 100644 --- a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -2,103 +2,117 @@ import flink.graphs.*; import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.aggregation.Aggregations; -import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.Collector; +import org.apache.flink.spargel.java.MessageIterator; +import org.apache.flink.spargel.java.MessagingFunction; +import org.apache.flink.spargel.java.OutgoingEdge; +import org.apache.flink.spargel.java.VertexUpdateFunction; -public class SingleSourceShortestPaths implements GraphAlgorithm { +import java.io.Serializable; - private final Long srcVertexId; +public class SingleSourceShortestPaths & Serializable> implements GraphAlgorithm { + + private final K srcVertexId; private final Integer maxIterations; - public SingleSourceShortestPaths(Long srcVertexId, Integer maxIterations) { + public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { this.srcVertexId = srcVertexId; this.maxIterations = maxIterations; } - private Graph computeDeltaShortestPath(Graph graph) { - DataSet> sourceVertex = graph.getVertices().filter( - new SelectVertex(srcVertexId)); - //assign the initial distance as a value for each vertex - DataSet> verticesWithInitialDistance = sourceVertex.cross(graph.getVertices()) - .map(new InitSrcVertex()); - - //open a delta iteration - DeltaIteration, Vertex> iteration = - verticesWithInitialDistance.iterateDelta(verticesWithInitialDistance, maxIterations, 0); - - //apply the step logic: join with the edges, choose the minimum distance and update if - //the new distance proposed is smaller - DataSet> changes = iteration.getWorkset() - .join(graph.getEdges()).where(0).equalTo(0).with(new NeighborWithDistanceJoin()) - .groupBy(0).aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()).where(0).equalTo(0) - .flatMap(new DistanceFilter()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - return Graph.create(result, graph.getEdges(), ExecutionEnvironment.getExecutionEnvironment()); - } + @Override + public Graph run(Graph input) { + DataSet> sourceVertex = input.getVertices().filter( + new SelectVertex(srcVertexId)); - private static final class SelectVertex implements FilterFunction> { + DataSet> verticesWithInitialDistance = sourceVertex.cross(input.getVertices()) + .map(new InitSrcVertex()); - private Long id; + Graph graph = Graph.create(verticesWithInitialDistance, input.getEdges(), + ExecutionEnvironment.getExecutionEnvironment()); - public SelectVertex(Long id) { - this.id = id; - } + return graph.runVertexCentricIteration( + new VertexDistanceUpdater(), + new MinDistanceMessenger(), + maxIterations + ); + } + + + /** + * Function that updates the value of a vertex by picking the minimum distance from all incoming messages. + * + * @param + */ + @SuppressWarnings("serial") + public static final class VertexDistanceUpdater & Serializable> + extends VertexUpdateFunction { @Override - public boolean filter(Vertex kvvVertex) throws Exception { - return kvvVertex.getId().equals(id); + public void updateVertex(K vertexKey, Double vertexValue, MessageIterator inMessages) { + + Double minDistance = Double.MAX_VALUE; + + for (double msg : inMessages) { + if (msg < minDistance) { + minDistance = msg; + } + } + + if (vertexValue > minDistance) { + vertexValue = minDistance; + } + + setNewVertexValue(vertexValue); } } - private static final class InitSrcVertex - implements MapFunction, Vertex>, Vertex> { + /** + * Distributes the minimum distance associated with a given vertex among all the target vertices + * summed up with the edge's value. + * + * @param + */ + @SuppressWarnings("serial") + public static final class MinDistanceMessenger & Serializable> + extends MessagingFunction { @Override - public Vertex map(Tuple2, Vertex> vertexVertexTuple2) throws Exception { - if(vertexVertexTuple2.f0.f0.equals(vertexVertexTuple2.f1.f0)) { - return new Vertex<>(vertexVertexTuple2.f0.f0, 0.0); - } else { - return new Vertex<>(vertexVertexTuple2.f1.f0, Double.MAX_VALUE); + public void sendMessages(K vertexKey, Double newDistance) throws Exception { + for (OutgoingEdge edge : getOutgoingEdges()) { + sendMessageTo(edge.target(), newDistance + edge.edgeValue()); } } } - public static final class NeighborWithDistanceJoin - implements JoinFunction, Edge, Vertex> { + private static final class SelectVertex & Serializable> + implements FilterFunction> { + private K id; + + public SelectVertex(K id) { + this.id = id; + } @Override - public Vertex join(Vertex kvvVertex, Edge kevEdge) throws Exception { - return new Vertex<>(kevEdge.f1, kvvVertex.f1 + kevEdge.f2); + public boolean filter(Vertex vertex) throws Exception { + return vertex.getId().equals(id); } } - public static final class DistanceFilter - implements FlatMapFunction, Vertex>, Vertex> { + private static final class InitSrcVertex & Serializable> + implements MapFunction, Vertex>, Vertex> { @Override - public void flatMap(Tuple2, Vertex> value, - Collector> collector) throws Exception { - if (value.f0.f1 < value.f1.f1) { - collector.collect(value.f0); + public Vertex map(Tuple2, Vertex> vertexVertexTuple2) throws Exception { + if(vertexVertexTuple2.f0.f0.equals(vertexVertexTuple2.f1.f0)) { + return new Vertex<>(vertexVertexTuple2.f0.f0, 0.0); + } else { + return new Vertex<>(vertexVertexTuple2.f1.f0, Double.MAX_VALUE); } } } - @Override - public Graph run(Graph input) { - - return computeDeltaShortestPath(input); - } - -} +} \ No newline at end of file From 905dd428e201560ede0c3e0f9a533f0aac62005e Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 20 Dec 2014 12:04:03 +0100 Subject: [PATCH 075/112] changed mapVertices to return a Graph and simplified SSSP --- src/main/java/flink/graphs/Graph.java | 21 +++--- .../SingleSourceShortestPathsExample.java | 41 ++---------- .../graphs/example/utils/ExampleUtils.java | 28 ++++++++ .../library/SingleSourceShortestPaths.java | 65 +++++++------------ .../java/flink/graphs/TestMapVertices.java | 53 ++++++++------- 5 files changed, 91 insertions(+), 117 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index c1bfeb3af8ee1..4f295eacd1e20 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -111,36 +111,35 @@ public DataSet> getEdges() { /** * Apply a function to the attribute of each vertex in the graph. * @param mapper - * @return + * @return a new graph */ - public DataSet> mapVertices(final MapFunction mapper) { - return vertices.map(new ApplyMapperToVertexWithType(mapper)); + public Graph mapVertices(final MapFunction, NV> mapper) { + DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper)); + return new Graph(mappedVertices, this.getEdges(), this.context); } private static final class ApplyMapperToVertexWithType & Serializable, VV extends Serializable, NV extends Serializable> implements MapFunction , Vertex>, ResultTypeQueryable> { - private MapFunction innerMapper; + private MapFunction, NV> innerMapper; - public ApplyMapperToVertexWithType(MapFunction theMapper) { + public ApplyMapperToVertexWithType(MapFunction, NV> theMapper) { this.innerMapper = theMapper; } public Vertex map(Vertex value) throws Exception { - return new Vertex(value.f0, innerMapper.map(value.f1)); + return new Vertex(value.f0, innerMapper.map(value)); } @Override public TypeInformation> getProducedType() { - @SuppressWarnings("unchecked") - TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, - (TypeInformation)vertexValueType); - + TypeInformation> vertextypeInfo = new TupleTypeInfo>(keyType, vertexValueType); + TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, vertextypeInfo); return new TupleTypeInfo>(keyType, newVertexValueType); } } - + /** * Apply a function to the attribute of each edge in the graph. * @param mapper diff --git a/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java index 4b1550832c9df..75e33dcae4bf9 100644 --- a/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java +++ b/src/main/java/flink/graphs/example/SingleSourceShortestPathsExample.java @@ -3,14 +3,13 @@ import flink.graphs.Edge; import flink.graphs.Graph; import flink.graphs.Vertex; +import flink.graphs.example.utils.ExampleUtils; import flink.graphs.library.SingleSourceShortestPaths; + import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import java.util.ArrayList; -import java.util.List; - public class SingleSourceShortestPathsExample implements ProgramDescription { private static int maxIterations = 5; @@ -19,54 +18,24 @@ public static void main (String [] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> vertices = getLongDoubleVertexData(env); + DataSet> vertices = ExampleUtils.getLongDoubleVertexData(env); - DataSet> edges = getLongDoubleEdgeData(env); + DataSet> edges = ExampleUtils.getLongDoubleEdgeData(env); Long srcVertexId = 1L; Graph graph = Graph.create(vertices, edges, env); DataSet> singleSourceShortestPaths = - graph.run(new SingleSourceShortestPaths(srcVertexId, maxIterations)).getVertices(); + graph.run(new SingleSourceShortestPaths(srcVertexId, maxIterations)).getVertices(); singleSourceShortestPaths.print(); env.execute(); } - @Override public String getDescription() { return "Single Source Shortest Paths"; } - - @SuppressWarnings("serial") - public static final DataSet> getLongDoubleVertexData( - ExecutionEnvironment env) { - List> vertices = new ArrayList>(); - vertices.add(new Vertex(1L, 1.0)); - vertices.add(new Vertex(2L, 2.0)); - vertices.add(new Vertex(3L, 3.0)); - vertices.add(new Vertex(4L, 4.0)); - vertices.add(new Vertex(5L, 5.0)); - - return env.fromCollection(vertices); - } - - @SuppressWarnings("serial") - public static final DataSet> getLongDoubleEdgeData( - ExecutionEnvironment env) { - List> edges = new ArrayList>(); - edges.add(new Edge(1L, 2L, 12.0)); - edges.add(new Edge(1L, 3L, 13.0)); - edges.add(new Edge(2L, 3L, 23.0)); - edges.add(new Edge(3L, 4L, 34.0)); - edges.add(new Edge(3L, 5L, 35.0)); - edges.add(new Edge(4L, 5L, 45.0)); - edges.add(new Edge(5L, 1L, 51.0)); - - return env.fromCollection(edges); - } - } diff --git a/src/main/java/flink/graphs/example/utils/ExampleUtils.java b/src/main/java/flink/graphs/example/utils/ExampleUtils.java index c4ff43edcf75d..4588230a86cfa 100644 --- a/src/main/java/flink/graphs/example/utils/ExampleUtils.java +++ b/src/main/java/flink/graphs/example/utils/ExampleUtils.java @@ -1,6 +1,8 @@ package flink.graphs.example.utils; import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -103,5 +105,31 @@ public void flatMap(Long key, Collector> out) throws Excep } }); } + + public static final DataSet> getLongDoubleVertexData( + ExecutionEnvironment env) { + List> vertices = new ArrayList>(); + vertices.add(new Vertex(1L, 1.0)); + vertices.add(new Vertex(2L, 2.0)); + vertices.add(new Vertex(3L, 3.0)); + vertices.add(new Vertex(4L, 4.0)); + vertices.add(new Vertex(5L, 5.0)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getLongDoubleEdgeData( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Edge(1L, 2L, 12.0)); + edges.add(new Edge(1L, 3L, 13.0)); + edges.add(new Edge(2L, 3L, 23.0)); + edges.add(new Edge(3L, 4L, 34.0)); + edges.add(new Edge(3L, 5L, 35.0)); + edges.add(new Edge(4L, 5L, 45.0)); + edges.add(new Edge(5L, 1L, 51.0)); + + return env.fromCollection(edges); + } } diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java index 624509103b320..632233ce2f06e 100644 --- a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -1,11 +1,8 @@ package flink.graphs.library; import flink.graphs.*; -import org.apache.flink.api.common.functions.FilterFunction; + 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.tuple.Tuple2; import org.apache.flink.spargel.java.MessageIterator; import org.apache.flink.spargel.java.MessagingFunction; import org.apache.flink.spargel.java.OutgoingEdge; @@ -13,6 +10,7 @@ import java.io.Serializable; +@SuppressWarnings("serial") public class SingleSourceShortestPaths & Serializable> implements GraphAlgorithm { private final K srcVertexId; @@ -25,29 +23,39 @@ public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { @Override public Graph run(Graph input) { - DataSet> sourceVertex = input.getVertices().filter( - new SelectVertex(srcVertexId)); - - DataSet> verticesWithInitialDistance = sourceVertex.cross(input.getVertices()) - .map(new InitSrcVertex()); - - Graph graph = Graph.create(verticesWithInitialDistance, input.getEdges(), - ExecutionEnvironment.getExecutionEnvironment()); - return graph.runVertexCentricIteration( + return input.mapVertices(new InitVerticesMapper(srcVertexId)) + .runVertexCentricIteration( new VertexDistanceUpdater(), new MinDistanceMessenger(), maxIterations ); } + public static final class InitVerticesMapper & Serializable> + implements MapFunction, Double> { + + private K srcVertexId; + + public InitVerticesMapper(K srcId) { + this.srcVertexId = srcId; + } + + public Double map(Vertex value) { + if (value.f0.equals(srcVertexId)) { + return 0.0; + } + else { + return Double.MAX_VALUE; + } + } + } /** * Function that updates the value of a vertex by picking the minimum distance from all incoming messages. * * @param */ - @SuppressWarnings("serial") public static final class VertexDistanceUpdater & Serializable> extends VertexUpdateFunction { @@ -76,7 +84,6 @@ public void updateVertex(K vertexKey, Double vertexValue, MessageIterator */ - @SuppressWarnings("serial") public static final class MinDistanceMessenger & Serializable> extends MessagingFunction { @@ -87,32 +94,4 @@ public void sendMessages(K vertexKey, Double newDistance) throws Exception { } } } - - private static final class SelectVertex & Serializable> - implements FilterFunction> { - private K id; - - public SelectVertex(K id) { - this.id = id; - } - - @Override - public boolean filter(Vertex vertex) throws Exception { - return vertex.getId().equals(id); - } - } - - private static final class InitSrcVertex & Serializable> - implements MapFunction, Vertex>, Vertex> { - - @Override - public Vertex map(Tuple2, Vertex> vertexVertexTuple2) throws Exception { - if(vertexVertexTuple2.f0.f0.equals(vertexVertexTuple2.f1.f0)) { - return new Vertex<>(vertexVertexTuple2.f0.f0, 0.0); - } else { - return new Vertex<>(vertexVertexTuple2.f1.f0, Double.MAX_VALUE); - } - } - } - } \ No newline at end of file diff --git a/src/test/java/flink/graphs/TestMapVertices.java b/src/test/java/flink/graphs/TestMapVertices.java index 309af0d6ead7f..9ed14db53ffb8 100644 --- a/src/test/java/flink/graphs/TestMapVertices.java +++ b/src/test/java/flink/graphs/TestMapVertices.java @@ -75,11 +75,11 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public Long map(Long value) throws Exception { - return value+1; + DataSet> mappedVertices = graph.mapVertices(new MapFunction, Long>() { + public Long map(Vertex value) throws Exception { + return value.getValue()+1; } - }); + }).getVertices(); mappedVertices.writeAsCsv(resultPath); env.execute(); @@ -98,31 +98,30 @@ public Long map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public String map(Long value) throws Exception { + DataSet> mappedVertices = graph.mapVertices(new MapFunction, String>() { + public String map(Vertex vertex) throws Exception { String stringValue; - if (value == 1) { + if (vertex.getValue() == 1) { stringValue = "one"; } - else if (value == 2) { + else if (vertex.getValue() == 2) { stringValue = "two"; } - else if (value == 3) { + else if (vertex.getValue() == 3) { stringValue = "three"; } - else if (value == 4) { + else if (vertex.getValue() == 4) { stringValue = "four"; } - else if (value == 5) { + else if (vertex.getValue() == 5) { stringValue = "five"; } else { stringValue = ""; } - return stringValue; } - }); + }).getVertices(); mappedVertices.writeAsCsv(resultPath); env.execute(); @@ -141,13 +140,13 @@ else if (value == 5) { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedVertices = graph.mapVertices(new MapFunction>() { - public Tuple1 map(Long value) throws Exception { + DataSet>> mappedVertices = graph.mapVertices(new MapFunction, Tuple1>() { + public Tuple1 map(Vertex vertex) throws Exception { Tuple1 tupleValue = new Tuple1(); - tupleValue.setFields(value); + tupleValue.setFields(vertex.getValue()); return tupleValue; } - }); + }).getVertices(); mappedVertices.writeAsCsv(resultPath); env.execute(); @@ -166,13 +165,13 @@ public Tuple1 map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedVertices = graph.mapVertices(new MapFunction() { - public DummyCustomType map(Long value) throws Exception { + DataSet> mappedVertices = graph.mapVertices(new MapFunction, DummyCustomType>() { + public DummyCustomType map(Vertex vertex) throws Exception { DummyCustomType dummyValue = new DummyCustomType(); - dummyValue.setIntField(value.intValue()); + dummyValue.setIntField(vertex.getValue().intValue()); return dummyValue; } - }); + }).getVertices(); mappedVertices.writeAsCsv(resultPath); env.execute(); @@ -192,14 +191,14 @@ public DummyCustomType map(Long value) throws Exception { TestGraphUtils.getLongLongEdgeData(env), env); DataSet>> mappedVertices = graph.mapVertices( - new MapFunction>() { - public DummyCustomParameterizedType map(Long value) throws Exception { + new MapFunction, DummyCustomParameterizedType>() { + public DummyCustomParameterizedType map(Vertex vertex) throws Exception { DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); - dummyValue.setIntField(value.intValue()); - dummyValue.setTField(new Double(value)); + dummyValue.setIntField(vertex.getValue().intValue()); + dummyValue.setTField(new Double(vertex.getValue())); return dummyValue; } - }); + }).getVertices(); mappedVertices.writeAsCsv(resultPath); env.execute(); @@ -214,5 +213,5 @@ public DummyCustomParameterizedType map(Long value) throws Exception { } } } - + } From 88e1cb5bbd66b1cc25456377d77003d35f19aeb3 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 20 Dec 2014 12:37:36 +0100 Subject: [PATCH 076/112] changed mapEdges to also return a Graph --- src/main/java/flink/graphs/Graph.java | 17 ++++---- src/test/java/flink/graphs/TestMapEdges.java | 45 ++++++++++---------- 2 files changed, 32 insertions(+), 30 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 4f295eacd1e20..48b45dedfb2a5 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -145,30 +145,31 @@ public TypeInformation> getProducedType() { * @param mapper * @return */ - public DataSet> mapEdges(final MapFunction mapper) { - return edges.map(new ApplyMapperToEdgeWithType(mapper)); + public Graph mapEdges(final MapFunction, NV> mapper) { + DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper)); + return new Graph(this.vertices, mappedEdges, this.context); } private static final class ApplyMapperToEdgeWithType & Serializable, EV extends Serializable, NV extends Serializable> implements MapFunction , Edge>, ResultTypeQueryable> { - private MapFunction innerMapper; + private MapFunction, NV> innerMapper; - public ApplyMapperToEdgeWithType(MapFunction theMapper) { + public ApplyMapperToEdgeWithType(MapFunction, NV> theMapper) { this.innerMapper = theMapper; } public Edge map(Edge value) throws Exception { - return new Edge(value.f0, value.f1, innerMapper.map(value.f2)); + return new Edge(value.f0, value.f1, innerMapper.map(value)); } @Override public TypeInformation> getProducedType() { - @SuppressWarnings("unchecked") + TypeInformation> edgeTypeInfo = new TupleTypeInfo>(keyType, keyType, edgeValueType); TypeInformation newEdgeValueType = TypeExtractor.getMapReturnTypes(innerMapper, - (TypeInformation)edgeValueType); - + edgeTypeInfo); + return new TupleTypeInfo>(keyType, keyType, newEdgeValueType); } } diff --git a/src/test/java/flink/graphs/TestMapEdges.java b/src/test/java/flink/graphs/TestMapEdges.java index 4fdb3af5ea77f..9612d8ff2adfa 100644 --- a/src/test/java/flink/graphs/TestMapEdges.java +++ b/src/test/java/flink/graphs/TestMapEdges.java @@ -75,11 +75,11 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { - public Long map(Long value) throws Exception { - return value+1; + DataSet> mappedEdges = graph.mapEdges(new MapFunction, Long>() { + public Long map(Edge edge) throws Exception { + return edge.getValue()+1; } - }); + }).getEdges(); mappedEdges.writeAsCsv(resultPath); env.execute(); @@ -100,11 +100,11 @@ public Long map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { - public String map(Long value) throws Exception { - return String.format("string(%d)", value); + DataSet> mappedEdges = graph.mapEdges(new MapFunction, String>() { + public String map(Edge edge) throws Exception { + return String.format("string(%d)", edge.getValue()); } - }); + }).getEdges(); mappedEdges.writeAsCsv(resultPath); env.execute(); @@ -125,13 +125,14 @@ public String map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet>> mappedEdges = graph.mapEdges(new MapFunction>() { - public Tuple1 map(Long value) throws Exception { + DataSet>> mappedEdges = graph.mapEdges(new MapFunction, + Tuple1>() { + public Tuple1 map(Edge edge) throws Exception { Tuple1 tupleValue = new Tuple1(); - tupleValue.setFields(value); + tupleValue.setFields(edge.getValue()); return tupleValue; } - }); + }).getEdges(); mappedEdges.writeAsCsv(resultPath); env.execute(); @@ -152,13 +153,14 @@ public Tuple1 map(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - DataSet> mappedEdges = graph.mapEdges(new MapFunction() { - public DummyCustomType map(Long value) throws Exception { + DataSet> mappedEdges = graph.mapEdges(new MapFunction, + DummyCustomType>() { + public DummyCustomType map(Edge edge) throws Exception { DummyCustomType dummyValue = new DummyCustomType(); - dummyValue.setIntField(value.intValue()); + dummyValue.setIntField(edge.getValue().intValue()); return dummyValue; } - }); + }).getEdges(); mappedEdges.writeAsCsv(resultPath); env.execute(); @@ -180,14 +182,14 @@ public DummyCustomType map(Long value) throws Exception { TestGraphUtils.getLongLongEdgeData(env), env); DataSet>> mappedEdges = graph.mapEdges( - new MapFunction>() { - public DummyCustomParameterizedType map(Long value) throws Exception { + new MapFunction, DummyCustomParameterizedType>() { + public DummyCustomParameterizedType map(Edge edge) throws Exception { DummyCustomParameterizedType dummyValue = new DummyCustomParameterizedType(); - dummyValue.setIntField(value.intValue()); - dummyValue.setTField(new Double(value)); + dummyValue.setIntField(edge.getValue().intValue()); + dummyValue.setTField(new Double(edge.getValue())); return dummyValue; } - }); + }).getEdges(); mappedEdges.writeAsCsv(resultPath); env.execute(); @@ -204,5 +206,4 @@ public DummyCustomParameterizedType map(Long value) throws Exception { } } } - } From edca836245fc017404a8a2b691a5c24d194723ae Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 21 Dec 2014 15:46:40 +0100 Subject: [PATCH 077/112] reduceOnNeighbors with vertex value --- src/main/java/flink/graphs/Graph.java | 130 +++++++++++++++ .../java/flink/graphs/NeighborsFunction.java | 13 ++ .../NeighborsFunctionWithVertexValue.java | 12 ++ ...ods.java => TestReduceOnEdgesMethods.java} | 4 +- .../graphs/TestReduceOnNeighborMethods.java | 157 ++++++++++++++++++ 5 files changed, 314 insertions(+), 2 deletions(-) create mode 100644 src/main/java/flink/graphs/NeighborsFunction.java create mode 100644 src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java rename src/test/java/flink/graphs/{TestNeighborMethods.java => TestReduceOnEdgesMethods.java} (98%) create mode 100644 src/test/java/flink/graphs/TestReduceOnNeighborMethods.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 48b45dedfb2a5..f2f62e33826cd 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -41,6 +41,7 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -451,6 +452,15 @@ public void flatMap(Edge edge, Collector>> out) { } } + private static final class EmitOneEdgeWithNeighborPerNode & Serializable, + VV extends Serializable, EV extends Serializable> implements FlatMapFunction< + Edge, Tuple3>> { + public void flatMap(Edge edge, Collector>> out) { + out.collect(new Tuple3>(edge.getSource(), edge.getTarget(), edge)); + out.collect(new Tuple3>(edge.getTarget(), edge.getSource(), edge)); + } + } + private static final class ApplyCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> implements CoGroupFunction, Edge, Tuple2>, @@ -978,4 +988,124 @@ public Graph run (GraphAlgorithm algorithm) { return algorithm.run(this); } + /** + * Compute an aggregate over the neighbors (edges and vertices) of each vertex. + * The function applied on the neighbors has access to the vertex value. + * @param neighborsFunction the function to apply to the neighborhood + * @param direction the edge direction (in-, out-, all-) + * @return a dataset of a Tuple2 with the vertex id and the computed value + * @throws IllegalArgumentException + */ + public DataSet> reduceOnNeighbors(NeighborsFunctionWithVertexValue neighborsFunction, + EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { + case IN: + // create pairs + DataSet, Vertex>> edgesWithSources = edges.join(this.vertices) + .where(0).equalTo(0); + return vertices.coGroup(edgesWithSources).where(0).equalTo("f0.f1").with( + new ApplyNeighborCoGroupFunction(neighborsFunction)); + case OUT: + // create pairs + DataSet, Vertex>> edgesWithTargets = edges.join(this.vertices) + .where(1).equalTo(0); + return vertices.coGroup(edgesWithTargets).where(0).equalTo("f0.f0").with( + new ApplyNeighborCoGroupFunction(neighborsFunction)); + case ALL: + // create pairs + DataSet, Vertex>> edgesWithNeighbors = edges.flatMap( + new EmitOneEdgeWithNeighborPerNode()).join(this.vertices) + .where(1).equalTo(0).with(new ProjectEdgeWithNeighbor()); + + return vertices.coGroup(edgesWithNeighbors).where(0).equalTo(0) + .with(new ApplyCoGroupFunctionOnAllNeighbors(neighborsFunction)); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + + private static final class ProjectEdgeWithNeighbor & Serializable, + VV extends Serializable, EV extends Serializable> implements + FlatJoinFunction>, Vertex, Tuple3, Vertex>> { + public void join(Tuple3> keysWithEdge, Vertex neighbor, + Collector, Vertex>> out) { + out.collect(new Tuple3, Vertex>(keysWithEdge.f0, + keysWithEdge.f2, neighbor)); + } + } + + private static final class ApplyNeighborCoGroupFunction & Serializable, + VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple2, Vertex>, Tuple2>, + ResultTypeQueryable> { + + private NeighborsFunctionWithVertexValue function; + + public ApplyNeighborCoGroupFunction (NeighborsFunctionWithVertexValue fun) { + this.function = fun; + } + public void coGroup(Iterable> vertex, + Iterable, Vertex>> neighbors, Collector> out) throws Exception { + out.collect(function.iterateNeighbors(vertex.iterator().next(), neighbors)); + } + @Override + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, + function.getClass(), 3, null, null)); + } + } + + private static final class ApplyCoGroupFunctionOnAllNeighbors & Serializable, + VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple3, Vertex>, Tuple2>, + ResultTypeQueryable> { + + private NeighborsFunctionWithVertexValue function; + + public ApplyCoGroupFunctionOnAllNeighbors (NeighborsFunctionWithVertexValue fun) { + this.function = fun; + } + + public void coGroup(Iterable> vertex, final Iterable, Vertex>> keysWithNeighbors, + Collector> out) throws Exception { + + final Iterator, Vertex>> neighborsIterator = new Iterator, Vertex>>() { + + final Iterator, Vertex>> keysWithEdgesIterator = + keysWithNeighbors.iterator(); + + @Override + public boolean hasNext() { + return keysWithEdgesIterator.hasNext(); + } + + @Override + public Tuple2, Vertex> next() { + Tuple3, Vertex> next = keysWithEdgesIterator.next(); + return new Tuple2, Vertex>(next.f1, next.f2); + } + + @Override + public void remove() { + keysWithEdgesIterator.remove(); + } + }; + + Iterable, Vertex>> neighborsIterable = new Iterable, Vertex>>() { + public Iterator, Vertex>> iterator() { + return neighborsIterator; + } + }; + + out.collect(function.iterateNeighbors(vertex.iterator().next(), neighborsIterable)); + } + + @Override + public TypeInformation> getProducedType() { + return new TupleTypeInfo>(keyType, + TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, + function.getClass(), 3, null, null)); + } + } } diff --git a/src/main/java/flink/graphs/NeighborsFunction.java b/src/main/java/flink/graphs/NeighborsFunction.java new file mode 100644 index 0000000000000..51580784ef227 --- /dev/null +++ b/src/main/java/flink/graphs/NeighborsFunction.java @@ -0,0 +1,13 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; + +public interface NeighborsFunction & Serializable, VV extends Serializable, + EV extends Serializable, O> extends Function, Serializable { + + Tuple2 iterateEdges(Iterable, Vertex>> neighbors) throws Exception; +} diff --git a/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java b/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java new file mode 100644 index 0000000000000..ae7ef2ec6c87d --- /dev/null +++ b/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java @@ -0,0 +1,12 @@ +package flink.graphs; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.java.tuple.Tuple2; + +public interface NeighborsFunctionWithVertexValue & Serializable, VV extends Serializable, + EV extends Serializable, O> extends Function, Serializable { + + Tuple2 iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) throws Exception; +} diff --git a/src/test/java/flink/graphs/TestNeighborMethods.java b/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java similarity index 98% rename from src/test/java/flink/graphs/TestNeighborMethods.java rename to src/test/java/flink/graphs/TestReduceOnEdgesMethods.java index 6a97767c33397..1fdccfa888341 100644 --- a/src/test/java/flink/graphs/TestNeighborMethods.java +++ b/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java @@ -15,7 +15,7 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) -public class TestNeighborMethods extends JavaProgramTestBase { +public class TestReduceOnEdgesMethods extends JavaProgramTestBase { private static int NUM_PROGRAMS = 6; @@ -23,7 +23,7 @@ public class TestNeighborMethods extends JavaProgramTestBase { private String resultPath; private String expectedResult; - public TestNeighborMethods(Configuration config) { + public TestReduceOnEdgesMethods(Configuration config) { super(config); } diff --git a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java new file mode 100644 index 0000000000000..b8927aa034e8a --- /dev/null +++ b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java @@ -0,0 +1,157 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestReduceOnNeighborMethods extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 3; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestReduceOnNeighborMethods(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Get the sum of out-neighbor values + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSumOfOutNeighborValues = + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + public Tuple2 iterateNeighbors(Vertex vertex, + Iterable, Vertex>> neighbors) { + long sum = 0; + for (Tuple2, Vertex> neighbor : neighbors) { + sum += neighbor.f1.getValue(); + } + return new Tuple2(vertex.getId(), sum); + } + }, EdgeDirection.OUT); + + verticesWithSumOfOutNeighborValues.writeAsCsv(resultPath); + env.execute(); + return "1,5\n" + + "2,3\n" + + "3,9\n" + + "4,5\n" + + "5,1\n"; + } + case 2: { + /* + * Get the sum of in-neighbor values + * time edge weights for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSum = + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + public Tuple2 iterateNeighbors(Vertex vertex, + Iterable, Vertex>> neighbors) { + long sum = 0; + for (Tuple2, Vertex> neighbor : neighbors) { + sum += neighbor.f0.getValue() * neighbor.f1.getValue(); + } + return new Tuple2(vertex.getId(), sum); + } + }, EdgeDirection.IN); + + verticesWithSum.writeAsCsv(resultPath); + env.execute(); + return "1,255\n" + + "2,12\n" + + "3,59\n" + + "4,102\n" + + "5,285\n"; + } + case 3: { + /* + * Get the sum of all neighbor values + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSumOfOutNeighborValues = + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + public Tuple2 iterateNeighbors(Vertex vertex, + Iterable, Vertex>> neighbors) { + long sum = 0; + for (Tuple2, Vertex> neighbor : neighbors) { + sum += neighbor.f1.getValue(); + } + return new Tuple2(vertex.getId(), sum); + } + }, EdgeDirection.ALL); + + verticesWithSumOfOutNeighborValues.writeAsCsv(resultPath); + env.execute(); + return "1,10\n" + + "2,4\n" + + "3,12\n" + + "4,8\n" + + "5,8\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } +} \ No newline at end of file From aeb9f0bca452d536a4756468920b47d9f108bec2 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 21 Dec 2014 16:12:09 +0100 Subject: [PATCH 078/112] return arbitrary type instead of Tuple2 for the neighbor methods --- src/main/java/flink/graphs/EdgesFunction.java | 2 +- .../graphs/EdgesFunctionWithVertexValue.java | 3 +- src/main/java/flink/graphs/Graph.java | 85 +++++++++---------- .../java/flink/graphs/NeighborsFunction.java | 3 +- .../NeighborsFunctionWithVertexValue.java | 2 +- .../graphs/TestReduceOnEdgesMethods.java | 12 +-- .../graphs/TestReduceOnNeighborMethods.java | 9 +- 7 files changed, 57 insertions(+), 59 deletions(-) diff --git a/src/main/java/flink/graphs/EdgesFunction.java b/src/main/java/flink/graphs/EdgesFunction.java index 4bfaff2ac43cc..11b30fd783370 100644 --- a/src/main/java/flink/graphs/EdgesFunction.java +++ b/src/main/java/flink/graphs/EdgesFunction.java @@ -8,5 +8,5 @@ public interface EdgesFunction & Serializable, EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateEdges(Iterable>> edges) throws Exception; + O iterateEdges(Iterable>> edges) throws Exception; } diff --git a/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java b/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java index 366c0b17120a1..d50be0a722eae 100644 --- a/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java +++ b/src/main/java/flink/graphs/EdgesFunctionWithVertexValue.java @@ -3,10 +3,9 @@ import java.io.Serializable; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.java.tuple.Tuple2; public interface EdgesFunctionWithVertexValue & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateEdges(Vertex v, Iterable> edges) throws Exception; + O iterateEdges(Vertex v, Iterable> edges) throws Exception; } diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index f2f62e33826cd..b5cb2ac36af5e 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -336,10 +336,11 @@ public Graph getUndirected() throws UnsupportedOperationException { * The function applied on the edges has access to the vertex value. * @param edgesFunction the function to apply to the neighborhood * @param direction the edge direction (in-, out-, all-) - * @return a dataset of a Tuple2 with the vertex id and the computed value + * @param the output type + * @return a dataset of a T * @throws IllegalArgumentException */ - public DataSet> reduceOnEdges(EdgesFunctionWithVertexValue edgesFunction, + public DataSet reduceOnEdges(EdgesFunctionWithVertexValue edgesFunction, EdgeDirection direction) throws IllegalArgumentException { switch (direction) { case IN: @@ -363,10 +364,11 @@ public DataSet> reduceOnEdges(EdgesFunctionWithVertexValue the output type + * @return a dataset of T * @throws IllegalArgumentException */ - public DataSet> reduceOnEdges(EdgesFunction edgesFunction, + public DataSet reduceOnEdges(EdgesFunction edgesFunction, EdgeDirection direction) throws IllegalArgumentException { switch (direction) { case IN: @@ -399,8 +401,8 @@ public Tuple2> map(Edge edge) { } private static final class ApplyGroupReduceFunctionOnAllEdges & Serializable, - EV extends Serializable, T> implements GroupReduceFunction>, Tuple2>, - ResultTypeQueryable> { + EV extends Serializable, T> implements GroupReduceFunction>, T>, + ResultTypeQueryable { private EdgesFunction function; @@ -409,20 +411,19 @@ public ApplyGroupReduceFunctionOnAllEdges(EdgesFunction fun) { } public void reduce(final Iterable>> keysWithEdges, - Collector> out) throws Exception { + Collector out) throws Exception { out.collect(function.iterateEdges(keysWithEdges)); } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null); } } private static final class ApplyGroupReduceFunction & Serializable, - EV extends Serializable, T> implements GroupReduceFunction>, Tuple2>, - ResultTypeQueryable> { + EV extends Serializable, T> implements GroupReduceFunction>, T>, + ResultTypeQueryable { private EdgesFunction function; @@ -431,15 +432,14 @@ public ApplyGroupReduceFunction(EdgesFunction fun) { } public void reduce(Iterable>> edges, - Collector> out) throws Exception { + Collector out) throws Exception { out.collect(function.iterateEdges(edges)); } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null); } } @@ -463,8 +463,8 @@ public void flatMap(Edge edge, Collector>> out) private static final class ApplyCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Edge, Tuple2>, - ResultTypeQueryable> { + implements CoGroupFunction, Edge, T>, + ResultTypeQueryable { private EdgesFunctionWithVertexValue function; @@ -472,20 +472,19 @@ public ApplyCoGroupFunction (EdgesFunctionWithVertexValue fun) { this.function = fun; } public void coGroup(Iterable> vertex, - Iterable> edges, Collector> out) throws Exception { + Iterable> edges, Collector out) throws Exception { out.collect(function.iterateEdges(vertex.iterator().next(), edges)); } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null); } } private static final class ApplyCoGroupFunctionOnAllEdges & Serializable, VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple2>, Tuple2>, - ResultTypeQueryable> { + implements CoGroupFunction, Tuple2>, T>, + ResultTypeQueryable { private EdgesFunctionWithVertexValue function; @@ -494,7 +493,7 @@ public ApplyCoGroupFunctionOnAllEdges (EdgesFunctionWithVertexValue> vertex, final Iterable>> keysWithEdges, - Collector> out) throws Exception { + Collector out) throws Exception { final Iterator> edgesIterator = new Iterator>() { @@ -526,9 +525,8 @@ public Iterator> iterator() { } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null); } } @@ -993,10 +991,11 @@ public Graph run (GraphAlgorithm algorithm) { * The function applied on the neighbors has access to the vertex value. * @param neighborsFunction the function to apply to the neighborhood * @param direction the edge direction (in-, out-, all-) - * @return a dataset of a Tuple2 with the vertex id and the computed value + * @param the output type + * @return a dataset of a T * @throws IllegalArgumentException */ - public DataSet> reduceOnNeighbors(NeighborsFunctionWithVertexValue neighborsFunction, + public DataSet reduceOnNeighbors(NeighborsFunctionWithVertexValue neighborsFunction, EdgeDirection direction) throws IllegalArgumentException { switch (direction) { case IN: @@ -1036,8 +1035,8 @@ public void join(Tuple3> keysWithEdge, Vertex neighbor, private static final class ApplyNeighborCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple2, Vertex>, Tuple2>, - ResultTypeQueryable> { + implements CoGroupFunction, Tuple2, Vertex>, T>, + ResultTypeQueryable { private NeighborsFunctionWithVertexValue function; @@ -1045,21 +1044,20 @@ public ApplyNeighborCoGroupFunction (NeighborsFunctionWithVertexValue> vertex, - Iterable, Vertex>> neighbors, Collector> out) throws Exception { + Iterable, Vertex>> neighbors, Collector out) throws Exception { out.collect(function.iterateNeighbors(vertex.iterator().next(), neighbors)); } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, - function.getClass(), 3, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, + function.getClass(), 3, null, null); } } private static final class ApplyCoGroupFunctionOnAllNeighbors & Serializable, VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple3, Vertex>, Tuple2>, - ResultTypeQueryable> { + implements CoGroupFunction, Tuple3, Vertex>, T>, + ResultTypeQueryable { private NeighborsFunctionWithVertexValue function; @@ -1068,7 +1066,7 @@ public ApplyCoGroupFunctionOnAllNeighbors (NeighborsFunctionWithVertexValue> vertex, final Iterable, Vertex>> keysWithNeighbors, - Collector> out) throws Exception { + Collector out) throws Exception { final Iterator, Vertex>> neighborsIterator = new Iterator, Vertex>>() { @@ -1102,10 +1100,9 @@ public Iterator, Vertex>> iterator() { } @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(keyType, - TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, - function.getClass(), 3, null, null)); + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, + function.getClass(), 3, null, null); } } } diff --git a/src/main/java/flink/graphs/NeighborsFunction.java b/src/main/java/flink/graphs/NeighborsFunction.java index 51580784ef227..63bc52735a390 100644 --- a/src/main/java/flink/graphs/NeighborsFunction.java +++ b/src/main/java/flink/graphs/NeighborsFunction.java @@ -3,11 +3,10 @@ import java.io.Serializable; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; public interface NeighborsFunction & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateEdges(Iterable, Vertex>> neighbors) throws Exception; + O iterateEdges(Iterable, Vertex>> neighbors) throws Exception; } diff --git a/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java b/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java index ae7ef2ec6c87d..d7b438c837e8d 100644 --- a/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java +++ b/src/main/java/flink/graphs/NeighborsFunctionWithVertexValue.java @@ -8,5 +8,5 @@ public interface NeighborsFunctionWithVertexValue & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - Tuple2 iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) throws Exception; + O iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) throws Exception; } diff --git a/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java b/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java index 1fdccfa888341..1081bd4640204 100644 --- a/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java +++ b/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java @@ -72,7 +72,7 @@ public static String runProgram(int progId, String resultPath) throws Exception TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue>() { public Tuple2 iterateEdges( Vertex v, @@ -108,7 +108,7 @@ public Tuple2 iterateEdges( TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue>() { public Tuple2 iterateEdges( Vertex v, @@ -144,7 +144,7 @@ public Tuple2 iterateEdges( TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithMaxEdgeWeight = - graph.reduceOnEdges(new EdgesFunctionWithVertexValue() { + graph.reduceOnEdges(new EdgesFunctionWithVertexValue>() { public Tuple2 iterateEdges(Vertex v, Iterable> edges) { @@ -177,7 +177,7 @@ public Tuple2 iterateEdges(Vertex v, TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunction>() { public Tuple2 iterateEdges(Iterable>> edges) { @@ -216,7 +216,7 @@ public Tuple2 iterateEdges(Iterable>> TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithLowestOutNeighbor = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunction>() { public Tuple2 iterateEdges(Iterable>> edges) { @@ -255,7 +255,7 @@ public Tuple2 iterateEdges(Iterable>> TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithMaxEdgeWeight = - graph.reduceOnEdges(new EdgesFunction() { + graph.reduceOnEdges(new EdgesFunction>() { public Tuple2 iterateEdges(Iterable>> edges) { diff --git a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java index b8927aa034e8a..7a1413ae08579 100644 --- a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java +++ b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java @@ -72,7 +72,8 @@ public static String runProgram(int progId, String resultPath) throws Exception TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithSumOfOutNeighborValues = - graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue>() { public Tuple2 iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) { long sum = 0; @@ -101,7 +102,8 @@ public Tuple2 iterateNeighbors(Vertex vertex, TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithSum = - graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue>() { public Tuple2 iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) { long sum = 0; @@ -130,7 +132,8 @@ public Tuple2 iterateNeighbors(Vertex vertex, TestGraphUtils.getLongLongEdgeData(env), env); DataSet> verticesWithSumOfOutNeighborValues = - graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue() { + graph.reduceOnNeighbors(new NeighborsFunctionWithVertexValue>() { public Tuple2 iterateNeighbors(Vertex vertex, Iterable, Vertex>> neighbors) { long sum = 0; From c65fee55f230c56213c3ccd732d3b42cd9c305ef Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 21 Dec 2014 16:50:52 +0100 Subject: [PATCH 079/112] reduceOnNeighbors without vertex value --- src/main/java/flink/graphs/Graph.java | 100 +++++++++++--- .../java/flink/graphs/NeighborsFunction.java | 2 +- .../graphs/TestReduceOnNeighborMethods.java | 125 +++++++++++++++++- 3 files changed, 201 insertions(+), 26 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index b5cb2ac36af5e..bc91b9906ae67 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -379,7 +379,7 @@ public DataSet reduceOnEdges(EdgesFunction edgesFunction, .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); case ALL: return edges.flatMap(new EmitOneEdgePerNode()).groupBy(0) - .reduceGroup(new ApplyGroupReduceFunctionOnAllEdges(edgesFunction)); + .reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); default: throw new IllegalArgumentException("Illegal edge direction"); } @@ -400,27 +400,6 @@ public Tuple2> map(Edge edge) { } } - private static final class ApplyGroupReduceFunctionOnAllEdges & Serializable, - EV extends Serializable, T> implements GroupReduceFunction>, T>, - ResultTypeQueryable { - - private EdgesFunction function; - - public ApplyGroupReduceFunctionOnAllEdges(EdgesFunction fun) { - this.function = fun; - } - - public void reduce(final Iterable>> keysWithEdges, - Collector out) throws Exception { - out.collect(function.iterateEdges(keysWithEdges)); - } - - @Override - public TypeInformation getProducedType() { - return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null); - } - } - private static final class ApplyGroupReduceFunction & Serializable, EV extends Serializable, T> implements GroupReduceFunction>, T>, ResultTypeQueryable { @@ -1023,6 +1002,83 @@ public DataSet reduceOnNeighbors(NeighborsFunctionWithVertexValue the output type + * @return a dataset of a T + * @throws IllegalArgumentException + */ + public DataSet reduceOnNeighbors(NeighborsFunction neighborsFunction, + EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { + case IN: + // create pairs + DataSet, Vertex>> edgesWithSources = edges.join(this.vertices) + .where(0).equalTo(0).with(new ProjectVertexIdJoin(1)); + return edgesWithSources.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)); + case OUT: + // create pairs + DataSet, Vertex>> edgesWithTargets = edges.join(this.vertices) + .where(1).equalTo(0).with(new ProjectVertexIdJoin(0)); + return edgesWithTargets.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)); + case ALL: + // create pairs + DataSet, Vertex>> edgesWithNeighbors = edges.flatMap( + new EmitOneEdgeWithNeighborPerNode()).join(this.vertices) + .where(1).equalTo(0).with(new ProjectEdgeWithNeighbor()); + + return edgesWithNeighbors.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)); + default: + throw new IllegalArgumentException("Illegal edge direction"); + } + } + + private static final class ApplyNeighborGroupReduceFunction & Serializable, + VV extends Serializable, EV extends Serializable, T> implements GroupReduceFunction< + Tuple3, Vertex>, T>, ResultTypeQueryable { + + private NeighborsFunction function; + + public ApplyNeighborGroupReduceFunction(NeighborsFunction fun) { + this.function = fun; + } + + public void reduce(Iterable, Vertex>> edges, + Collector out) throws Exception { + out.collect(function.iterateNeighbors(edges)); + + } + + @Override + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(NeighborsFunction.class, function.getClass(), 3, null, null); + } + } + + private static final class ProjectVertexIdJoin & Serializable, + VV extends Serializable, EV extends Serializable> implements FlatJoinFunction, + Vertex, Tuple3, Vertex>> { + + private int fieldPosition; + + public ProjectVertexIdJoin(int position) { + this.fieldPosition = position; + } + @SuppressWarnings("unchecked") + public void join(Edge edge, Vertex otherVertex, + Collector, Vertex>> out) { + out.collect(new Tuple3, Vertex>( + (K)edge.getField(fieldPosition), edge, otherVertex)); + } + } + private static final class ProjectEdgeWithNeighbor & Serializable, VV extends Serializable, EV extends Serializable> implements FlatJoinFunction>, Vertex, Tuple3, Vertex>> { diff --git a/src/main/java/flink/graphs/NeighborsFunction.java b/src/main/java/flink/graphs/NeighborsFunction.java index 63bc52735a390..124aea0907591 100644 --- a/src/main/java/flink/graphs/NeighborsFunction.java +++ b/src/main/java/flink/graphs/NeighborsFunction.java @@ -8,5 +8,5 @@ public interface NeighborsFunction & Serializable, VV extends Serializable, EV extends Serializable, O> extends Function, Serializable { - O iterateEdges(Iterable, Vertex>> neighbors) throws Exception; + O iterateNeighbors(Iterable, Vertex>> neighbors) throws Exception; } diff --git a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java index 7a1413ae08579..be0a86751a924 100644 --- a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java +++ b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java @@ -3,11 +3,13 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; +import java.util.Iterator; import java.util.LinkedList; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; @@ -17,7 +19,7 @@ @RunWith(Parameterized.class) public class TestReduceOnNeighborMethods extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 3; + private static int NUM_PROGRAMS = 6; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -95,7 +97,7 @@ public Tuple2 iterateNeighbors(Vertex vertex, case 2: { /* * Get the sum of in-neighbor values - * time edge weights for each vertex + * times the edge weights for each vertex */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), @@ -125,6 +127,7 @@ public Tuple2 iterateNeighbors(Vertex vertex, case 3: { /* * Get the sum of all neighbor values + * including own vertex value * for each vertex */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -140,7 +143,123 @@ public Tuple2 iterateNeighbors(Vertex vertex, for (Tuple2, Vertex> neighbor : neighbors) { sum += neighbor.f1.getValue(); } - return new Tuple2(vertex.getId(), sum); + return new Tuple2(vertex.getId(), sum + vertex.getValue()); + } + }, EdgeDirection.ALL); + + verticesWithSumOfOutNeighborValues.writeAsCsv(resultPath); + env.execute(); + return "1,11\n" + + "2,6\n" + + "3,15\n" + + "4,12\n" + + "5,13\n"; + } + case 4: { + /* + * Get the sum of out-neighbor values + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSumOfOutNeighborValues = + graph.reduceOnNeighbors(new NeighborsFunction>() { + public Tuple2 iterateNeighbors( + Iterable, Vertex>> neighbors) { + long sum = 0; + Tuple3, Vertex> first = + new Tuple3, Vertex>(); + Iterator, Vertex>> neighborsIterator = + neighbors.iterator(); + if (neighborsIterator.hasNext()) { + first = neighborsIterator.next(); + sum = first.f2.getValue(); + } + while(neighborsIterator.hasNext()) { + sum += neighborsIterator.next().f2.getValue(); + } + return new Tuple2(first.f0, sum); + } + }, EdgeDirection.OUT); + + verticesWithSumOfOutNeighborValues.writeAsCsv(resultPath); + env.execute(); + return "1,5\n" + + "2,3\n" + + "3,9\n" + + "4,5\n" + + "5,1\n"; + } + case 5: { + /* + * Get the sum of in-neighbor values + * times the edge weights for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSum = + graph.reduceOnNeighbors(new NeighborsFunction>() { + public Tuple2 iterateNeighbors( + Iterable, Vertex>> neighbors) { + long sum = 0; + Tuple3, Vertex> first = + new Tuple3, Vertex>(); + Iterator, Vertex>> neighborsIterator = + neighbors.iterator(); + if (neighborsIterator.hasNext()) { + first = neighborsIterator.next(); + sum = first.f2.getValue() * first.f1.getValue(); + } + while(neighborsIterator.hasNext()) { + Tuple3, Vertex> next = neighborsIterator.next(); + sum += next.f2.getValue() * next.f1.getValue(); + } + return new Tuple2(first.f0, sum); + } + }, EdgeDirection.IN); + + + verticesWithSum.writeAsCsv(resultPath); + env.execute(); + return "1,255\n" + + "2,12\n" + + "3,59\n" + + "4,102\n" + + "5,285\n"; + } + case 6: { + /* + * Get the sum of all neighbor values + * for each vertex + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + DataSet> verticesWithSumOfOutNeighborValues = + graph.reduceOnNeighbors(new NeighborsFunction>() { + public Tuple2 iterateNeighbors( + Iterable, Vertex>> neighbors) { + long sum = 0; + Tuple3, Vertex> first = + new Tuple3, Vertex>(); + Iterator, Vertex>> neighborsIterator = + neighbors.iterator(); + if (neighborsIterator.hasNext()) { + first = neighborsIterator.next(); + sum = first.f2.getValue(); + } + while(neighborsIterator.hasNext()) { + sum += neighborsIterator.next().f2.getValue(); + } + return new Tuple2(first.f0, sum); } }, EdgeDirection.ALL); From a9b7a0ab5229a4ccba61c7cba249f424e45d7882 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 22 Dec 2014 17:32:57 +0100 Subject: [PATCH 080/112] correct graph execution environment in InvalidVertexIdsValidator --- src/main/java/flink/graphs/Graph.java | 4 ++++ .../flink/graphs/validation/InvalidVertexIdsValidator.java | 3 +-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index bc91b9906ae67..85717d8e3d303 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -92,6 +92,10 @@ public Graph(DataSet> vertices, DataSet> edges, Execut Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } + public ExecutionEnvironment getContext() { + return this.context; + } + /** * Function that checks whether a graph's ids are valid * @return diff --git a/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java b/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java index 1b68abe71f6ee..6b7a61906eb8b 100644 --- a/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java +++ b/src/main/java/flink/graphs/validation/InvalidVertexIdsValidator.java @@ -4,7 +4,6 @@ import org.apache.flink.api.common.functions.FlatMapFunction; 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.tuple.Tuple1; import org.apache.flink.util.Collector; @@ -31,7 +30,7 @@ public DataSet validate(Graph graph) { DataSet invalidIds = graph.getVertices().coGroup(edgeIds).where(0).equalTo(0) .with(new GroupInvalidIds()).first(1); - return GraphUtils.count(invalidIds.map(new KToTupleMap()), ExecutionEnvironment.getExecutionEnvironment()) + return GraphUtils.count(invalidIds.map(new KToTupleMap()), graph.getContext()) .map(new InvalidIdsMap()); } From 0ef9f0c0b4de4f452e4b2c53a728735c35f8d496 Mon Sep 17 00:00:00 2001 From: andralungu Date: Tue, 23 Dec 2014 22:35:40 +0100 Subject: [PATCH 081/112] joinWithVertices implemented and tested --- src/main/java/flink/graphs/Graph.java | 46 +++- .../java/flink/graphs/TestGraphUtils.java | 28 +++ .../flink/graphs/TestJoinWithVertices.java | 228 ++++++++++++++++++ 3 files changed, 301 insertions(+), 1 deletion(-) create mode 100644 src/test/java/flink/graphs/TestJoinWithVertices.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 85717d8e3d303..948256e5f46e6 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -179,7 +179,51 @@ public TypeInformation> getProducedType() { } } - /** + /** + * Method that joins the vertex DataSet with an input DataSet and applies a UDF on the resulted values. + * @param inputDataSet + * @param mapper - the UDF applied + * @return - a new graph where the vertex values have been updated. + */ + public Graph joinWithVertices(DataSet> inputDataSet, + final MapFunction, VV> mapper) { + + DataSet> resultedVertices = this.getVertices() + .coGroup(inputDataSet).where(0).equalTo(0) + .with(new ApplyCoGroupToVertexValues(mapper)); + + return Graph.create(resultedVertices, this.getEdges(), this.getContext()); + } + + private static final class ApplyCoGroupToVertexValues & Serializable, + VV extends Serializable, T> + implements CoGroupFunction, Tuple2, Vertex> { + + private MapFunction, VV> mapper; + + public ApplyCoGroupToVertexValues(MapFunction, VV> mapper) { + this.mapper = mapper; + } + + @Override + public void coGroup(Iterable> iterableDS1, Iterable> iterableDS2, + Collector> collector) throws Exception { + + Iterator> iteratorDS1 = iterableDS1.iterator(); + Iterator> iteratorDS2 = iterableDS2.iterator(); + + if(iteratorDS2.hasNext() && iteratorDS1.hasNext()) { + Tuple2 iteratorDS2Next = iteratorDS2.next(); + + collector.collect(new Vertex(iteratorDS2Next.f0, mapper + .map(new Tuple2(iteratorDS1.next().f1, iteratorDS2Next.f1)))); + } else if(iteratorDS1.hasNext()) { + collector.collect(iteratorDS1.next()); + } + } + } + + /** * Apply value-based filtering functions to the graph * and return a sub-graph that satisfies the predicates * for both vertex values and edge values. diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index ac2adc6a436ba..11e43e7854c92 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -6,6 +6,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; public class TestGraphUtils { @@ -35,6 +36,33 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } + public static final DataSet> getLongLongTuple2Data( + ExecutionEnvironment env) { + List> tuples = new ArrayList>(); + tuples.add(new Tuple2(1L, 10L)); + tuples.add(new Tuple2(2L, 20L)); + tuples.add(new Tuple2(3L, 30L)); + tuples.add(new Tuple2(4L, 40L)); + tuples.add(new Tuple2(6L, 60L)); + + return env.fromCollection(tuples); + } + + public static final DataSet>> getLongCustomTuple2Data( + ExecutionEnvironment env) { + List>> tuples = new ArrayList>>(); + tuples.add(new Tuple2>(1L, + new DummyCustomParameterizedType(10, 10f))); + tuples.add(new Tuple2>(2L, + new DummyCustomParameterizedType(20, 20f))); + tuples.add(new Tuple2>(3L, + new DummyCustomParameterizedType(30, 30f))); + tuples.add(new Tuple2>(4L, + new DummyCustomParameterizedType(40, 40f))); + return env.fromCollection(tuples); + } + /** * A graph with invalid vertex ids */ diff --git a/src/test/java/flink/graphs/TestJoinWithVertices.java b/src/test/java/flink/graphs/TestJoinWithVertices.java new file mode 100644 index 0000000000000..d2b7ba6614e2f --- /dev/null +++ b/src/test/java/flink/graphs/TestJoinWithVertices.java @@ -0,0 +1,228 @@ +package flink.graphs; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; + +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +@RunWith(Parameterized.class) +public class TestJoinWithVertices extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 5; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestJoinWithVertices(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch (progId) { + case 1: { + /* + * Test joinWithVertices with the input DataSet parameter identical + * to the vertex DataSet + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithVertices(graph.getVertices() + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Vertex vertex) throws Exception { + return new Tuple2(vertex.getId(), vertex.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getVertices().writeAsCsv(resultPath); + env.execute(); + + return "1,2\n" + + "2,4\n" + + "3,6\n" + + "4,8\n" + + "5,10\n"; + } + case 2: { + /* + * Test joinWithVertices with the input DataSet passed as a parameter containing + * less elements than the vertex DataSet, but of the same type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithVertices(graph.getVertices().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Vertex vertex) throws Exception { + return new Tuple2(vertex.getId(), vertex.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getVertices().writeAsCsv(resultPath); + env.execute(); + + return "1,2\n" + + "2,4\n" + + "3,6\n" + + "4,4\n" + + "5,5\n"; + } + case 3: { + /* + * Test joinWithVertices with the input DataSet passed as a parameter containing + * less elements than the vertex DataSet and of a different type(Boolean) + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithVertices(graph.getVertices().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Vertex vertex) throws Exception { + return new Tuple2(vertex.getId(), true); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + if(tuple.f1) { + return tuple.f0 * 2; + } + else { + return tuple.f0; + } + } + }); + + result.getVertices().writeAsCsv(resultPath); + env.execute(); + + return "1,2\n" + + "2,4\n" + + "3,6\n" + + "4,4\n" + + "5,5\n"; + } + case 4: { + /* + * Test joinWithVertices with an input DataSet containing different keys than the vertex DataSet + * - the iterator becomes empty. + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithVertices(TestGraphUtils.getLongLongTuple2Data(env), + new MapFunction, Long>() { + public Long map(Tuple2 tuple) throws Exception { + return tuple.f1; + } + }); + + result.getVertices().writeAsCsv(resultPath); + env.execute(); + + return "1,10\n" + + "2,20\n" + + "3,30\n" + + "4,40\n" + + "5,5\n"; + } + case 5: { + /* + * Test joinWithVertices with a DataSet containing custom parametrised type input values + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithVertices(TestGraphUtils.getLongCustomTuple2Data(env), + new MapFunction>, Long>() { + public Long map(Tuple2> tuple) throws Exception { + return (long) tuple.f1.getIntField(); + } + }); + + result.getVertices().writeAsCsv(resultPath); + env.execute(); + + return "1,10\n" + + "2,20\n" + + "3,30\n" + + "4,40\n" + + "5,5\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } +} From 2a322a0d634784a16fcf6de2401caa103b0b2ede Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 30 Dec 2014 13:48:53 +0200 Subject: [PATCH 082/112] [sssp example] only update the vertex value if new distance is smaller --- .../java/flink/graphs/library/SingleSourceShortestPaths.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java index 632233ce2f06e..7f72f2a25cf44 100644 --- a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -71,10 +71,8 @@ public void updateVertex(K vertexKey, Double vertexValue, MessageIterator minDistance) { - vertexValue = minDistance; + setNewVertexValue(minDistance); } - - setNewVertexValue(vertexValue); } } From 43a89f3080634074c06cc144ec6fd04260cecb02 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 4 Jan 2015 19:07:38 +0100 Subject: [PATCH 083/112] set joins to sort-merge joins in isWeaklyConnected --- src/main/java/flink/graphs/Graph.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 948256e5f46e6..ecffbcc24ed03 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -761,11 +762,13 @@ public DataSet isWeaklyConnected (int maxIterations) { .iterateDelta(verticesWithInitialIds, maxIterations, 0); DataSet> changes = iteration.getWorkset() - .join(edgeIds).where(0).equalTo(0) + .join(edgeIds, JoinHint.REPARTITION_SORT_MERGE) + .where(0).equalTo(0) .with(new FindNeighborsJoin()) .groupBy(0) .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()).where(0).equalTo(0) + .join(iteration.getSolutionSet(), JoinHint.REPARTITION_SORT_MERGE) + .where(0).equalTo(0) .with(new VertexWithNewComponentJoin()); DataSet> components = iteration.closeWith(changes, changes); From 9646afa8e4eb435df0e7a46d47d2199a71df6daa Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 4 Jan 2015 19:15:19 +0100 Subject: [PATCH 084/112] changed flink version to 0.9-SNAPSHOT --- pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 95ff80c54f29f..5a54456737b0e 100644 --- a/pom.xml +++ b/pom.xml @@ -49,27 +49,27 @@ under the License. org.apache.flink flink-java - 0.8-incubating-SNAPSHOT + 0.9-SNAPSHOT org.apache.flink flink-clients - 0.8-incubating-SNAPSHOT + 0.9-SNAPSHOT org.apache.flink flink-spargel - 0.8-incubating-SNAPSHOT + 0.9-SNAPSHOT org.apache.flink flink-avro - 0.8-incubating-SNAPSHOT + 0.9-SNAPSHOT org.apache.flink flink-test-utils - 0.8-incubating-SNAPSHOT + 0.9-SNAPSHOT junit From 54166e27579771d5882c4edab7e71dd3c7d77f8f Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 4 Jan 2015 19:19:23 +0100 Subject: [PATCH 085/112] get rid of Vertex to Tuple2 and Edge to Tuple3 conversions in runVertexCentricIteration --- src/main/java/flink/graphs/Graph.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index ecffbcc24ed03..d54f554f0d41b 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -52,10 +52,8 @@ import org.apache.flink.util.Collector; import org.apache.flink.types.NullValue; -import flink.graphs.utils.EdgeToTuple3Map; import flink.graphs.utils.GraphUtils; import flink.graphs.utils.Tuple2ToVertexMap; -import flink.graphs.utils.VertexToTuple2Map; import flink.graphs.validation.GraphValidator; @SuppressWarnings("serial") @@ -958,10 +956,13 @@ public Graph union (Graph graph) { * @param maximumNumberOfIterations * @return */ - public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, + @SuppressWarnings("unchecked") + public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, MessagingFunction messagingFunction, int maximumNumberOfIterations) { - DataSet> newVertices = vertices.map(new VertexToTuple2Map()).runOperation( - VertexCentricIteration.withValuedEdges(edges.map(new EdgeToTuple3Map()), + DataSet> tupleVertices = (DataSet>) (DataSet) vertices; + DataSet> tupleEdges = (DataSet>) (DataSet) edges; + DataSet> newVertices = tupleVertices.runOperation( + VertexCentricIteration.withValuedEdges(tupleEdges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); return new Graph(newVertices.map(new Tuple2ToVertexMap()), edges, context); } From d26445f0fbf346dad9464e08554d3279f1cdb4cc Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 4 Jan 2015 21:45:26 +0100 Subject: [PATCH 086/112] key and value types shouldn't be static fields; fixed #42 --- src/main/java/flink/graphs/Graph.java | 54 +++--- .../java/flink/graphs/TestGraphCreation.java | 68 +------- .../graphs/TestGraphCreationWithMapper.java | 154 ++++++++++++++++++ .../java/flink/graphs/TestGraphUtils.java | 14 ++ 4 files changed, 190 insertions(+), 100 deletions(-) create mode 100644 src/test/java/flink/graphs/TestGraphCreationWithMapper.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index d54f554f0d41b..03fbf948cb70a 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -61,18 +61,10 @@ public class Graph & Serializable, VV extends Serializab EV extends Serializable> implements Serializable { private final ExecutionEnvironment context; - private final DataSet> vertices; - private final DataSet> edges; - private boolean isUndirected; - private static TypeInformation keyType; - private static TypeInformation vertexValueType; - private static TypeInformation edgeValueType; - - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { /** a graph is directed by default */ @@ -85,10 +77,6 @@ public Graph(DataSet> vertices, DataSet> edges, Execut this.edges = edges; this.context = context; this.isUndirected = undirected; - - Graph.keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - Graph.vertexValueType = ((TupleTypeInfo) vertices.getType()).getTypeAt(1); - Graph.edgeValueType = ((TupleTypeInfo) edges.getType()).getTypeAt(2); } public ExecutionEnvironment getContext() { @@ -111,14 +99,14 @@ public DataSet> getVertices() { public DataSet> getEdges() { return edges; } - + /** * Apply a function to the attribute of each vertex in the graph. * @param mapper * @return a new graph */ public Graph mapVertices(final MapFunction, NV> mapper) { - DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper)); + DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper)); return new Graph(mappedVertices, this.getEdges(), this.context); } @@ -127,7 +115,6 @@ private static final class ApplyMapperToVertexWithType & , Vertex>, ResultTypeQueryable> { private MapFunction, NV> innerMapper; - public ApplyMapperToVertexWithType(MapFunction, NV> theMapper) { this.innerMapper = theMapper; } @@ -138,9 +125,9 @@ public Vertex map(Vertex value) throws Exception { @Override public TypeInformation> getProducedType() { - TypeInformation> vertextypeInfo = new TupleTypeInfo>(keyType, vertexValueType); - TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, vertextypeInfo); - return new TupleTypeInfo>(keyType, newVertexValueType); + return new TupleTypeInfo>( + ((TupleTypeInfo)(TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null))).getTypeAt(0), + TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); } } @@ -150,7 +137,7 @@ public TypeInformation> getProducedType() { * @return */ public Graph mapEdges(final MapFunction, NV> mapper) { - DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper)); + DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper)); return new Graph(this.vertices, mappedEdges, this.context); } @@ -170,12 +157,11 @@ public Edge map(Edge value) throws Exception { @Override public TypeInformation> getProducedType() { - TypeInformation> edgeTypeInfo = new TupleTypeInfo>(keyType, keyType, edgeValueType); - TypeInformation newEdgeValueType = TypeExtractor.getMapReturnTypes(innerMapper, - edgeTypeInfo); - - return new TupleTypeInfo>(keyType, keyType, newEdgeValueType); - } + TypeInformation keyType = ((TupleTypeInfo) + (TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null))).getTypeAt(0); + return new TupleTypeInfo>(keyType, keyType, + TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); + } } /** @@ -466,7 +452,7 @@ public void reduce(Iterable>> edges, @Override public TypeInformation getProducedType() { return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null); - } + } } private static final class EmitOneEdgePerNode & Serializable, @@ -629,22 +615,20 @@ private static final class ApplyMapperToVertexValuesWithType, Vertex>, ResultTypeQueryable> { private MapFunction innerMapper; - + public ApplyMapperToVertexValuesWithType(MapFunction theMapper) { this.innerMapper = theMapper; } - + public Vertex map(Tuple1 value) throws Exception { return new Vertex(value.f0, innerMapper.map(value.f0)); } - + @Override public TypeInformation> getProducedType() { - @SuppressWarnings("unchecked") - TypeInformation newVertexValueType = TypeExtractor.getMapReturnTypes(innerMapper, - (TypeInformation)keyType); - - return new TupleTypeInfo>(keyType, newVertexValueType); + return new TupleTypeInfo>( + TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null), + TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); } } @@ -1191,7 +1175,7 @@ public Tuple2, Vertex> next() { Tuple3, Vertex> next = keysWithEdgesIterator.next(); return new Tuple2, Vertex>(next.f1, next.f2); } - + @Override public void remove() { keysWithEdgesIterator.remove(); diff --git a/src/test/java/flink/graphs/TestGraphCreation.java b/src/test/java/flink/graphs/TestGraphCreation.java index b1b4d133142cb..8bbad9f0d6fc5 100644 --- a/src/test/java/flink/graphs/TestGraphCreation.java +++ b/src/test/java/flink/graphs/TestGraphCreation.java @@ -8,7 +8,6 @@ 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.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.types.NullValue; @@ -17,13 +16,12 @@ import org.junit.runners.Parameterized.Parameters; import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; -import flink.graphs.TestGraphUtils.DummyCustomType; import flink.graphs.validation.InvalidVertexIdsValidator; @RunWith(Parameterized.class) public class TestGraphCreation extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 5; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -104,66 +102,6 @@ public Long map(Long vertexId) { "5,5\n"; } case 3: { - /* - * Test create() with edge dataset and a mapper that assigns a double constant as value - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), - new MapFunction() { - public Double map(Long value) { - return 0.1d; - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,0.1\n" + - "2,0.1\n" + - "3,0.1\n" + - "4,0.1\n" + - "5,0.1\n"; - } - case 4: { - /* - * Test create() with edge dataset and a mapper that assigns a Tuple2 as value - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph, Long> graph = Graph.create( - TestGraphUtils.getLongLongEdgeData(env), new MapFunction>() { - public Tuple2 map(Long vertexId) { - return new Tuple2(vertexId*2, 42l); - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(2,42)\n" + - "2,(4,42)\n" + - "3,(6,42)\n" + - "4,(8,42)\n" + - "5,(10,42)\n"; - } - case 5: { - /* - * Test create() with edge dataset and a mapper that assigns a custom vertex value - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.create( - TestGraphUtils.getLongLongEdgeData(env), new MapFunction() { - public DummyCustomType map(Long vertexId) { - return new DummyCustomType(vertexId.intValue()-1, false); - } - }, env); - - graph.getVertices().writeAsCsv(resultPath); - env.execute(); - return "1,(F,0)\n" + - "2,(F,1)\n" + - "3,(F,2)\n" + - "4,(F,3)\n" + - "5,(F,4)\n"; - } - case 6: { /* * Test create() with edge dataset and a mapper that assigns a parametrized custom vertex value */ @@ -190,7 +128,7 @@ public DummyCustomParameterizedType map(Long vertexId) { "4,(8.0,3)\n" + "5,(10.0,4)\n"; } - case 7: { + case 4: { /* * Test validate(): */ @@ -204,7 +142,7 @@ public DummyCustomParameterizedType map(Long vertexId) { return "true\n"; } - case 8: { + case 5: { /* * Test validate() - invalid vertex ids */ diff --git a/src/test/java/flink/graphs/TestGraphCreationWithMapper.java b/src/test/java/flink/graphs/TestGraphCreationWithMapper.java new file mode 100644 index 0000000000000..73ecb7d012bb5 --- /dev/null +++ b/src/test/java/flink/graphs/TestGraphCreationWithMapper.java @@ -0,0 +1,154 @@ +package flink.graphs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import flink.graphs.TestGraphUtils.DummyCustomType; + +@RunWith(Parameterized.class) +public class TestGraphCreationWithMapper extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 4; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestGraphCreationWithMapper(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + case 1: { + /* + * Test create() with edge dataset and a mapper that assigns a double constant as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getLongLongEdgeData(env), + new MapFunction() { + public Double map(Long value) { + return 0.1d; + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,0.1\n" + + "2,0.1\n" + + "3,0.1\n" + + "4,0.1\n" + + "5,0.1\n"; + } + case 2: { + /* + * Test create() with edge dataset and a mapper that assigns a Tuple2 as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph, Long> graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction>() { + public Tuple2 map(Long vertexId) { + return new Tuple2(vertexId*2, 42l); + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(2,42)\n" + + "2,(4,42)\n" + + "3,(6,42)\n" + + "4,(8,42)\n" + + "5,(10,42)\n"; + } + case 3: { + /* + * Test create() with edge dataset with String key type + * and a mapper that assigns a double constant as value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create(TestGraphUtils.getStringLongEdgeData(env), + new MapFunction() { + public Double map(String value) { + return 0.1d; + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,0.1\n" + + "2,0.1\n" + + "3,0.1\n" + + "4,0.1\n" + + "5,0.1\n"; + } + case 4: { + /* + * Test create() with edge dataset and a mapper that assigns a custom vertex value + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + Graph graph = Graph.create( + TestGraphUtils.getLongLongEdgeData(env), new MapFunction() { + public DummyCustomType map(Long vertexId) { + return new DummyCustomType(vertexId.intValue()-1, false); + } + }, env); + + graph.getVertices().writeAsCsv(resultPath); + env.execute(); + return "1,(F,0)\n" + + "2,(F,1)\n" + + "3,(F,2)\n" + + "4,(F,3)\n" + + "5,(F,4)\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } + + } diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 11e43e7854c92..981661973768e 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -35,6 +35,20 @@ public static final DataSet> getLongLongEdgeData( return env.fromCollection(edges); } + + public static final DataSet> getStringLongEdgeData( + ExecutionEnvironment env) { + List> edges = new ArrayList>(); + edges.add(new Edge("1", "2", 12L)); + edges.add(new Edge("1", "3", 13L)); + edges.add(new Edge("2", "3", 23L)); + edges.add(new Edge("3", "4", 34L)); + edges.add(new Edge("3", "5", 35L)); + edges.add(new Edge("4", "5", 45L)); + edges.add(new Edge("5", "1", 51L)); + + return env.fromCollection(edges); + } public static final DataSet> getLongLongTuple2Data( ExecutionEnvironment env) { From 2729ae752431eb529be64f49774003864185a3b4 Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 6 Jan 2015 20:54:25 +0100 Subject: [PATCH 087/112] fix ClassCastException and Type errors in mapVertices; fixes #41 and #46 --- src/main/java/flink/graphs/Graph.java | 51 ++++++++++++++++++--------- 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 03fbf948cb70a..1cd5c90de7d64 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -106,7 +106,9 @@ public DataSet> getEdges() { * @return a new graph */ public Graph mapVertices(final MapFunction, NV> mapper) { - DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper)); + TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper, + keyType)); return new Graph(mappedVertices, this.getEdges(), this.context); } @@ -115,19 +117,24 @@ private static final class ApplyMapperToVertexWithType & , Vertex>, ResultTypeQueryable> { private MapFunction, NV> innerMapper; - public ApplyMapperToVertexWithType(MapFunction, NV> theMapper) { + private transient TypeInformation keyType; + public ApplyMapperToVertexWithType(MapFunction, NV> theMapper, TypeInformation keyType) { this.innerMapper = theMapper; + this.keyType = keyType; } public Vertex map(Vertex value) throws Exception { return new Vertex(value.f0, innerMapper.map(value)); } + @SuppressWarnings("unchecked") @Override public TypeInformation> getProducedType() { - return new TupleTypeInfo>( - ((TupleTypeInfo)(TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null))).getTypeAt(0), - TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); + @SuppressWarnings("rawtypes") + TypeInformation returnType = new TupleTypeInfo(Vertex.class, keyType, valueType); + return (TypeInformation>) returnType; } } @@ -137,7 +144,9 @@ public TypeInformation> getProducedType() { * @return */ public Graph mapEdges(final MapFunction, NV> mapper) { - DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper)); + TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper, + keyType)); return new Graph(this.vertices, mappedEdges, this.context); } @@ -146,21 +155,25 @@ private static final class ApplyMapperToEdgeWithType & S , Edge>, ResultTypeQueryable> { private MapFunction, NV> innerMapper; + private transient TypeInformation keyType; - public ApplyMapperToEdgeWithType(MapFunction, NV> theMapper) { + public ApplyMapperToEdgeWithType(MapFunction, NV> theMapper, TypeInformation keyType) { this.innerMapper = theMapper; + this.keyType = keyType; } public Edge map(Edge value) throws Exception { return new Edge(value.f0, value.f1, innerMapper.map(value)); } + @SuppressWarnings("unchecked") @Override public TypeInformation> getProducedType() { - TypeInformation keyType = ((TupleTypeInfo) - (TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null))).getTypeAt(0); - return new TupleTypeInfo>(keyType, keyType, - TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); + @SuppressWarnings("rawtypes") + TypeInformation returnType = new TupleTypeInfo(Edge.class, keyType, keyType, valueType); + return (TypeInformation>) returnType; } } @@ -604,9 +617,10 @@ Graph create(DataSet> edges, ExecutionEnvironment public static & Serializable, VV extends Serializable, EV extends Serializable> Graph create(DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { + TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); DataSet> vertices = edges.flatMap(new EmitSrcAndTargetAsTuple1()) - .distinct().map(new ApplyMapperToVertexValuesWithType(mapper)); + .distinct().map(new ApplyMapperToVertexValuesWithType(mapper, keyType)); return new Graph(vertices, edges, context); } @@ -615,20 +629,25 @@ private static final class ApplyMapperToVertexValuesWithType, Vertex>, ResultTypeQueryable> { private MapFunction innerMapper; + private transient TypeInformation keyType; - public ApplyMapperToVertexValuesWithType(MapFunction theMapper) { + public ApplyMapperToVertexValuesWithType(MapFunction theMapper, TypeInformation keyType) { this.innerMapper = theMapper; + this.keyType = keyType; } public Vertex map(Tuple1 value) throws Exception { return new Vertex(value.f0, innerMapper.map(value.f0)); } + @SuppressWarnings("unchecked") @Override public TypeInformation> getProducedType() { - return new TupleTypeInfo>( - TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 0, null, null), - TypeExtractor.createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null)); + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); + @SuppressWarnings("rawtypes") + TypeInformation returnType = new TupleTypeInfo(Vertex.class, keyType, valueType); + return (TypeInformation>) returnType; } } From a10e62f445b06f85ba26e6e731ce8a063a86b030 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 9 Jan 2015 10:39:07 +0100 Subject: [PATCH 088/112] Fix bug in TestReduceOnNeighborMethods caused by object reuse --- .../graphs/TestReduceOnNeighborMethods.java | 41 +++++++------------ 1 file changed, 14 insertions(+), 27 deletions(-) diff --git a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java index be0a86751a924..fec9b1222cdba 100644 --- a/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java +++ b/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java @@ -170,18 +170,14 @@ public Tuple2 iterateNeighbors(Vertex vertex, public Tuple2 iterateNeighbors( Iterable, Vertex>> neighbors) { long sum = 0; - Tuple3, Vertex> first = - new Tuple3, Vertex>(); - Iterator, Vertex>> neighborsIterator = + Tuple3, Vertex> next = null; + Iterator, Vertex>> neighborsIterator = neighbors.iterator(); - if (neighborsIterator.hasNext()) { - first = neighborsIterator.next(); - sum = first.f2.getValue(); - } while(neighborsIterator.hasNext()) { - sum += neighborsIterator.next().f2.getValue(); + next = neighborsIterator.next(); + sum += next.f2.getValue(); } - return new Tuple2(first.f0, sum); + return new Tuple2(next.f0, sum); } }, EdgeDirection.OUT); @@ -208,19 +204,14 @@ public Tuple2 iterateNeighbors( public Tuple2 iterateNeighbors( Iterable, Vertex>> neighbors) { long sum = 0; - Tuple3, Vertex> first = - new Tuple3, Vertex>(); - Iterator, Vertex>> neighborsIterator = + Tuple3, Vertex> next = null; + Iterator, Vertex>> neighborsIterator = neighbors.iterator(); - if (neighborsIterator.hasNext()) { - first = neighborsIterator.next(); - sum = first.f2.getValue() * first.f1.getValue(); - } while(neighborsIterator.hasNext()) { - Tuple3, Vertex> next = neighborsIterator.next(); + next = neighborsIterator.next(); sum += next.f2.getValue() * next.f1.getValue(); } - return new Tuple2(first.f0, sum); + return new Tuple2(next.f0, sum); } }, EdgeDirection.IN); @@ -248,18 +239,14 @@ public Tuple2 iterateNeighbors( public Tuple2 iterateNeighbors( Iterable, Vertex>> neighbors) { long sum = 0; - Tuple3, Vertex> first = - new Tuple3, Vertex>(); - Iterator, Vertex>> neighborsIterator = + Tuple3, Vertex> next = null; + Iterator, Vertex>> neighborsIterator = neighbors.iterator(); - if (neighborsIterator.hasNext()) { - first = neighborsIterator.next(); - sum = first.f2.getValue(); - } while(neighborsIterator.hasNext()) { - sum += neighborsIterator.next().f2.getValue(); + next = neighborsIterator.next(); + sum += next.f2.getValue(); } - return new Tuple2(first.f0, sum); + return new Tuple2(next.f0, sum); } }, EdgeDirection.ALL); From 66d7a577d7f77a430461371a5d29ac9de4d145c2 Mon Sep 17 00:00:00 2001 From: andralungu Date: Thu, 8 Jan 2015 18:16:31 +0100 Subject: [PATCH 089/112] joinWithEdges implemented and tested --- src/main/java/flink/graphs/Graph.java | 123 ++++ .../java/flink/graphs/TestGraphUtils.java | 90 +++ .../java/flink/graphs/TestJoinWithEdges.java | 584 ++++++++++++++++++ 3 files changed, 797 insertions(+) create mode 100644 src/test/java/flink/graphs/TestJoinWithEdges.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 1cd5c90de7d64..71a701bd73e60 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -221,6 +221,129 @@ public void coGroup(Iterable> iterableDS1, Iterable> } } + /** + * Method that joins the edge DataSet with an input DataSet on a composite key of both source and target + * and applies a UDF on the resulted values. + * @param inputDataSet + * @param mapper - the UDF applied + * @param + * @return - a new graph where the edge values have been updated. + */ + public Graph joinWithEdges(DataSet> inputDataSet, + final MapFunction, EV> mapper) { + + DataSet> resultedEdges = this.getEdges() + .coGroup(inputDataSet).where(0,1).equalTo(0,1) + .with(new ApplyCoGroupToEdgeValues(mapper)); + + return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + } + + private static final class ApplyCoGroupToEdgeValues & Serializable, + EV extends Serializable, T> + implements CoGroupFunction, Tuple3, Edge> { + + private MapFunction, EV> mapper; + + public ApplyCoGroupToEdgeValues(MapFunction, EV> mapper) { + this.mapper = mapper; + } + + @Override + public void coGroup(Iterable> iterableDS1, + Iterable> iterableDS2, + Collector> collector) throws Exception { + + Iterator> iteratorDS1 = iterableDS1.iterator(); + Iterator> iteratorDS2 = iterableDS2.iterator(); + + if(iteratorDS2.hasNext() && iteratorDS1.hasNext()) { + Tuple3 iteratorDS2Next = iteratorDS2.next(); + + collector.collect(new Edge(iteratorDS2Next.f0, iteratorDS2Next.f1, mapper + .map(new Tuple2(iteratorDS1.next().f2, iteratorDS2Next.f2)))); + + } else if(iteratorDS1.hasNext()) { + collector.collect(iteratorDS1.next()); + } + } + } + + /** + * Method that joins the edge DataSet with an input DataSet on the source key of the edges and the first attribute + * of the input DataSet and applies a UDF on the resulted values. + * Should the inputDataSet contain the same key more than once, only the first value will be considered. + * @param inputDataSet + * @param mapper - the UDF applied + * @param + * @return - a new graph where the edge values have been updated. + */ + public Graph joinWithEdgesOnSource(DataSet> inputDataSet, + final MapFunction, EV> mapper) { + + DataSet> resultedEdges = this.getEdges() + .coGroup(inputDataSet).where(0).equalTo(0) + .with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(mapper)); + + return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + } + + private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget & Serializable, + EV extends Serializable, T> + implements CoGroupFunction, Tuple2, Edge> { + + private MapFunction, EV> mapper; + + public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(MapFunction, EV> mapper) { + this.mapper = mapper; + } + + + @Override + public void coGroup(Iterable> iterableDS1, + Iterable> iterableDS2, + Collector> collector) throws Exception { + + Iterator> iteratorDS1 = iterableDS1.iterator(); + Iterator> iteratorDS2 = iterableDS2.iterator(); + + if(iteratorDS2.hasNext()) { + Tuple2 iteratorDS2Next = iteratorDS2.next(); + + while(iteratorDS1.hasNext()) { + Edge iteratorDS1Next = iteratorDS1.next(); + + collector.collect(new Edge(iteratorDS1Next.f0, iteratorDS1Next.f1, mapper + .map(new Tuple2(iteratorDS1Next.f2, iteratorDS2Next.f1)))); + } + + } else { + while(iteratorDS1.hasNext()) { + collector.collect(iteratorDS1.next()); + } + } + } + } + + /** + * Method that joins the edge DataSet with an input DataSet on the target key of the edges and the first attribute + * of the input DataSet and applies a UDF on the resulted values. + * Should the inputDataSet contain the same key more than once, only the first value will be considered. + * @param inputDataSet + * @param mapper - the UDF applied + * @param + * @return - a new graph where the edge values have been updated. + */ + public Graph joinWithEdgesOnTarget(DataSet> inputDataSet, + final MapFunction, EV> mapper) { + + DataSet> resultedEdges = this.getEdges() + .coGroup(inputDataSet).where(1).equalTo(0) + .with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(mapper)); + + return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + } + /** * Apply value-based filtering functions to the graph * and return a sub-graph that satisfies the predicates diff --git a/src/test/java/flink/graphs/TestGraphUtils.java b/src/test/java/flink/graphs/TestGraphUtils.java index 981661973768e..d5062c5715212 100644 --- a/src/test/java/flink/graphs/TestGraphUtils.java +++ b/src/test/java/flink/graphs/TestGraphUtils.java @@ -7,6 +7,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; public class TestGraphUtils { @@ -62,6 +63,48 @@ public static final DataSet> getLongLongTuple2Data( return env.fromCollection(tuples); } + public static final DataSet> getLongLongTuple2SourceData( + ExecutionEnvironment env) { + List> tuples = new ArrayList>(); + tuples.add(new Tuple2(1L, 10L)); + tuples.add(new Tuple2(1L, 20L)); + tuples.add(new Tuple2(2L, 30L)); + tuples.add(new Tuple2(3L, 40L)); + tuples.add(new Tuple2(3L, 50L)); + tuples.add(new Tuple2(4L, 60L)); + tuples.add(new Tuple2(6L, 70L)); + + return env.fromCollection(tuples); + } + + public static final DataSet> getLongLongTuple2TargetData( + ExecutionEnvironment env) { + List> tuples = new ArrayList>(); + tuples.add(new Tuple2(2L, 10L)); + tuples.add(new Tuple2(3L, 20L)); + tuples.add(new Tuple2(3L, 30L)); + tuples.add(new Tuple2(4L, 40L)); + tuples.add(new Tuple2(6L, 50L)); + tuples.add(new Tuple2(6L, 60L)); + tuples.add(new Tuple2(1L, 70L)); + + return env.fromCollection(tuples); + } + + public static final DataSet> getLongLongLongTuple3Data( + ExecutionEnvironment env) { + List> tuples = new ArrayList<>(); + tuples.add(new Tuple3(1L, 2L, 12L)); + tuples.add(new Tuple3(1L, 3L, 13L)); + tuples.add(new Tuple3(2L, 3L, 23L)); + tuples.add(new Tuple3(3L, 4L, 34L)); + tuples.add(new Tuple3(3L, 6L, 36L)); + tuples.add(new Tuple3(4L, 6L, 46L)); + tuples.add(new Tuple3(6L, 1L, 61L)); + + return env.fromCollection(tuples); + } + public static final DataSet>> getLongCustomTuple2Data( ExecutionEnvironment env) { List>> tuples = new ArrayList>> g return env.fromCollection(tuples); } + public static final DataSet>> getLongCustomTuple2SourceData( + ExecutionEnvironment env) { + List>> tuples = new ArrayList>>(); + tuples.add(new Tuple2>(1L, + new DummyCustomParameterizedType(10, 10f))); + tuples.add(new Tuple2>(1L, + new DummyCustomParameterizedType(20, 20f))); + tuples.add(new Tuple2>(2L, + new DummyCustomParameterizedType(30, 30f))); + tuples.add(new Tuple2>(3L, + new DummyCustomParameterizedType(40, 40f))); + + return env.fromCollection(tuples); + } + + public static final DataSet>> getLongCustomTuple2TargetData( + ExecutionEnvironment env) { + List>> tuples = new ArrayList>>(); + tuples.add(new Tuple2>(2L, + new DummyCustomParameterizedType(10, 10f))); + tuples.add(new Tuple2>(3L, + new DummyCustomParameterizedType(20, 20f))); + tuples.add(new Tuple2>(3L, + new DummyCustomParameterizedType(30, 30f))); + tuples.add(new Tuple2>(4L, + new DummyCustomParameterizedType(40, 40f))); + + return env.fromCollection(tuples); + } + + public static final DataSet>> getLongLongCustomTuple3Data( + ExecutionEnvironment env) { + List>> tuples = new ArrayList<>(); + tuples.add(new Tuple3>(1L, 2L, + new DummyCustomParameterizedType(10, 10f))); + tuples.add(new Tuple3>(1L, 3L, + new DummyCustomParameterizedType(20, 20f))); + tuples.add(new Tuple3>(2L, 3L, + new DummyCustomParameterizedType(30, 30f))); + tuples.add(new Tuple3>(3L, 4L, + new DummyCustomParameterizedType(40, 40f))); + + return env.fromCollection(tuples); + } + /** * A graph with invalid vertex ids */ diff --git a/src/test/java/flink/graphs/TestJoinWithEdges.java b/src/test/java/flink/graphs/TestJoinWithEdges.java new file mode 100644 index 0000000000000..711cd6106c366 --- /dev/null +++ b/src/test/java/flink/graphs/TestJoinWithEdges.java @@ -0,0 +1,584 @@ +package flink.graphs; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + +@RunWith(Parameterized.class) +public class TestJoinWithEdges extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 15; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public TestJoinWithEdges(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = GraphProgs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class GraphProgs { + + @SuppressWarnings("serial") + public static String runProgram(int progId, String resultPath) throws Exception { + + switch (progId) { + case 1: { + /* + * Test joinWithEdges with the input DataSet parameter identical + * to the edge DataSet + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdges(graph.getEdges() + .map(new MapFunction, Tuple3>() { + @Override + public Tuple3 map(Edge edge) throws Exception { + return new Tuple3(edge.getSource(), + edge.getTarget(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,68\n" + + "3,5,70\n" + + "4,5,90\n" + + "5,1,102\n"; + } + case 2: { + /* + * Test joinWithEdges with the input DataSet passed as a parameter containing + * less elements than the edge DataSet, but of the same type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdges(graph.getEdges().first(3) + .map(new MapFunction, Tuple3>() { + @Override + public Tuple3 map(Edge edge) throws Exception { + return new Tuple3(edge.getSource(), + edge.getTarget(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 3: { + /* + * Test joinWithEdges with the input DataSet passed as a parameter containing + * less elements than the edge DataSet and of a different type(Boolean) + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdges(graph.getEdges().first(3) + .map(new MapFunction, Tuple3>() { + @Override + public Tuple3 map(Edge edge) throws Exception { + return new Tuple3(edge.getSource(), + edge.getTarget(), true); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + if(tuple.f1) { + return tuple.f0 * 2; + } + else { + return tuple.f0; + } + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 4: { + /* + * Test joinWithEdges with the input DataSet containing different keys than the edge DataSet + * - the iterator becomes empty. + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdges(TestGraphUtils.getLongLongLongTuple3Data(env), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f1 * 2; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,68\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 5: { + /* + * Test joinWithEdges with a DataSet containing custom parametrised type input values + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdges(TestGraphUtils.getLongLongCustomTuple3Data(env), + new MapFunction>, Long>() { + public Long map(Tuple2> tuple) throws Exception { + return (long) tuple.f1.getIntField(); + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,10\n" + + "1,3,20\n" + + "2,3,30\n" + + "3,4,40\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 6: { + /* + * Test joinWithEdgesOnSource with the input DataSet parameter identical + * to the edge DataSet + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnSource(graph.getEdges() + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getSource(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,25\n" + + "2,3,46\n" + + "3,4,68\n" + + "3,5,69\n" + + "4,5,90\n" + + "5,1,102\n"; + } + case 7: { + /* + * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing + * less elements than the edge DataSet, but of the same type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnSource(graph.getEdges().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getSource(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,25\n" + + "2,3,46\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 8: { + /* + * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing + * less elements than the edge DataSet and of a different type(Boolean) + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnSource(graph.getEdges().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getSource(), true); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + if (tuple.f1) { + return tuple.f0 * 2; + } else { + return tuple.f0; + } + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 9: { + /* + * Test joinWithEdgesOnSource with the input DataSet containing different keys than the edge DataSet + * - the iterator becomes empty. + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnSource(TestGraphUtils.getLongLongTuple2SourceData(env), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f1 * 2; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,20\n" + + "1,3,20\n" + + "2,3,60\n" + + "3,4,80\n" + + "3,5,80\n" + + "4,5,120\n" + + "5,1,51\n"; + } + case 10: { + /* + * Test joinWithEdgesOnSource with a DataSet containing custom parametrised type input values + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnSource(TestGraphUtils.getLongCustomTuple2SourceData(env), + new MapFunction>, Long>() { + public Long map(Tuple2> tuple) throws Exception { + return (long) tuple.f1.getIntField(); + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,10\n" + + "1,3,10\n" + + "2,3,30\n" + + "3,4,40\n" + + "3,5,40\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 11: { + /* + * Test joinWithEdgesOnTarget with the input DataSet parameter identical + * to the edge DataSet + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnTarget(graph.getEdges() + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getTarget(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,36\n" + + "3,4,68\n" + + "3,5,70\n" + + "4,5,80\n" + + "5,1,102\n"; + } + case 12: { + /* + * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing + * less elements than the edge DataSet, but of the same type + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnTarget(graph.getEdges().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getTarget(), edge.getValue()); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f0 + tuple.f1; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,36\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 13: { + /* + * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing + * less elements than the edge DataSet and of a different type(Boolean) + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnTarget(graph.getEdges().first(3) + .map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.getTarget(), true); + } + }), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + if (tuple.f1) { + return tuple.f0 * 2; + } else { + return tuple.f0; + } + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,24\n" + + "1,3,26\n" + + "2,3,46\n" + + "3,4,34\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + case 14: { + /* + * Test joinWithEdgesOnTarget with the input DataSet containing different keys than the edge DataSet + * - the iterator becomes empty. + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongLongTuple2TargetData(env), + new MapFunction, Long>() { + + @Override + public Long map(Tuple2 tuple) throws Exception { + return tuple.f1 * 2; + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,20\n" + + "1,3,40\n" + + "2,3,40\n" + + "3,4,80\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,140\n"; + } + case 15: { + /* + * Test joinWithEdgesOnTarget with a DataSet containing custom parametrised type input values + */ + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), + TestGraphUtils.getLongLongEdgeData(env), env); + + Graph result = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongCustomTuple2TargetData(env), + new MapFunction>, Long>() { + public Long map(Tuple2> tuple) throws Exception { + return (long) tuple.f1.getIntField(); + } + }); + + result.getEdges().writeAsCsv(resultPath); + env.execute(); + + return "1,2,10\n" + + "1,3,20\n" + + "2,3,20\n" + + "3,4,40\n" + + "3,5,35\n" + + "4,5,45\n" + + "5,1,51\n"; + } + default: + throw new IllegalArgumentException("Invalid program id"); + } + } + } +} + From 226369d30b8692480bd66c0c605c4e746c35446c Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 10 Jan 2015 18:32:01 +0100 Subject: [PATCH 090/112] changed pageRank example to use joinWithEdgesOnSource --- src/main/java/flink/graphs/Graph.java | 90 +++++++++---------- .../flink/graphs/example/PageRankExample.java | 18 +++- .../java/flink/graphs/TestJoinWithEdges.java | 3 +- 3 files changed, 58 insertions(+), 53 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 71a701bd73e60..51b8c30f06a30 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -88,7 +88,6 @@ public ExecutionEnvironment getContext() { * @return */ public DataSet validate(GraphValidator validator) { - return validator.validate(this); } @@ -184,39 +183,39 @@ public TypeInformation> getProducedType() { * @return - a new graph where the vertex values have been updated. */ public Graph joinWithVertices(DataSet> inputDataSet, - final MapFunction, VV> mapper) { + final MapFunction, VV> mapper) { DataSet> resultedVertices = this.getVertices() .coGroup(inputDataSet).where(0).equalTo(0) .with(new ApplyCoGroupToVertexValues(mapper)); - return Graph.create(resultedVertices, this.getEdges(), this.getContext()); } private static final class ApplyCoGroupToVertexValues & Serializable, - VV extends Serializable, T> - implements CoGroupFunction, Tuple2, Vertex> { + VV extends Serializable, T> implements CoGroupFunction, Tuple2, Vertex> { private MapFunction, VV> mapper; - public ApplyCoGroupToVertexValues(MapFunction, VV> mapper) { this.mapper = mapper; } @Override - public void coGroup(Iterable> iterableDS1, Iterable> iterableDS2, + public void coGroup(Iterable> vertices, Iterable> input, Collector> collector) throws Exception { - Iterator> iteratorDS1 = iterableDS1.iterator(); - Iterator> iteratorDS2 = iterableDS2.iterator(); + final Iterator> vertexIterator = vertices.iterator(); + final Iterator> inputIterator = input.iterator(); - if(iteratorDS2.hasNext() && iteratorDS1.hasNext()) { - Tuple2 iteratorDS2Next = iteratorDS2.next(); + if (vertexIterator.hasNext()) { + if(inputIterator.hasNext()) { + final Tuple2 inputNext = inputIterator.next(); - collector.collect(new Vertex(iteratorDS2Next.f0, mapper - .map(new Tuple2(iteratorDS1.next().f1, iteratorDS2Next.f1)))); - } else if(iteratorDS1.hasNext()) { - collector.collect(iteratorDS1.next()); + collector.collect(new Vertex(inputNext.f0, mapper + .map(new Tuple2(vertexIterator.next().f1, inputNext.f1)))); + } else { + collector.collect(vertexIterator.next()); + } + } } } @@ -229,13 +228,12 @@ public void coGroup(Iterable> iterableDS1, Iterable> * @param * @return - a new graph where the edge values have been updated. */ - public Graph joinWithEdges(DataSet> inputDataSet, - final MapFunction, EV> mapper) { + public Graph joinWithEdges(DataSet> inputDataSet, + final MapFunction, EV> mapper) { DataSet> resultedEdges = this.getEdges() .coGroup(inputDataSet).where(0,1).equalTo(0,1) .with(new ApplyCoGroupToEdgeValues(mapper)); - return Graph.create(this.getVertices(), resultedEdges, this.getContext()); } @@ -244,27 +242,27 @@ private static final class ApplyCoGroupToEdgeValues & Se implements CoGroupFunction, Tuple3, Edge> { private MapFunction, EV> mapper; - public ApplyCoGroupToEdgeValues(MapFunction, EV> mapper) { this.mapper = mapper; } @Override - public void coGroup(Iterable> iterableDS1, - Iterable> iterableDS2, + public void coGroup(Iterable> edges, + Iterable> input, Collector> collector) throws Exception { - Iterator> iteratorDS1 = iterableDS1.iterator(); - Iterator> iteratorDS2 = iterableDS2.iterator(); + final Iterator> edgesIterator = edges.iterator(); + final Iterator> inputIterator = input.iterator(); - if(iteratorDS2.hasNext() && iteratorDS1.hasNext()) { - Tuple3 iteratorDS2Next = iteratorDS2.next(); + if (edgesIterator.hasNext()) { + if(inputIterator.hasNext()) { + final Tuple3 inputNext = inputIterator.next(); - collector.collect(new Edge(iteratorDS2Next.f0, iteratorDS2Next.f1, mapper - .map(new Tuple2(iteratorDS1.next().f2, iteratorDS2Next.f2)))); - - } else if(iteratorDS1.hasNext()) { - collector.collect(iteratorDS1.next()); + collector.collect(new Edge(inputNext.f0, inputNext.f1, mapper + .map(new Tuple2(edgesIterator.next().f2, inputNext.f2)))); + } else { + collector.collect(edgesIterator.next()); + } } } } @@ -279,7 +277,7 @@ public void coGroup(Iterable> iterableDS1, * @return - a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnSource(DataSet> inputDataSet, - final MapFunction, EV> mapper) { + final MapFunction, EV> mapper) { DataSet> resultedEdges = this.getEdges() .coGroup(inputDataSet).where(0).equalTo(0) @@ -289,37 +287,33 @@ public Graph joinWithEdgesOnSource(DataSet> inputDat } private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget & Serializable, - EV extends Serializable, T> - implements CoGroupFunction, Tuple2, Edge> { + EV extends Serializable, T> implements CoGroupFunction, Tuple2, Edge> { private MapFunction, EV> mapper; - public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(MapFunction, EV> mapper) { this.mapper = mapper; } - @Override - public void coGroup(Iterable> iterableDS1, - Iterable> iterableDS2, + public void coGroup(Iterable> edges, Iterable> input, Collector> collector) throws Exception { - Iterator> iteratorDS1 = iterableDS1.iterator(); - Iterator> iteratorDS2 = iterableDS2.iterator(); + final Iterator> edgesIterator = edges.iterator(); + final Iterator> inputIterator = input.iterator(); - if(iteratorDS2.hasNext()) { - Tuple2 iteratorDS2Next = iteratorDS2.next(); + if(inputIterator.hasNext()) { + final Tuple2 inputNext = inputIterator.next(); - while(iteratorDS1.hasNext()) { - Edge iteratorDS1Next = iteratorDS1.next(); + while(edgesIterator.hasNext()) { + Edge edgesNext = edgesIterator.next(); - collector.collect(new Edge(iteratorDS1Next.f0, iteratorDS1Next.f1, mapper - .map(new Tuple2(iteratorDS1Next.f2, iteratorDS2Next.f1)))); + collector.collect(new Edge(edgesNext.f0, edgesNext.f1, mapper + .map(new Tuple2(edgesNext.f2, inputNext.f1)))); } } else { - while(iteratorDS1.hasNext()) { - collector.collect(iteratorDS1.next()); + while(edgesIterator.hasNext()) { + collector.collect(edgesIterator.next()); } } } @@ -335,7 +329,7 @@ public void coGroup(Iterable> iterableDS1, * @return - a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnTarget(DataSet> inputDataSet, - final MapFunction, EV> mapper) { + final MapFunction, EV> mapper) { DataSet> resultedEdges = this.getEdges() .coGroup(inputDataSet).where(1).equalTo(0) diff --git a/src/main/java/flink/graphs/example/PageRankExample.java b/src/main/java/flink/graphs/example/PageRankExample.java index 0fc80849b464c..e3f815a45c676 100644 --- a/src/main/java/flink/graphs/example/PageRankExample.java +++ b/src/main/java/flink/graphs/example/PageRankExample.java @@ -8,11 +8,13 @@ import org.apache.flink.api.common.functions.*; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; public class PageRankExample implements ProgramDescription { - public static void main (String [] args) throws Exception { + @SuppressWarnings("serial") + public static void main (String [] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -21,9 +23,19 @@ public static void main (String [] args) throws Exception { DataSet> links = getLinksDataSet(env); Graph network = new Graph(pages, links, env); + + DataSet> vertexOutDegrees = network.outDegrees(); + + // assign the transition probabilities as the edge weights + Graph networkWithWeights = network.joinWithEdgesOnSource(vertexOutDegrees, + new MapFunction, Double>() { + public Double map(Tuple2 value) { + return value.f0 / value.f1; + } + }); DataSet> pageRanks = - network.run(new PageRank(numPages, DAMPENING_FACTOR, maxIterations)).getVertices(); + networkWithWeights.run(new PageRank(numPages, DAMPENING_FACTOR, maxIterations)).getVertices(); pageRanks.print(); @@ -60,7 +72,7 @@ public void flatMap(Long key, Collector> out) throws Exceptio int numOutEdges = (int) (Math.random() * (numPages / 2)); for (int i = 0; i < numOutEdges; i++) { long target = (long) (Math.random() * numPages) + 1; - out.collect(new Edge(key, target, 1.0 / numOutEdges)); + out.collect(new Edge(key, target, 1.0)); } } }); diff --git a/src/test/java/flink/graphs/TestJoinWithEdges.java b/src/test/java/flink/graphs/TestJoinWithEdges.java index 711cd6106c366..7375d0c973c90 100644 --- a/src/test/java/flink/graphs/TestJoinWithEdges.java +++ b/src/test/java/flink/graphs/TestJoinWithEdges.java @@ -580,5 +580,4 @@ public Long map(Tuple2> } } } -} - +} \ No newline at end of file From e23425d87dc1e5b01b978702825f02a7e7c72344 Mon Sep 17 00:00:00 2001 From: vasia Date: Sat, 10 Jan 2015 18:33:46 +0100 Subject: [PATCH 091/112] added description in GraphMetrics example --- src/main/java/flink/graphs/example/GraphMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/flink/graphs/example/GraphMetrics.java b/src/main/java/flink/graphs/example/GraphMetrics.java index 68ca82aa76099..48a5fe4fb7d50 100644 --- a/src/main/java/flink/graphs/example/GraphMetrics.java +++ b/src/main/java/flink/graphs/example/GraphMetrics.java @@ -34,7 +34,7 @@ public class GraphMetrics implements ProgramDescription { @Override public String getDescription() { - return null; + return "Graph Metrics Example"; } public static void main(String[] args) throws Exception { From 65f680914c1d4afce6178fbb4b7952a5035880b4 Mon Sep 17 00:00:00 2001 From: Carsten Brandt Date: Mon, 12 Jan 2015 20:59:13 +0100 Subject: [PATCH 092/112] Expose the full Vertex and Edge object in filter functions Expose the full Vertex and Edge object in filter functions to allow filtering by key value: - subgraph() - filterOnVertices() - filterOnEdges() fixes #56 --- src/main/java/flink/graphs/Graph.java | 46 +++---------------- .../flink/graphs/TestGraphOperations.java | 24 +++++----- 2 files changed, 19 insertions(+), 51 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 51b8c30f06a30..1990f26cfe754 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -346,10 +346,9 @@ public Graph joinWithEdgesOnTarget(DataSet> inputDat * @param edgeFilter * @return */ - public Graph subgraph(FilterFunction vertexFilter, FilterFunction edgeFilter) { + public Graph subgraph(FilterFunction> vertexFilter, FilterFunction> edgeFilter) { - DataSet> filteredVertices = this.vertices.filter( - new ApplyVertexFilter(vertexFilter)); + DataSet> filteredVertices = this.vertices.filter(vertexFilter); DataSet> remainingEdges = this.edges.join(filteredVertices) .where(0).equalTo(0) @@ -357,8 +356,7 @@ public Graph subgraph(FilterFunction vertexFilter, FilterFunction .join(filteredVertices).where(1).equalTo(0) .with(new ProjectEdge()); - DataSet> filteredEdges = remainingEdges.filter( - new ApplyEdgeFilter(edgeFilter)); + DataSet> filteredEdges = remainingEdges.filter(edgeFilter); return new Graph(filteredVertices, filteredEdges, this.context); } @@ -370,10 +368,9 @@ public Graph subgraph(FilterFunction vertexFilter, FilterFunction * @param vertexFilter * @return */ - public Graph filterOnVertices(FilterFunction vertexFilter) { + public Graph filterOnVertices(FilterFunction> vertexFilter) { - DataSet> filteredVertices = this.vertices.filter( - new ApplyVertexFilter(vertexFilter)); + DataSet> filteredVertices = this.vertices.filter(vertexFilter); DataSet> remainingEdges = this.edges.join(filteredVertices) .where(0).equalTo(0) @@ -391,9 +388,8 @@ public Graph filterOnVertices(FilterFunction vertexFilter) { * @param edgeFilter * @return */ - public Graph filterOnEdges(FilterFunction edgeFilter) { - DataSet> filteredEdges = this.edges.filter( - new ApplyEdgeFilter(edgeFilter)); + public Graph filterOnEdges(FilterFunction> edgeFilter) { + DataSet> filteredEdges = this.edges.filter(edgeFilter); return new Graph(this.vertices, filteredEdges, this.context); } @@ -408,34 +404,6 @@ public void join(Edge first, } } - private static final class ApplyVertexFilter & Serializable, - VV extends Serializable> implements FilterFunction> { - - private FilterFunction innerFilter; - - public ApplyVertexFilter(FilterFunction theFilter) { - this.innerFilter = theFilter; - } - - public boolean filter(Vertex value) throws Exception { - return innerFilter.filter(value.f1); - } - - } - - private static final class ApplyEdgeFilter & Serializable, - EV extends Serializable> implements FilterFunction> { - - private FilterFunction innerFilter; - - public ApplyEdgeFilter(FilterFunction theFilter) { - this.innerFilter = theFilter; - } - public boolean filter(Edge value) throws Exception { - return innerFilter.filter(value.f2); - } - } - /** * Return the out-degree of all vertices in the graph * @return A DataSet of Tuple2 diff --git a/src/test/java/flink/graphs/TestGraphOperations.java b/src/test/java/flink/graphs/TestGraphOperations.java index a49396dd43696..cb285c02a1f61 100644 --- a/src/test/java/flink/graphs/TestGraphOperations.java +++ b/src/test/java/flink/graphs/TestGraphOperations.java @@ -109,14 +109,14 @@ public static String runProgram(int progId, String resultPath) throws Exception Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph.subgraph(new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 2); + graph.subgraph(new FilterFunction>() { + public boolean filter(Vertex vertex) throws Exception { + return (vertex.getValue() > 2); } }, - new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 34); + new FilterFunction>() { + public boolean filter(Edge edge) throws Exception { + return (edge.getValue() > 34); } }).getEdges().writeAsCsv(resultPath); @@ -132,9 +132,9 @@ public boolean filter(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph.filterOnVertices(new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 2); + graph.filterOnVertices(new FilterFunction>() { + public boolean filter(Vertex vertex) throws Exception { + return (vertex.getValue() > 2); } }).getEdges().writeAsCsv(resultPath); @@ -151,9 +151,9 @@ public boolean filter(Long value) throws Exception { Graph graph = Graph.create(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env); - graph.filterOnEdges(new FilterFunction() { - public boolean filter(Long value) throws Exception { - return (value > 34); + graph.filterOnEdges(new FilterFunction>() { + public boolean filter(Edge edge) throws Exception { + return (edge.getValue() > 34); } }).getEdges().writeAsCsv(resultPath); From a5e19b4021df601937c7a76f6c0feab228557b3f Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 15 Jan 2015 17:39:59 +0100 Subject: [PATCH 093/112] updated readme --- README.md | 137 +++++++++++++++++++++++++----------------------------- 1 file changed, 64 insertions(+), 73 deletions(-) diff --git a/README.md b/README.md index c02383a1eb9ff..b75e95fd9668d 100644 --- a/README.md +++ b/README.md @@ -1,77 +1,68 @@ -flink-graph +Gelly =========== Graph API for Apache Flink -##Implemented Operations - -###Graph Class - -* readTuple2CsvFile -* readEdgesCsvFile -* readGraphFromCsvFile - -###Vertex Class - -###Edge Class -* reverse() - -##Tested Operations -* mapVertices() -* mapEdges() -* subgraph() -* filterOnVertices(vertexFilter) -* filterOnEdges(edgeFilter) -* outDegrees() -* inDegrees() -* getDegrees() -* getUndirected() -* reverse() -* addVertex() -* removeVertex() -* addEdge() -* removeEdge() -* union() -* isWeaklyConnected() -* runVertexCentricIteration() -* fromCollection(vertices, edges) -* getVertices() -* getEdges() -* create(vertices, edges) -* numberOfVertices() -* numberOfEdges() -* getVertexIds() -* getEdgeIds() - -##Wishlist - -###Graph Class -* fromCollection(edges) -* getNeighborhoodGraph(Vertex src, int distance) -* vertexCentricComputation() -* edgeCentricComputation() -* partitionCentricComputation() - -###Vertex Class -* getDegree() -* inDegree() -* outDegree() -* getInNeighbors() -* getOutNeighbors() -* getAllNeighbors() - - -###Edge Class - -##Other (low priority) -* partitionBy -* sample -* centrality -* pagerank -* distance -* clusteringCoefficient -* dfs -* bfs -* sssp -* isIsomorphic -* isSubgraphOf +##Implemented Methods + +###Graph Creation + +* create +* fromCollection + +###Graph Properties and Metrics + +* getVertices +* getEdges +* getVertexIds +* getEdgeIds +* numberOfVertices +* numberOfEdges +* getDegrees +* inDegrees +* outDegrees +* isWeaklyConnected + +###Graph Mutations + +* addVertex +* addEdge +* removeVertex +* removeEdge + +###Graph Transformations + +* mapVertices +* mapEdges +* union +* filterOnVertices +* filterOnEdges +* subgraph +* reverse +* getUndirected +* joinWithVertices +* joinWithEdges +* joinWithEdgesOnSource +* joinWithEdgesOnTarget + + +### Neighborhood Methods + +* reduceOnEdges +* reduceOnNeighbors + +### Graph Validation + +* validate +* InvalidVertexIdsValidator + +## Graph Algorithms + +* PageRank +* SingleSourceShortestPaths + +## Examples + +* GraphMetrics +* PageRank +* SingleSourceShortestPaths From 88d5552d6c88c712e5c4a37e0cc78afee704f03f Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 15 Jan 2015 18:14:39 +0100 Subject: [PATCH 094/112] use type hints for mapVertices, mapEdges and create --- src/main/java/flink/graphs/Graph.java | 144 ++++++++++---------------- 1 file changed, 53 insertions(+), 91 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 1990f26cfe754..fb72a200bdff3 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -57,8 +57,7 @@ import flink.graphs.validation.GraphValidator; @SuppressWarnings("serial") -public class Graph & Serializable, VV extends Serializable, - EV extends Serializable> implements Serializable { +public class Graph & Serializable, VV extends Serializable, EV extends Serializable> { private final ExecutionEnvironment context; private final DataSet> vertices; @@ -104,76 +103,52 @@ public DataSet> getEdges() { * @param mapper * @return a new graph */ - public Graph mapVertices(final MapFunction, NV> mapper) { + @SuppressWarnings({ "unchecked", "rawtypes" }) + public Graph mapVertices(final MapFunction, NV> mapper) { + TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - DataSet> mappedVertices = vertices.map(new ApplyMapperToVertexWithType(mapper, - keyType)); + + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + + TypeInformation> returnType = (TypeInformation>) + new TupleTypeInfo(Vertex.class, keyType, valueType); + + DataSet> mappedVertices = vertices + .map(new MapFunction, Vertex>() { + public Vertex map(Vertex value) throws Exception { + return new Vertex(value.f0, mapper.map(value)); + } + }) + .returns(returnType); + return new Graph(mappedVertices, this.getEdges(), this.context); } - private static final class ApplyMapperToVertexWithType & Serializable, - VV extends Serializable, NV extends Serializable> implements MapFunction - , Vertex>, ResultTypeQueryable> { - - private MapFunction, NV> innerMapper; - private transient TypeInformation keyType; - public ApplyMapperToVertexWithType(MapFunction, NV> theMapper, TypeInformation keyType) { - this.innerMapper = theMapper; - this.keyType = keyType; - } - - public Vertex map(Vertex value) throws Exception { - return new Vertex(value.f0, innerMapper.map(value)); - } - - @SuppressWarnings("unchecked") - @Override - public TypeInformation> getProducedType() { - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); - @SuppressWarnings("rawtypes") - TypeInformation returnType = new TupleTypeInfo(Vertex.class, keyType, valueType); - return (TypeInformation>) returnType; - } - } - /** * Apply a function to the attribute of each edge in the graph. * @param mapper * @return */ - public Graph mapEdges(final MapFunction, NV> mapper) { + @SuppressWarnings({ "unchecked", "rawtypes" }) + public Graph mapEdges(final MapFunction, NV> mapper) { + TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); - DataSet> mappedEdges = edges.map(new ApplyMapperToEdgeWithType(mapper, - keyType)); - return new Graph(this.vertices, mappedEdges, this.context); - } - - private static final class ApplyMapperToEdgeWithType & Serializable, - EV extends Serializable, NV extends Serializable> implements MapFunction - , Edge>, ResultTypeQueryable> { - - private MapFunction, NV> innerMapper; - private transient TypeInformation keyType; - - public ApplyMapperToEdgeWithType(MapFunction, NV> theMapper, TypeInformation keyType) { - this.innerMapper = theMapper; - this.keyType = keyType; - } - - public Edge map(Edge value) throws Exception { - return new Edge(value.f0, value.f1, innerMapper.map(value)); - } - - @SuppressWarnings("unchecked") - @Override - public TypeInformation> getProducedType() { - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); - @SuppressWarnings("rawtypes") - TypeInformation returnType = new TupleTypeInfo(Edge.class, keyType, keyType, valueType); - return (TypeInformation>) returnType; + + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + + TypeInformation> returnType = (TypeInformation>) + new TupleTypeInfo(Edge.class, keyType, keyType, valueType); + + DataSet> mappedEdges = edges.map(new MapFunction, Edge>() { + public Edge map(Edge value) throws Exception { + return new Edge(value.f0, value.f1, mapper.map(value)); } + }) + .returns(returnType); + + return new Graph(this.vertices, mappedEdges, this.context); } /** @@ -473,6 +448,7 @@ public Graph getUndirected() throws UnsupportedOperationException { */ public DataSet reduceOnEdges(EdgesFunctionWithVertexValue edgesFunction, EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { case IN: return vertices.coGroup(edges).where(0).equalTo(1).with( @@ -501,6 +477,7 @@ public DataSet reduceOnEdges(EdgesFunctionWithVertexValue e */ public DataSet reduceOnEdges(EdgesFunction edgesFunction, EdgeDirection direction) throws IllegalArgumentException { + switch (direction) { case IN: return edges.map(new ProjectVertexIdMap(1)) @@ -544,7 +521,6 @@ public ApplyGroupReduceFunction(EdgesFunction fun) { public void reduce(Iterable>> edges, Collector out) throws Exception { out.collect(function.iterateEdges(edges)); - } @Override @@ -585,6 +561,7 @@ public void coGroup(Iterable> vertex, Iterable> edges, Collector out) throws Exception { out.collect(function.iterateEdges(vertex.iterator().next(), edges)); } + @Override public TypeInformation getProducedType() { return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null); @@ -624,7 +601,7 @@ public void remove() { keysWithEdgesIterator.remove(); } }; - + Iterable> edgesIterable = new Iterable>() { public Iterator> iterator() { return edgesIterator; @@ -703,39 +680,24 @@ Graph create(DataSet> edges, ExecutionEnvironment Graph create(DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + TypeInformation> returnType = (TypeInformation>) + new TupleTypeInfo(Vertex.class, keyType, valueType); + DataSet> vertices = edges.flatMap(new EmitSrcAndTargetAsTuple1()) - .distinct().map(new ApplyMapperToVertexValuesWithType(mapper, keyType)); + .distinct().map(new MapFunction, Vertex>(){ + public Vertex map(Tuple1 value) throws Exception { + return new Vertex(value.f0, mapper.map(value.f0)); + } + }).returns(returnType); return new Graph(vertices, edges, context); } - - private static final class ApplyMapperToVertexValuesWithType & Serializable, - VV extends Serializable> implements MapFunction - , Vertex>, ResultTypeQueryable> { - - private MapFunction innerMapper; - private transient TypeInformation keyType; - public ApplyMapperToVertexValuesWithType(MapFunction theMapper, TypeInformation keyType) { - this.innerMapper = theMapper; - this.keyType = keyType; - } - - public Vertex map(Tuple1 value) throws Exception { - return new Vertex(value.f0, innerMapper.map(value.f0)); - } - - @SuppressWarnings("unchecked") - @Override - public TypeInformation> getProducedType() { - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, innerMapper.getClass(), 1, null, null); - @SuppressWarnings("rawtypes") - TypeInformation returnType = new TupleTypeInfo(Vertex.class, keyType, valueType); - return (TypeInformation>) returnType; - } - } - private static final class EmitSrcAndTarget & Serializable, EV extends Serializable> implements FlatMapFunction, Vertex> { public void flatMap(Edge edge, From ae19a54aefff2dabc56da38180e29070c4c4c14a Mon Sep 17 00:00:00 2001 From: Carsten Brandt Date: Thu, 15 Jan 2015 19:53:23 +0100 Subject: [PATCH 095/112] Updated JavaDoc for Graph class --- src/main/java/flink/graphs/Graph.java | 305 +++++++++++++++----------- 1 file changed, 180 insertions(+), 125 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index fb72a200bdff3..646c747772180 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -56,6 +56,17 @@ import flink.graphs.utils.Tuple2ToVertexMap; import flink.graphs.validation.GraphValidator; +/** + * Represents a Graph consisting of {@link Edge edges} and {@link Vertex vertices}. + * + * + * @see flink.graphs.Edge + * @see flink.graphs.Vertex + * + * @param the key type for edge and vertex identifiers + * @param the value type for vertexes + * @param the value type for edges + */ @SuppressWarnings("serial") public class Graph & Serializable, VV extends Serializable, EV extends Serializable> { @@ -64,12 +75,27 @@ public class Graph & Serializable, VV extends Serializab private final DataSet> edges; private boolean isUndirected; + /** + * Creates a graph from two datasets: vertices and edges + * + * @param vertices a DataSet of vertices. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + */ public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { /** a graph is directed by default */ this(vertices, edges, context, false); } + /** + * Creates a graph from two datasets: vertices and edges and allow setting the undirected property + * + * @param vertices a DataSet of vertices. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @param undirected whether this is an undirected graph + */ public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, boolean undirected) { this.vertices = vertices; @@ -78,29 +104,38 @@ public Graph(DataSet> vertices, DataSet> edges, Execut this.isUndirected = undirected; } + /** + * @return the flink execution environment. + */ public ExecutionEnvironment getContext() { return this.context; } /** * Function that checks whether a graph's ids are valid - * @return - */ + * @return true if the graph's ids are valid, false if not. + */ public DataSet validate(GraphValidator validator) { return validator.validate(this); } + /** + * @return the vertex dataset. + */ public DataSet> getVertices() { return vertices; } + /** + * @return the edge dataset. + */ public DataSet> getEdges() { return edges; } /** * Apply a function to the attribute of each vertex in the graph. - * @param mapper + * @param mapper the map function to apply. * @return a new graph */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -124,11 +159,11 @@ public Vertex map(Vertex value) throws Exception { return new Graph(mappedVertices, this.getEdges(), this.context); } - + /** * Apply a function to the attribute of each edge in the graph. - * @param mapper - * @return + * @param mapper the map function to apply. + * @return a new graph */ @SuppressWarnings({ "unchecked", "rawtypes" }) public Graph mapEdges(final MapFunction, NV> mapper) { @@ -152,10 +187,10 @@ public Edge map(Edge value) throws Exception { } /** - * Method that joins the vertex DataSet with an input DataSet and applies a UDF on the resulted values. - * @param inputDataSet - * @param mapper - the UDF applied - * @return - a new graph where the vertex values have been updated. + * Joins the vertex DataSet of this graph with an input DataSet and applies a UDF on the resulted values. + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. + * @return a new graph where the vertex values have been updated. */ public Graph joinWithVertices(DataSet> inputDataSet, final MapFunction, VV> mapper) { @@ -196,12 +231,12 @@ public void coGroup(Iterable> vertices, Iterable> inp } /** - * Method that joins the edge DataSet with an input DataSet on a composite key of both source and target + * Joins the edge DataSet with an input DataSet on a composite key of both source and target * and applies a UDF on the resulted values. - * @param inputDataSet - * @param mapper - the UDF applied + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. * @param - * @return - a new graph where the edge values have been updated. + * @return a new graph where the edge values have been updated. */ public Graph joinWithEdges(DataSet> inputDataSet, final MapFunction, EV> mapper) { @@ -243,13 +278,13 @@ public void coGroup(Iterable> edges, } /** - * Method that joins the edge DataSet with an input DataSet on the source key of the edges and the first attribute + * Joins the edge DataSet with an input DataSet on the source key of the edges and the first attribute * of the input DataSet and applies a UDF on the resulted values. - * Should the inputDataSet contain the same key more than once, only the first value will be considered. - * @param inputDataSet - * @param mapper - the UDF applied + * In case the inputDataSet contains the same key more than once, only the first value will be considered. + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. * @param - * @return - a new graph where the edge values have been updated. + * @return a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnSource(DataSet> inputDataSet, final MapFunction, EV> mapper) { @@ -295,13 +330,13 @@ public void coGroup(Iterable> edges, Iterable> input, } /** - * Method that joins the edge DataSet with an input DataSet on the target key of the edges and the first attribute + * Joins the edge DataSet with an input DataSet on the target key of the edges and the first attribute * of the input DataSet and applies a UDF on the resulted values. * Should the inputDataSet contain the same key more than once, only the first value will be considered. - * @param inputDataSet - * @param mapper - the UDF applied + * @param inputDataSet the DataSet to join with. + * @param mapper the UDF map function to apply. * @param - * @return - a new graph where the edge values have been updated. + * @return a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnTarget(DataSet> inputDataSet, final MapFunction, EV> mapper) { @@ -314,14 +349,14 @@ public Graph joinWithEdgesOnTarget(DataSet> inputDat } /** - * Apply value-based filtering functions to the graph - * and return a sub-graph that satisfies the predicates - * for both vertex values and edge values. - * @param vertexFilter - * @param edgeFilter - * @return - */ - public Graph subgraph(FilterFunction> vertexFilter, FilterFunction> edgeFilter) { + * Apply filtering functions to the graph + * and return a sub-graph that satisfies the predicates + * for both vertices and edges. + * @param vertexFilter the filter function for vertices. + * @param edgeFilter the filter function for edges. + * @return the resulting sub-graph. + */ + public Graph subgraph(FilterFunction> vertexFilter, FilterFunction> edgeFilter) { DataSet> filteredVertices = this.vertices.filter(vertexFilter); @@ -337,11 +372,11 @@ public Graph subgraph(FilterFunction> vertexFilter, Fil } /** - * Apply value-based filtering functions to the graph + * Apply a filtering function to the graph * and return a sub-graph that satisfies the predicates * only for the vertices. - * @param vertexFilter - * @return + * @param vertexFilter the filter function for vertices. + * @return the resulting sub-graph. */ public Graph filterOnVertices(FilterFunction> vertexFilter) { @@ -357,18 +392,18 @@ public Graph filterOnVertices(FilterFunction> vertexFil } /** - * Apply value-based filtering functions to the graph + * Apply a filtering function to the graph * and return a sub-graph that satisfies the predicates * only for the edges. - * @param edgeFilter - * @return + * @param edgeFilter the filter function for edges. + * @return the resulting sub-graph. */ public Graph filterOnEdges(FilterFunction> edgeFilter) { DataSet> filteredEdges = this.edges.filter(edgeFilter); return new Graph(this.vertices, filteredEdges, this.context); } - + @ConstantFieldsFirst("0->0;1->1;2->2") private static final class ProjectEdge & Serializable, VV extends Serializable, EV extends Serializable> implements FlatJoinFunction, Vertex, @@ -424,7 +459,7 @@ public DataSet> getDegrees() { /** * Convert the directed graph into an undirected graph * by adding all inverse-direction edges. - * + * @return the undirected graph. */ public Graph getUndirected() throws UnsupportedOperationException { if (this.isUndirected) { @@ -644,9 +679,10 @@ public Graph reverse() throws UnsupportedOperationException { /** * Creates a graph from a dataset of vertices and a dataset of edges - * @param vertices - * @param edges - * @return + * @param vertices a DataSet of vertices. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph @@ -658,8 +694,9 @@ public Graph reverse() throws UnsupportedOperationException { /** * Creates a graph from a DataSet of edges. * Vertices are created automatically and their values are set to NullValue. - * @param edges - * @return + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph */ public static & Serializable, EV extends Serializable> Graph create(DataSet> edges, ExecutionEnvironment context) { @@ -674,7 +711,7 @@ Graph create(DataSet> edges, ExecutionEnvironment * by applying the provided map function to the vertex ids. * @param edges the input edges * @param mapper the map function to set the initial vertex value - * @return + * @return the newly created graph */ public static & Serializable, VV extends Serializable, EV extends Serializable> Graph create(DataSet> edges, final MapFunction mapper, @@ -718,19 +755,32 @@ public void flatMap(Edge edge, Collector> out) { } /** - * Read and create the graph Tuple2 dataset from a csv file - * @param env - * @param filePath - * @param delimiter - * @param Tuple2IdClass - * @param Tuple2ValueClass - * @return + * Read and create the graph vertex Tuple2 DataSet from a csv file + * + * The CSV file should be of the following format: + * + * + * + * For example, with space delimiter: + * + * 1 57 + * 2 45 + * 3 77 + * 4 12 + * + * @param context the flink execution environment. + * @param filePath the path to the CSV file. + * @param delimiter the CSV delimiter. + * @param Tuple2IdClass The class to use for Vertex IDs + * @param Tuple2ValueClass The class to use for Vertex Values + * @return a set of vertices and their values. */ public static & Serializable, VV extends Serializable> - DataSet> readTuple2CsvFile(ExecutionEnvironment env, String filePath, + DataSet> + readTuple2CsvFile(ExecutionEnvironment context, String filePath, char delimiter, Class Tuple2IdClass, Class Tuple2ValueClass) { - CsvReader reader = new CsvReader(filePath, env); + CsvReader reader = new CsvReader(filePath, context); DataSet> vertices = reader.fieldDelimiter(delimiter).types(Tuple2IdClass, Tuple2ValueClass) .map(new MapFunction, Tuple2>() { @@ -741,23 +791,21 @@ public Tuple2 map(Tuple2 value) throws Exception { return vertices; } - /** - * @return Singleton DataSet containing the vertex count - */ + /** + * @return Singleton DataSet containing the vertex count + */ public DataSet numberOfVertices () { return GraphUtils.count(vertices, context); } - /** - * - * @return Singleton DataSet containing the edge count - */ + /** + * @return Singleton DataSet containing the edge count + */ public DataSet numberOfEdges () { return GraphUtils.count(edges, context); } /** - * * @return The IDs of the vertices as DataSet */ public DataSet getVertexIds () { @@ -772,6 +820,9 @@ public K map(Vertex vertex) { } } + /** + * @return The IDs of the edges as DataSet + */ public DataSet> getEdgeIds () { return edges.map(new ExtractEdgeIDsMapper()); } @@ -784,11 +835,11 @@ public Tuple2 map(Edge edge) throws Exception { } } - /** - * Checks the weak connectivity of a graph. - * @param maxIterations the maximum number of iterations for the inner delta iteration - * @return true if the graph is weakly connected. - */ + /** + * Checks the weak connectivity of a graph. + * @param maxIterations the maximum number of iterations for the inner delta iteration + * @return true if the graph is weakly connected. + */ public DataSet isWeaklyConnected (int maxIterations) { Graph graph; @@ -870,17 +921,17 @@ public Graph fromCollection (Collection> vertices, Colle DataSet> e = context.fromCollection(edges); return new Graph(v, e, context); - } + } - /** - * Adds the input vertex and edges to the graph. - * If the vertex already exists in the graph, it will not be added again, - * but the given edges will. - * @param vertex - * @param edges - * @return - */ - @SuppressWarnings("unchecked") + /** + * Adds the input vertex and edges to the graph. + * If the vertex already exists in the graph, it will not be added again, + * but the given edges will. + * @param vertex the vertex to add to the graph + * @param edges a list of edges to add to the grap + * @return the new graph containing the existing and newly added vertices and edges + */ + @SuppressWarnings("unchecked") public Graph addVertex (final Vertex vertex, List> edges) { DataSet> newVertex = this.context.fromElements(vertex); @@ -896,14 +947,14 @@ public Graph addVertex (final Vertex vertex, List> return Graph.create(newVertices, newEdges, context); } - /** + /** * Adds the given edge to the graph. * If the source and target vertices do not exist in the graph, * they will also be added. - * @param source - * @param target - * @param edgeValue - * @return + * @param source the source vertex of the edge + * @param target the target vertex of the edge + * @param edgeValue the edge value + * @return the new graph containing the existing vertices and edges plus the newly added edge */ public Graph addEdge (Vertex source, Vertex target, EV edgeValue) { Graph partialGraph = this.fromCollection(Arrays.asList(source, target), @@ -911,12 +962,13 @@ public Graph addEdge (Vertex source, Vertex target, EV ed return this.union(partialGraph); } - /** - * Removes the given vertex and its edges from the graph. - * @param vertex - * @return - */ - public Graph removeVertex (Vertex vertex) { + /** + * Removes the given vertex and its edges from the graph. + * @param vertex the vertex to remove + * @return the new graph containing the existing vertices and edges without the removed vertex and its edges + */ + public Graph removeVertex (Vertex vertex) { + DataSet> newVertices = getVertices().filter( new RemoveVertexFilter(vertex)); DataSet> newEdges = getEdges().filter( @@ -960,11 +1012,11 @@ public boolean filter(Edge edge) throws Exception { return true; } } - + /** * Removes all edges that match the given edge from the graph. - * @param edge - * @return + * @param edge the edge to remove + * @return the new graph containing the existing vertices and edges without the removed edges */ public Graph removeEdge (Edge edge) { DataSet> newEdges = getEdges().filter( @@ -990,8 +1042,8 @@ public boolean filter(Edge edge) { /** * Performs union on the vertices and edges sets of the input graphs * removing duplicate vertices but maintaining duplicate edges. - * @param graph - * @return + * @param graph the graph to perform union with + * @return a new graph */ public Graph union (Graph graph) { DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); @@ -999,14 +1051,14 @@ public Graph union (Graph graph) { return new Graph(unionedVertices, unionedEdges, this.context); } - /** - * Runs a Vertex-Centric iteration on the graph. - * @param vertexUpdateFunction - * @param messagingFunction - * @param maximumNumberOfIterations - * @return - */ - @SuppressWarnings("unchecked") + /** + * Runs a Vertex-Centric iteration on the graph. + * @param vertexUpdateFunction the vertex update function + * @param messagingFunction the messaging function + * @param maximumNumberOfIterations maximum number of iterations to perform + * @return + */ + @SuppressWarnings("unchecked") public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, MessagingFunction messagingFunction, int maximumNumberOfIterations) { DataSet> tupleVertices = (DataSet>) (DataSet) vertices; @@ -1018,49 +1070,52 @@ public Graph runVertexCentricIteration(VertexUpdateFunction & Serializable, VV extends Serializable, - EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, - Collection> v, Collection> e) throws Exception { - DataSet> vertices = env.fromCollection(v); - DataSet> edges = env.fromCollection(e); - - return Graph.create(vertices, edges, env); + * Creates a graph from the given vertex and edge collections + * @param context the flink execution environment. + * @param v the collection of vertices + * @param e the collection of edges + * @return a new graph formed from the set of edges and vertices + */ + public static & Serializable, VV extends Serializable, + EV extends Serializable> Graph + fromCollection(ExecutionEnvironment context, Collection> v, + Collection> e) throws Exception { + + DataSet> vertices = context.fromCollection(v); + DataSet> edges = context.fromCollection(e); + + return Graph.create(vertices, edges, context); } /** * Vertices may not have a value attached or may receive a value as a result of running the algorithm. - * @param env + * @param context the flink execution environment. * @param e the collection of edges * @return a new graph formed from the edges, with no value for the vertices */ public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, - Collection> e) { + EV extends Serializable> Graph + fromCollection(ExecutionEnvironment context, Collection> e) { - DataSet> edges = env.fromCollection(e); + DataSet> edges = context.fromCollection(e); - return Graph.create(edges, env); + return Graph.create(edges, context); } /** * Vertices may have an initial value defined by a function. - * @param env + * @param context the flink execution environment. * @param e the collection of edges * @return a new graph formed from the edges, with a custom value for the vertices, * determined by the mapping function */ public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph fromCollection(ExecutionEnvironment env, - Collection> e, - final MapFunction mapper) { - DataSet> edges = env.fromCollection(e); - return Graph.create(edges, mapper, env); + EV extends Serializable> Graph + fromCollection(ExecutionEnvironment context, Collection> e, + final MapFunction mapper) { + + DataSet> edges = context.fromCollection(e); + return Graph.create(edges, mapper, context); } public Graph run (GraphAlgorithm algorithm) { From db3d81381cf2b6604ef422b0264f7fcb8cb77d0f Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 8 Jan 2015 17:40:35 +0100 Subject: [PATCH 096/112] cloned spargel classes --- pom.xml | 5 - src/main/java/flink/graphs/Graph.java | 6 +- .../java/flink/graphs/library/PageRank.java | 9 +- .../library/SingleSourceShortestPaths.java | 8 +- .../flink/graphs/spargel/MessageIterator.java | 58 ++ .../graphs/spargel/MessagingFunction.java | 283 +++++++++ .../flink/graphs/spargel/OutgoingEdge.java | 64 ++ .../spargel/VertexCentricIteration.java | 599 ++++++++++++++++++ .../graphs/spargel/VertexUpdateFunction.java | 145 +++++ .../TestVertexCentricConnectedComponents.java | 7 +- 10 files changed, 1164 insertions(+), 20 deletions(-) create mode 100644 src/main/java/flink/graphs/spargel/MessageIterator.java create mode 100644 src/main/java/flink/graphs/spargel/MessagingFunction.java create mode 100644 src/main/java/flink/graphs/spargel/OutgoingEdge.java create mode 100644 src/main/java/flink/graphs/spargel/VertexCentricIteration.java create mode 100644 src/main/java/flink/graphs/spargel/VertexUpdateFunction.java diff --git a/pom.xml b/pom.xml index 5a54456737b0e..49b9b98af87d4 100644 --- a/pom.xml +++ b/pom.xml @@ -56,11 +56,6 @@ under the License. flink-clients 0.9-SNAPSHOT - - org.apache.flink - flink-spargel - 0.9-SNAPSHOT - org.apache.flink flink-avro diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 646c747772180..247f5fcb2d27f 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -46,12 +46,12 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.VertexCentricIteration; -import org.apache.flink.spargel.java.VertexUpdateFunction; import org.apache.flink.util.Collector; import org.apache.flink.types.NullValue; +import flink.graphs.spargel.MessagingFunction; +import flink.graphs.spargel.VertexCentricIteration; +import flink.graphs.spargel.VertexUpdateFunction; import flink.graphs.utils.GraphUtils; import flink.graphs.utils.Tuple2ToVertexMap; import flink.graphs.validation.GraphValidator; diff --git a/src/main/java/flink/graphs/library/PageRank.java b/src/main/java/flink/graphs/library/PageRank.java index 7ab9964bb5dbc..b7ca52bd08194 100644 --- a/src/main/java/flink/graphs/library/PageRank.java +++ b/src/main/java/flink/graphs/library/PageRank.java @@ -3,11 +3,10 @@ import flink.graphs.Graph; import flink.graphs.GraphAlgorithm; - -import org.apache.flink.spargel.java.MessageIterator; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.OutgoingEdge; -import org.apache.flink.spargel.java.VertexUpdateFunction; +import flink.graphs.spargel.MessageIterator; +import flink.graphs.spargel.MessagingFunction; +import flink.graphs.spargel.OutgoingEdge; +import flink.graphs.spargel.VertexUpdateFunction; import java.io.Serializable; diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java index 7f72f2a25cf44..c29909c0e0128 100644 --- a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -1,12 +1,12 @@ package flink.graphs.library; import flink.graphs.*; +import flink.graphs.spargel.MessageIterator; +import flink.graphs.spargel.MessagingFunction; +import flink.graphs.spargel.OutgoingEdge; +import flink.graphs.spargel.VertexUpdateFunction; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.spargel.java.MessageIterator; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.OutgoingEdge; -import org.apache.flink.spargel.java.VertexUpdateFunction; import java.io.Serializable; diff --git a/src/main/java/flink/graphs/spargel/MessageIterator.java b/src/main/java/flink/graphs/spargel/MessageIterator.java new file mode 100644 index 0000000000000..08ba2c095bc97 --- /dev/null +++ b/src/main/java/flink/graphs/spargel/MessageIterator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package flink.graphs.spargel; + +import java.util.Iterator; + +import org.apache.flink.api.java.tuple.Tuple2; + +/** + * An iterator that returns messages. The iterator is {@link java.lang.Iterable} at the same time to support + * the foreach syntax. + */ +public final class MessageIterator implements Iterator, Iterable, java.io.Serializable { + private static final long serialVersionUID = 1L; + + private transient Iterator> source; + + + final void setSource(Iterator> source) { + this.source = source; + } + + @Override + public final boolean hasNext() { + return this.source.hasNext(); + } + + @Override + public final Message next() { + return this.source.next().f1; + } + + @Override + public final void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator iterator() { + return this; + } +} diff --git a/src/main/java/flink/graphs/spargel/MessagingFunction.java b/src/main/java/flink/graphs/spargel/MessagingFunction.java new file mode 100644 index 0000000000000..52a881eef2f0b --- /dev/null +++ b/src/main/java/flink/graphs/spargel/MessagingFunction.java @@ -0,0 +1,283 @@ +/* + * 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 flink.graphs.spargel; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; + +import org.apache.flink.api.common.aggregators.Aggregator; +import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.types.Value; +import org.apache.flink.util.Collector; + +/** + * The base class for functions that produce messages between vertices as a part of a {@link VertexCentricIteration}. + * + * @param The type of the vertex key (the vertex identifier). + * @param The type of the vertex value (the state of the vertex). + * @param The type of the message sent between vertices along the edges. + * @param The type of the values that are associated with the edges. + */ +public abstract class MessagingFunction, VertexValue, Message, EdgeValue> implements Serializable { + + private static final long serialVersionUID = 1L; + + // -------------------------------------------------------------------------------------------- + // Public API Methods + // -------------------------------------------------------------------------------------------- + + /** + * This method is invoked once per superstep for each vertex that was changed in that superstep. + * It needs to produce the messages that will be received by vertices in the next superstep. + * + * @param vertexKey The key of the vertex that was changed. + * @param vertexValue The value (state) of the vertex that was changed. + * + * @throws Exception The computation may throw exceptions, which causes the superstep to fail. + */ + public abstract void sendMessages(VertexKey vertexKey, VertexValue vertexValue) throws Exception; + + /** + * This method is executed one per superstep before the vertex update function is invoked for each vertex. + * + * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail. + */ + public void preSuperstep() throws Exception {} + + /** + * This method is executed one per superstep after the vertex update function has been invoked for each vertex. + * + * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail. + */ + public void postSuperstep() throws Exception {} + + + /** + * Gets an {@link java.lang.Iterable} with all outgoing edges. This method is mutually exclusive with + * {@link #sendMessageToAllNeighbors(Object)} and may be called only once. + * + * @return An iterator with all outgoing edges. + */ + @SuppressWarnings("unchecked") + public Iterable> getOutgoingEdges() { + if (edgesUsed) { + throw new IllegalStateException("Can use either 'getOutgoingEdges()' or 'sendMessageToAllTargets()' exactly once."); + } + edgesUsed = true; + + if (this.edgeWithValueIter != null) { + this.edgeWithValueIter.set((Iterator>) edges); + return this.edgeWithValueIter; + } else { + this.edgeNoValueIter.set((Iterator>) edges); + return this.edgeNoValueIter; + } + } + + /** + * Sends the given message to all vertices that are targets of an outgoing edge of the changed vertex. + * This method is mutually exclusive to the method {@link #getOutgoingEdges()} and may be called only once. + * + * @param m The message to send. + */ + public void sendMessageToAllNeighbors(Message m) { + if (edgesUsed) { + throw new IllegalStateException("Can use either 'getOutgoingEdges()' or 'sendMessageToAllTargets()' exactly once."); + } + + edgesUsed = true; + + outValue.f1 = m; + + while (edges.hasNext()) { + Tuple next = (Tuple) edges.next(); + VertexKey k = next.getField(1); + outValue.f0 = k; + out.collect(outValue); + } + } + + /** + * Sends the given message to the vertex identified by the given key. If the target vertex does not exist, + * the next superstep will cause an exception due to a non-deliverable message. + * + * @param target The key (id) of the target vertex to message. + * @param m The message. + */ + public void sendMessageTo(VertexKey target, Message m) { + outValue.f0 = target; + outValue.f1 = m; + out.collect(outValue); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Gets the number of the superstep, starting at 1. + * + * @return The number of the current superstep. + */ + public int getSuperstepNumber() { + return this.runtimeContext.getSuperstepNumber(); + } + + /** + * Gets the iteration aggregator registered under the given name. The iteration aggregator is combines + * all aggregates globally once per superstep and makes them available in the next superstep. + * + * @param name The name of the aggregator. + * @return The aggregator registered under this name, or null, if no aggregator was registered. + */ + public > T getIterationAggregator(String name) { + return this.runtimeContext.getIterationAggregator(name); + } + + /** + * Get the aggregated value that an aggregator computed in the previous iteration. + * + * @param name The name of the aggregator. + * @return The aggregated value of the previous iteration. + */ + public T getPreviousIterationAggregate(String name) { + return this.runtimeContext.getPreviousIterationAggregate(name); + } + + /** + * Gets the broadcast data set registered under the given name. Broadcast data sets + * are available on all parallel instances of a function. They can be registered via + * {@link VertexCentricIteration#addBroadcastSetForMessagingFunction(String, org.apache.flink.api.java.DataSet)}. + * + * @param name The name under which the broadcast set is registered. + * @return The broadcast data set. + */ + public Collection getBroadcastSet(String name) { + return this.runtimeContext.getBroadcastVariable(name); + } + + // -------------------------------------------------------------------------------------------- + // internal methods and state + // -------------------------------------------------------------------------------------------- + + private Tuple2 outValue; + + private IterationRuntimeContext runtimeContext; + + private Iterator edges; + + private Collector> out; + + private EdgesIteratorNoEdgeValue edgeNoValueIter; + + private EdgesIteratorWithEdgeValue edgeWithValueIter; + + private boolean edgesUsed; + + + void init(IterationRuntimeContext context, boolean hasEdgeValue) { + this.runtimeContext = context; + this.outValue = new Tuple2(); + + if (hasEdgeValue) { + this.edgeWithValueIter = new EdgesIteratorWithEdgeValue(); + } else { + this.edgeNoValueIter = new EdgesIteratorNoEdgeValue(); + } + } + + void set(Iterator edges, Collector> out) { + this.edges = edges; + this.out = out; + this.edgesUsed = false; + } + + + + private static final class EdgesIteratorNoEdgeValue, EdgeValue> + implements Iterator>, Iterable> + { + private Iterator> input; + + private OutgoingEdge edge = new OutgoingEdge(); + + + void set(Iterator> input) { + this.input = input; + } + + @Override + public boolean hasNext() { + return input.hasNext(); + } + + @Override + public OutgoingEdge next() { + Tuple2 next = input.next(); + edge.set(next.f1, null); + return edge; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator> iterator() { + return this; + } + } + + + private static final class EdgesIteratorWithEdgeValue, EdgeValue> + implements Iterator>, Iterable> + { + private Iterator> input; + + private OutgoingEdge edge = new OutgoingEdge(); + + void set(Iterator> input) { + this.input = input; + } + + @Override + public boolean hasNext() { + return input.hasNext(); + } + + @Override + public OutgoingEdge next() { + Tuple3 next = input.next(); + edge.set(next.f1, next.f2); + return edge; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + @Override + public Iterator> iterator() { + return this; + } + } +} diff --git a/src/main/java/flink/graphs/spargel/OutgoingEdge.java b/src/main/java/flink/graphs/spargel/OutgoingEdge.java new file mode 100644 index 0000000000000..7505409ae6c19 --- /dev/null +++ b/src/main/java/flink/graphs/spargel/OutgoingEdge.java @@ -0,0 +1,64 @@ +/* + * 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 flink.graphs.spargel; + +/** + * Edge objects represent edges between vertices. Edges are defined by their source and target + * vertex id. Edges may have an associated value (for example a weight or a distance), if the + * graph algorithm was initialized with the + * {@link VertexCentricIteration#withValuedEdges(org.apache.flink.api.java.DataSet, VertexUpdateFunction, MessagingFunction, int)} + * method. + * + * @param The type of the vertex key. + * @param The type of the value associated with the edge. For scenarios where the edges do not hold + * value, this type may be arbitrary. + */ +public final class OutgoingEdge, EdgeValue> implements java.io.Serializable { + + private static final long serialVersionUID = 1L; + + private VertexKey target; + + private EdgeValue edgeValue; + + void set(VertexKey target, EdgeValue edgeValue) { + this.target = target; + this.edgeValue = edgeValue; + } + + /** + * Gets the target vertex id. + * + * @return The target vertex id. + */ + public VertexKey target() { + return target; + } + + /** + * Gets the value associated with the edge. The value may be null if the iteration was initialized with + * an edge data set without edge values. + * Typical examples of edge values are weights or distances of the path represented by the edge. + * + * @return The value associated with the edge. + */ + public EdgeValue edgeValue() { + return edgeValue; + } +} diff --git a/src/main/java/flink/graphs/spargel/VertexCentricIteration.java b/src/main/java/flink/graphs/spargel/VertexCentricIteration.java new file mode 100644 index 0000000000000..b15c8c49b88db --- /dev/null +++ b/src/main/java/flink/graphs/spargel/VertexCentricIteration.java @@ -0,0 +1,599 @@ +/* + * 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 flink.graphs.spargel; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.Validate; +import org.apache.flink.api.common.aggregators.Aggregator; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DeltaIteration; +import org.apache.flink.api.common.functions.RichCoGroupFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.operators.CoGroupOperator; +import org.apache.flink.api.java.operators.CustomUnaryOperation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; + +/** + * This class represents iterative graph computations, programmed in a vertex-centric perspective. + * It is a special case of Bulk Synchronous Parallel computation. The paradigm has also been + * implemented by Google's Pregel system and by Apache Giraph. + *

+ * Vertex centric algorithms operate on graphs, which are defined through vertices and edges. The + * algorithms send messages along the edges and update the state of vertices based on + * the old state and the incoming messages. All vertices have an initial state. + * The computation terminates once no vertex updates it state any more. + * Additionally, a maximum number of iterations (supersteps) may be specified. + *

+ * The computation is here represented by two functions: + *

    + *
  • The {@link VertexUpdateFunction} receives incoming messages and may updates the state for + * the vertex. If a state is updated, messages are sent from this vertex. Initially, all vertices are + * considered updated.
  • + *
  • The {@link MessagingFunction} takes the new vertex state and sends messages along the outgoing + * edges of the vertex. The outgoing edges may optionally have an associated value, such as a weight.
  • + *
+ *

+ * Vertex-centric graph iterations are instantiated by the + * {@link #withPlainEdges(DataSet, VertexUpdateFunction, MessagingFunction, int)} method, or the + * {@link #withValuedEdges(DataSet, VertexUpdateFunction, MessagingFunction, int)} method, depending on whether + * the graph's edges are carrying values. + * + * @param The type of the vertex key (the vertex identifier). + * @param The type of the vertex value (the state of the vertex). + * @param The type of the message sent between vertices along the edges. + * @param The type of the values that are associated with the edges. + */ +public class VertexCentricIteration, VertexValue, Message, EdgeValue> + implements CustomUnaryOperation, Tuple2> +{ + private final VertexUpdateFunction updateFunction; + + private final MessagingFunction messagingFunction; + + private final DataSet> edgesWithoutValue; + + private final DataSet> edgesWithValue; + + private final Map> aggregators; + + private final int maximumNumberOfIterations; + + private final List>> bcVarsUpdate = new ArrayList>>(4); + + private final List>> bcVarsMessaging = new ArrayList>>(4); + + private final TypeInformation messageType; + + private DataSet> initialVertices; + + private String name; + + private int parallelism = -1; + + private boolean unmanagedSolutionSet; + + // ---------------------------------------------------------------------------------- + + private VertexCentricIteration(VertexUpdateFunction uf, + MessagingFunction mf, + DataSet> edgesWithoutValue, + int maximumNumberOfIterations) + { + Validate.notNull(uf); + Validate.notNull(mf); + Validate.notNull(edgesWithoutValue); + Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); + + // check that the edges are actually a valid tuple set of vertex key types + TypeInformation> edgesType = edgesWithoutValue.getType(); + Validate.isTrue(edgesType.isTupleType() && edgesType.getArity() == 2, "The edges data set (for edges without edge values) must consist of 2-tuples."); + + TupleTypeInfo tupleInfo = (TupleTypeInfo) edgesType; + Validate.isTrue(tupleInfo.getTypeAt(0).equals(tupleInfo.getTypeAt(1)) + && Comparable.class.isAssignableFrom(tupleInfo.getTypeAt(0).getTypeClass()), + "Both tuple fields (source and target vertex id) must be of the data type that represents the vertex key and implement the java.lang.Comparable interface."); + + this.updateFunction = uf; + this.messagingFunction = mf; + this.edgesWithoutValue = edgesWithoutValue; + this.edgesWithValue = null; + this.maximumNumberOfIterations = maximumNumberOfIterations; + this.aggregators = new HashMap>(); + + this.messageType = getMessageType(mf); + } + + private VertexCentricIteration(VertexUpdateFunction uf, + MessagingFunction mf, + DataSet> edgesWithValue, + int maximumNumberOfIterations, + boolean edgeHasValueMarker) + { + Validate.notNull(uf); + Validate.notNull(mf); + Validate.notNull(edgesWithValue); + Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); + + // check that the edges are actually a valid tuple set of vertex key types + TypeInformation> edgesType = edgesWithValue.getType(); + Validate.isTrue(edgesType.isTupleType() && edgesType.getArity() == 3, "The edges data set (for edges with edge values) must consist of 3-tuples."); + + TupleTypeInfo tupleInfo = (TupleTypeInfo) edgesType; + Validate.isTrue(tupleInfo.getTypeAt(0).equals(tupleInfo.getTypeAt(1)) + && Comparable.class.isAssignableFrom(tupleInfo.getTypeAt(0).getTypeClass()), + "The first two tuple fields (source and target vertex id) must be of the data type that represents the vertex key and implement the java.lang.Comparable interface."); + + Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); + + this.updateFunction = uf; + this.messagingFunction = mf; + this.edgesWithoutValue = null; + this.edgesWithValue = edgesWithValue; + this.maximumNumberOfIterations = maximumNumberOfIterations; + this.aggregators = new HashMap>(); + + this.messageType = getMessageType(mf); + } + + private TypeInformation getMessageType(MessagingFunction mf) { + return TypeExtractor.createTypeInfo(MessagingFunction.class, mf.getClass(), 2, null, null); + } + + /** + * Registers a new aggregator. Aggregators registered here are available during the execution of the vertex updates + * via {@link VertexUpdateFunction#getIterationAggregator(String)} and + * {@link VertexUpdateFunction#getPreviousIterationAggregate(String)}. + * + * @param name The name of the aggregator, used to retrieve it and its aggregates during execution. + * @param aggregator The aggregator. + */ + public void registerAggregator(String name, Aggregator aggregator) { + this.aggregators.put(name, aggregator); + } + + /** + * Adds a data set as a broadcast set to the messaging function. + * + * @param name The name under which the broadcast data is available in the messaging function. + * @param data The data set to be broadcasted. + */ + public void addBroadcastSetForMessagingFunction(String name, DataSet data) { + this.bcVarsMessaging.add(new Tuple2>(name, data)); + } + + /** + * Adds a data set as a broadcast set to the vertex update function. + * + * @param name The name under which the broadcast data is available in the vertex update function. + * @param data The data set to be broadcasted. + */ + public void addBroadcastSetForUpdateFunction(String name, DataSet data) { + this.bcVarsUpdate.add(new Tuple2>(name, data)); + } + + /** + * Sets the name for the vertex-centric iteration. The name is displayed in logs and messages. + * + * @param name The name for the iteration. + */ + public void setName(String name) { + this.name = name; + } + + /** + * Gets the name from this vertex-centric iteration. + * + * @return The name of the iteration. + */ + public String getName() { + return name; + } + + /** + * Sets the degree of parallelism for the iteration. + * + * @param parallelism The degree of parallelism. + */ + public void setParallelism(int parallelism) { + Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default)."); + this.parallelism = parallelism; + } + + /** + * Gets the iteration's degree of parallelism. + * + * @return The iterations parallelism, or -1, if not set. + */ + public int getParallelism() { + return parallelism; + } + + /** + * Defines whether the solution set is kept in managed memory (Flink's internal way of keeping object + * in serialized form) or as a simple object map. + * By default, the solution set runs in managed memory. + * + * @param unmanaged True, to keep the solution set in unmanaged memory, false otherwise. + */ + public void setSolutionSetUnmanagedMemory(boolean unmanaged) { + this.unmanagedSolutionSet = unmanaged; + } + + /** + * Gets whether the solution set is kept in managed memory (Flink's internal way of keeping object + * in serialized form) or as a simple object map. + * By default, the solution set runs in managed memory. + * + * @return True, if the solution set is in unmanaged memory, false otherwise. + */ + public boolean isSolutionSetUnmanagedMemory() { + return this.unmanagedSolutionSet; + } + + // -------------------------------------------------------------------------------------------- + // Custom Operator behavior + // -------------------------------------------------------------------------------------------- + + /** + * Sets the input data set for this operator. In the case of this operator this input data set represents + * the set of vertices with their initial state. + * + * @param inputData The input data set, which in the case of this operator represents the set of + * vertices with their initial state. + * + * @see org.apache.flink.api.java.operators.CustomUnaryOperation#setInput(org.apache.flink.api.java.DataSet) + */ + @Override + public void setInput(DataSet> inputData) { + // sanity check that we really have two tuples + TypeInformation> inputType = inputData.getType(); + Validate.isTrue(inputType.isTupleType() && inputType.getArity() == 2, "The input data set (the initial vertices) must consist of 2-tuples."); + + // check that the key type here is the same as for the edges + TypeInformation keyType = ((TupleTypeInfo) inputType).getTypeAt(0); + TypeInformation edgeType = edgesWithoutValue != null ? edgesWithoutValue.getType() : edgesWithValue.getType(); + TypeInformation edgeKeyType = ((TupleTypeInfo) edgeType).getTypeAt(0); + + Validate.isTrue(keyType.equals(edgeKeyType), "The first tuple field (the vertex id) of the input data set (the initial vertices) " + + "must be the same data type as the first fields of the edge data set (the source vertex id). " + + "Here, the key type for the vertex ids is '%s' and the key type for the edges is '%s'.", keyType, edgeKeyType); + + this.initialVertices = inputData; + } + + /** + * Creates the operator that represents this vertex-centric graph computation. + * + * @return The operator that represents this vertex-centric graph computation. + */ + @Override + public DataSet> createResult() { + if (this.initialVertices == null) { + throw new IllegalStateException("The input data set has not been set."); + } + + // prepare some type information + TypeInformation> vertexTypes = initialVertices.getType(); + TypeInformation keyType = ((TupleTypeInfo) initialVertices.getType()).getTypeAt(0); + TypeInformation> messageTypeInfo = new TupleTypeInfo>(keyType, messageType); + + // set up the iteration operator + final String name = (this.name != null) ? this.name : + "Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")"; + final int[] zeroKeyPos = new int[] {0}; + + final DeltaIteration, Tuple2> iteration = + this.initialVertices.iterateDelta(this.initialVertices, this.maximumNumberOfIterations, zeroKeyPos); + iteration.name(name); + iteration.parallelism(parallelism); + iteration.setSolutionSetUnManaged(unmanagedSolutionSet); + + // register all aggregators + for (Map.Entry> entry : this.aggregators.entrySet()) { + iteration.registerAggregator(entry.getKey(), entry.getValue()); + } + + // build the messaging function (co group) + CoGroupOperator> messages; + if (edgesWithoutValue != null) { + MessagingUdfNoEdgeValues messenger = new MessagingUdfNoEdgeValues(messagingFunction, messageTypeInfo); + messages = this.edgesWithoutValue.coGroup(iteration.getWorkset()).where(0).equalTo(0).with(messenger); + } + else { + MessagingUdfWithEdgeValues messenger = new MessagingUdfWithEdgeValues(messagingFunction, messageTypeInfo); + messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(0).equalTo(0).with(messenger); + } + + // configure coGroup message function with name and broadcast variables + messages = messages.name("Messaging"); + for (Tuple2> e : this.bcVarsMessaging) { + messages = messages.withBroadcastSet(e.f1, e.f0); + } + + VertexUpdateUdf updateUdf = new VertexUpdateUdf(updateFunction, vertexTypes); + + // build the update function (co group) + CoGroupOperator> updates = + messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf); + + // configure coGroup update function with name and broadcast variables + updates = updates.name("Vertex State Updates"); + for (Tuple2> e : this.bcVarsUpdate) { + updates = updates.withBroadcastSet(e.f1, e.f0); + } + + // let the operator know that we preserve the key field + updates.withConstantSetFirst("0").withConstantSetSecond("0"); + + return iteration.closeWith(updates, updates); + + } + + // -------------------------------------------------------------------------------------------- + // Constructor builders to avoid signature conflicts with generic type erasure + // -------------------------------------------------------------------------------------------- + + /** + * Creates a new vertex-centric iteration operator for graphs where the edges are not associated with a value. + * + * @param edgesWithoutValue The data set containing edges. Edges are represented as 2-tuples: (source-id, target-id) + * @param vertexUpdateFunction The function that updates the state of the vertices from the incoming messages. + * @param messagingFunction The function that turns changed vertex states into messages along the edges. + * + * @param The type of the vertex key (the vertex identifier). + * @param The type of the vertex value (the state of the vertex). + * @param The type of the message sent between vertices along the edges. + * + * @return An in stance of the vertex-centric graph computation operator. + */ + public static final , VertexValue, Message> + VertexCentricIteration withPlainEdges( + DataSet> edgesWithoutValue, + VertexUpdateFunction vertexUpdateFunction, + MessagingFunction messagingFunction, + int maximumNumberOfIterations) + { + @SuppressWarnings("unchecked") + MessagingFunction tmf = + (MessagingFunction) messagingFunction; + + return new VertexCentricIteration(vertexUpdateFunction, tmf, edgesWithoutValue, maximumNumberOfIterations); + } + + /** + * Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as + * a weight or distance). + * + * @param edgesWithValue The data set containing edges. Edges are represented as 2-tuples: (source-id, target-id) + * @param uf The function that updates the state of the vertices from the incoming messages. + * @param mf The function that turns changed vertex states into messages along the edges. + * + * @param The type of the vertex key (the vertex identifier). + * @param The type of the vertex value (the state of the vertex). + * @param The type of the message sent between vertices along the edges. + * @param The type of the values that are associated with the edges. + * + * @return An in stance of the vertex-centric graph computation operator. + */ + public static final , VertexValue, Message, EdgeValue> + VertexCentricIteration withValuedEdges( + DataSet> edgesWithValue, + VertexUpdateFunction uf, + MessagingFunction mf, + int maximumNumberOfIterations) + { + return new VertexCentricIteration(uf, mf, edgesWithValue, maximumNumberOfIterations, true); + } + + // -------------------------------------------------------------------------------------------- + // Wrapping UDFs + // -------------------------------------------------------------------------------------------- + + private static final class VertexUpdateUdf, VertexValue, Message> + extends RichCoGroupFunction, Tuple2, Tuple2> + implements ResultTypeQueryable> + { + private static final long serialVersionUID = 1L; + + private final VertexUpdateFunction vertexUpdateFunction; + + private final MessageIterator messageIter = new MessageIterator(); + + private transient TypeInformation> resultType; + + + private VertexUpdateUdf(VertexUpdateFunction vertexUpdateFunction, + TypeInformation> resultType) + { + this.vertexUpdateFunction = vertexUpdateFunction; + this.resultType = resultType; + } + + @Override + public void coGroup(Iterable> messages, Iterable> vertex, + Collector> out) + throws Exception + { + final Iterator> vertexIter = vertex.iterator(); + + if (vertexIter.hasNext()) { + Tuple2 vertexState = vertexIter.next(); + + @SuppressWarnings("unchecked") + Iterator> downcastIter = (Iterator>) (Iterator) messages.iterator(); + messageIter.setSource(downcastIter); + + vertexUpdateFunction.setOutput(vertexState, out); + vertexUpdateFunction.updateVertex(vertexState.f0, vertexState.f1, messageIter); + } + else { + final Iterator> messageIter = messages.iterator(); + if (messageIter.hasNext()) { + String message = "Target vertex does not exist!."; + try { + Tuple2 next = messageIter.next(); + message = "Target vertex '" + next.f0 + "' does not exist!."; + } catch (Throwable t) {} + throw new Exception(message); + } else { + throw new Exception(); + } + } + } + + @Override + public void open(Configuration parameters) throws Exception { + if (getIterationRuntimeContext().getSuperstepNumber() == 1) { + this.vertexUpdateFunction.init(getIterationRuntimeContext()); + } + this.vertexUpdateFunction.preSuperstep(); + } + + @Override + public void close() throws Exception { + this.vertexUpdateFunction.postSuperstep(); + } + + @Override + public TypeInformation> getProducedType() { + return this.resultType; + } + } + + /* + * UDF that encapsulates the message sending function for graphs where the edges have no associated values. + */ + private static final class MessagingUdfNoEdgeValues, VertexValue, Message> + extends RichCoGroupFunction, Tuple2, Tuple2> + implements ResultTypeQueryable> + { + private static final long serialVersionUID = 1L; + + private final MessagingFunction messagingFunction; + + private transient TypeInformation> resultType; + + + private MessagingUdfNoEdgeValues(MessagingFunction messagingFunction, + TypeInformation> resultType) + { + this.messagingFunction = messagingFunction; + this.resultType = resultType; + } + + @Override + public void coGroup(Iterable> edges, + Iterable> state, Collector> out) + throws Exception + { + final Iterator> stateIter = state.iterator(); + + if (stateIter.hasNext()) { + Tuple2 newVertexState = stateIter.next(); + messagingFunction.set((Iterator) edges.iterator(), out); + messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + if (getIterationRuntimeContext().getSuperstepNumber() == 1) { + this.messagingFunction.init(getIterationRuntimeContext(), false); + } + + this.messagingFunction.preSuperstep(); + } + + @Override + public void close() throws Exception { + this.messagingFunction.postSuperstep(); + } + + @Override + public TypeInformation> getProducedType() { + return this.resultType; + } + } + + /* + * UDF that encapsulates the message sending function for graphs where the edges have an associated value. + */ + private static final class MessagingUdfWithEdgeValues, VertexValue, Message, EdgeValue> + extends RichCoGroupFunction, Tuple2, Tuple2> + implements ResultTypeQueryable> + { + private static final long serialVersionUID = 1L; + + private final MessagingFunction messagingFunction; + + private transient TypeInformation> resultType; + + + private MessagingUdfWithEdgeValues(MessagingFunction messagingFunction, + TypeInformation> resultType) + { + this.messagingFunction = messagingFunction; + this.resultType = resultType; + } + + @Override + public void coGroup(Iterable> edges, + Iterable> state, Collector> out) + throws Exception + { + final Iterator> stateIter = state.iterator(); + + if (stateIter.hasNext()) { + Tuple2 newVertexState = stateIter.next(); + messagingFunction.set((Iterator) edges.iterator(), out); + messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + if (getIterationRuntimeContext().getSuperstepNumber() == 1) { + this.messagingFunction.init(getIterationRuntimeContext(), true); + } + + this.messagingFunction.preSuperstep(); + } + + @Override + public void close() throws Exception { + this.messagingFunction.postSuperstep(); + } + + @Override + public TypeInformation> getProducedType() { + return this.resultType; + } + } +} diff --git a/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java b/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java new file mode 100644 index 0000000000000..c3fd2b12b3ac0 --- /dev/null +++ b/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java @@ -0,0 +1,145 @@ +/* + * 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 flink.graphs.spargel; + +import java.io.Serializable; +import java.util.Collection; + +import org.apache.flink.api.common.aggregators.Aggregator; +import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.types.Value; +import org.apache.flink.util.Collector; + +/** + * This class must be extended by functions that compute the state of the vertex depending on the old state and the + * incoming messages. The central method is {@link #updateVertex(Comparable, Object, MessageIterator)}, which is + * invoked once per vertex per superstep. + * + * The vertex key type. + * The vertex value type. + * The message type. + */ +public abstract class VertexUpdateFunction, VertexValue, Message> implements Serializable { + + private static final long serialVersionUID = 1L; + + // -------------------------------------------------------------------------------------------- + // Public API Methods + // -------------------------------------------------------------------------------------------- + + /** + * This method is invoked once per vertex per superstep. It receives the current state of the vertex, as well as + * the incoming messages. It may set a new vertex state via {@link #setNewVertexValue(Object)}. If the vertex + * state is changed, it will trigger the sending of messages via the {@link MessagingFunction}. + * + * @param vertexKey The key (identifier) of the vertex. + * @param vertexValue The value (state) of the vertex. + * @param inMessages The incoming messages to this vertex. + * + * @throws Exception The computation may throw exceptions, which causes the superstep to fail. + */ + public abstract void updateVertex(VertexKey vertexKey, VertexValue vertexValue, MessageIterator inMessages) throws Exception; + + /** + * This method is executed one per superstep before the vertex update function is invoked for each vertex. + * + * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail. + */ + public void preSuperstep() throws Exception {} + + /** + * This method is executed one per superstep after the vertex update function has been invoked for each vertex. + * + * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail. + */ + public void postSuperstep() throws Exception {} + + /** + * Sets the new value of this vertex. Setting a new value triggers the sending of outgoing messages from this vertex. + * + * @param newValue The new vertex value. + */ + public void setNewVertexValue(VertexValue newValue) { + outVal.f1 = newValue; + out.collect(outVal); + } + + /** + * Gets the number of the superstep, starting at 1. + * + * @return The number of the current superstep. + */ + public int getSuperstepNumber() { + return this.runtimeContext.getSuperstepNumber(); + } + + /** + * Gets the iteration aggregator registered under the given name. The iteration aggregator is combines + * all aggregates globally once per superstep and makes them available in the next superstep. + * + * @param name The name of the aggregator. + * @return The aggregator registered under this name, or null, if no aggregator was registered. + */ + public > T getIterationAggregator(String name) { + return this.runtimeContext.getIterationAggregator(name); + } + + /** + * Get the aggregated value that an aggregator computed in the previous iteration. + * + * @param name The name of the aggregator. + * @return The aggregated value of the previous iteration. + */ + public T getPreviousIterationAggregate(String name) { + return this.runtimeContext.getPreviousIterationAggregate(name); + } + + /** + * Gets the broadcast data set registered under the given name. Broadcast data sets + * are available on all parallel instances of a function. They can be registered via + * {@link VertexCentricIteration#addBroadcastSetForUpdateFunction(String, org.apache.flink.api.java.DataSet)}. + * + * @param name The name under which the broadcast set is registered. + * @return The broadcast data set. + */ + public Collection getBroadcastSet(String name) { + return this.runtimeContext.getBroadcastVariable(name); + } + + // -------------------------------------------------------------------------------------------- + // internal methods + // -------------------------------------------------------------------------------------------- + + private IterationRuntimeContext runtimeContext; + + private Collector> out; + + private Tuple2 outVal; + + + void init(IterationRuntimeContext context) { + this.runtimeContext = context; + } + + void setOutput(Tuple2 val, Collector> out) { + this.out = out; + this.outVal = val; + } +} diff --git a/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java index 9a45718d78329..30ac49b52a55a 100644 --- a/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java +++ b/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java @@ -25,13 +25,14 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.spargel.java.MessageIterator; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.VertexUpdateFunction; import org.apache.flink.test.testdata.ConnectedComponentsData; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.types.NullValue; +import flink.graphs.spargel.MessageIterator; +import flink.graphs.spargel.MessagingFunction; +import flink.graphs.spargel.VertexUpdateFunction; + @SuppressWarnings("serial") public class TestVertexCentricConnectedComponents extends JavaProgramTestBase { From 24d948081a4819db2506b49af5797de5ee66e603 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 8 Jan 2015 20:29:14 +0100 Subject: [PATCH 097/112] changed spargel classes to work with Vertex and Edge types --- src/main/java/flink/graphs/Graph.java | 10 +- .../java/flink/graphs/library/PageRank.java | 7 +- .../library/SingleSourceShortestPaths.java | 5 +- .../graphs/spargel/MessagingFunction.java | 89 ++----- .../flink/graphs/spargel/OutgoingEdge.java | 64 ----- .../spargel/VertexCentricIteration.java | 237 ++++-------------- .../graphs/spargel/VertexUpdateFunction.java | 12 +- 7 files changed, 83 insertions(+), 341 deletions(-) delete mode 100644 src/main/java/flink/graphs/spargel/OutgoingEdge.java diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 247f5fcb2d27f..425a377a63f05 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -53,7 +53,6 @@ import flink.graphs.spargel.VertexCentricIteration; import flink.graphs.spargel.VertexUpdateFunction; import flink.graphs.utils.GraphUtils; -import flink.graphs.utils.Tuple2ToVertexMap; import flink.graphs.validation.GraphValidator; /** @@ -1058,15 +1057,12 @@ public Graph union (Graph graph) { * @param maximumNumberOfIterations maximum number of iterations to perform * @return */ - @SuppressWarnings("unchecked") public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, MessagingFunction messagingFunction, int maximumNumberOfIterations) { - DataSet> tupleVertices = (DataSet>) (DataSet) vertices; - DataSet> tupleEdges = (DataSet>) (DataSet) edges; - DataSet> newVertices = tupleVertices.runOperation( - VertexCentricIteration.withValuedEdges(tupleEdges, + DataSet> newVertices = vertices.runOperation( + VertexCentricIteration.withEdges(edges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); - return new Graph(newVertices.map(new Tuple2ToVertexMap()), edges, context); + return new Graph(newVertices, edges, context); } /** diff --git a/src/main/java/flink/graphs/library/PageRank.java b/src/main/java/flink/graphs/library/PageRank.java index b7ca52bd08194..d29a9dcbbb514 100644 --- a/src/main/java/flink/graphs/library/PageRank.java +++ b/src/main/java/flink/graphs/library/PageRank.java @@ -1,11 +1,10 @@ package flink.graphs.library; - +import flink.graphs.Edge; import flink.graphs.Graph; import flink.graphs.GraphAlgorithm; import flink.graphs.spargel.MessageIterator; import flink.graphs.spargel.MessagingFunction; -import flink.graphs.spargel.OutgoingEdge; import flink.graphs.spargel.VertexUpdateFunction; import java.io.Serializable; @@ -69,8 +68,8 @@ public static final class RankMessenger & Serializable> @Override public void sendMessages(K vertexId, Double newRank) { - for (OutgoingEdge edge : getOutgoingEdges()) { - sendMessageTo(edge.target(), newRank * edge.edgeValue()); + for (Edge edge : getOutgoingEdges()) { + sendMessageTo(edge.getTarget(), newRank * edge.getValue()); } } } diff --git a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java index c29909c0e0128..0da8a90711d88 100644 --- a/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java +++ b/src/main/java/flink/graphs/library/SingleSourceShortestPaths.java @@ -3,7 +3,6 @@ import flink.graphs.*; import flink.graphs.spargel.MessageIterator; import flink.graphs.spargel.MessagingFunction; -import flink.graphs.spargel.OutgoingEdge; import flink.graphs.spargel.VertexUpdateFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -87,8 +86,8 @@ public static final class MinDistanceMessenger & Seriali @Override public void sendMessages(K vertexKey, Double newDistance) throws Exception { - for (OutgoingEdge edge : getOutgoingEdges()) { - sendMessageTo(edge.target(), newDistance + edge.edgeValue()); + for (Edge edge : getOutgoingEdges()) { + sendMessageTo(edge.getTarget(), newDistance + edge.getValue()); } } } diff --git a/src/main/java/flink/graphs/spargel/MessagingFunction.java b/src/main/java/flink/graphs/spargel/MessagingFunction.java index 52a881eef2f0b..ab451bbc7ae34 100644 --- a/src/main/java/flink/graphs/spargel/MessagingFunction.java +++ b/src/main/java/flink/graphs/spargel/MessagingFunction.java @@ -26,10 +26,11 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.types.Value; import org.apache.flink.util.Collector; +import flink.graphs.Edge; + /** * The base class for functions that produce messages between vertices as a part of a {@link VertexCentricIteration}. * @@ -38,7 +39,8 @@ * @param The type of the message sent between vertices along the edges. * @param The type of the values that are associated with the edges. */ -public abstract class MessagingFunction, VertexValue, Message, EdgeValue> implements Serializable { +public abstract class MessagingFunction & Serializable, + VertexValue extends Serializable, Message, EdgeValue extends Serializable> implements Serializable { private static final long serialVersionUID = 1L; @@ -79,19 +81,13 @@ public void postSuperstep() throws Exception {} * @return An iterator with all outgoing edges. */ @SuppressWarnings("unchecked") - public Iterable> getOutgoingEdges() { + public Iterable> getOutgoingEdges() { if (edgesUsed) { throw new IllegalStateException("Can use either 'getOutgoingEdges()' or 'sendMessageToAllTargets()' exactly once."); } edgesUsed = true; - - if (this.edgeWithValueIter != null) { - this.edgeWithValueIter.set((Iterator>) edges); - return this.edgeWithValueIter; - } else { - this.edgeNoValueIter.set((Iterator>) edges); - return this.edgeNoValueIter; - } + this.edgeIterator.set((Iterator>) edges); + return this.edgeIterator; } /** @@ -186,22 +182,15 @@ public Collection getBroadcastSet(String name) { private Collector> out; - private EdgesIteratorNoEdgeValue edgeNoValueIter; - - private EdgesIteratorWithEdgeValue edgeWithValueIter; + private EdgesIterator edgeIterator; private boolean edgesUsed; - void init(IterationRuntimeContext context, boolean hasEdgeValue) { + void init(IterationRuntimeContext context) { this.runtimeContext = context; this.outValue = new Tuple2(); - - if (hasEdgeValue) { - this.edgeWithValueIter = new EdgesIteratorWithEdgeValue(); - } else { - this.edgeNoValueIter = new EdgesIteratorNoEdgeValue(); - } + this.edgeIterator = new EdgesIterator(); } void set(Iterator edges, Collector> out) { @@ -210,52 +199,15 @@ void set(Iterator edges, Collector> out) { this.edgesUsed = false; } - - - private static final class EdgesIteratorNoEdgeValue, EdgeValue> - implements Iterator>, Iterable> - { - private Iterator> input; - - private OutgoingEdge edge = new OutgoingEdge(); - - - void set(Iterator> input) { - this.input = input; - } - - @Override - public boolean hasNext() { - return input.hasNext(); - } - - @Override - public OutgoingEdge next() { - Tuple2 next = input.next(); - edge.set(next.f1, null); - return edge; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator> iterator() { - return this; - } - } - - - private static final class EdgesIteratorWithEdgeValue, EdgeValue> - implements Iterator>, Iterable> + private static final class EdgesIterator & Serializable, + EdgeValue extends Serializable> + implements Iterator>, Iterable> { - private Iterator> input; + private Iterator> input; - private OutgoingEdge edge = new OutgoingEdge(); + private Edge edge = new Edge(); - void set(Iterator> input) { + void set(Iterator> input) { this.input = input; } @@ -265,9 +217,10 @@ public boolean hasNext() { } @Override - public OutgoingEdge next() { - Tuple3 next = input.next(); - edge.set(next.f1, next.f2); + public Edge next() { + Edge next = input.next(); + edge.setTarget(next.f1); + edge.setValue(next.f2); return edge; } @@ -276,7 +229,7 @@ public void remove() { throw new UnsupportedOperationException(); } @Override - public Iterator> iterator() { + public Iterator> iterator() { return this; } } diff --git a/src/main/java/flink/graphs/spargel/OutgoingEdge.java b/src/main/java/flink/graphs/spargel/OutgoingEdge.java deleted file mode 100644 index 7505409ae6c19..0000000000000 --- a/src/main/java/flink/graphs/spargel/OutgoingEdge.java +++ /dev/null @@ -1,64 +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 flink.graphs.spargel; - -/** - * Edge objects represent edges between vertices. Edges are defined by their source and target - * vertex id. Edges may have an associated value (for example a weight or a distance), if the - * graph algorithm was initialized with the - * {@link VertexCentricIteration#withValuedEdges(org.apache.flink.api.java.DataSet, VertexUpdateFunction, MessagingFunction, int)} - * method. - * - * @param The type of the vertex key. - * @param The type of the value associated with the edge. For scenarios where the edges do not hold - * value, this type may be arbitrary. - */ -public final class OutgoingEdge, EdgeValue> implements java.io.Serializable { - - private static final long serialVersionUID = 1L; - - private VertexKey target; - - private EdgeValue edgeValue; - - void set(VertexKey target, EdgeValue edgeValue) { - this.target = target; - this.edgeValue = edgeValue; - } - - /** - * Gets the target vertex id. - * - * @return The target vertex id. - */ - public VertexKey target() { - return target; - } - - /** - * Gets the value associated with the edge. The value may be null if the iteration was initialized with - * an edge data set without edge values. - * Typical examples of edge values are weights or distances of the path represented by the edge. - * - * @return The value associated with the edge. - */ - public EdgeValue edgeValue() { - return edgeValue; - } -} diff --git a/src/main/java/flink/graphs/spargel/VertexCentricIteration.java b/src/main/java/flink/graphs/spargel/VertexCentricIteration.java index b15c8c49b88db..5f89e907a759d 100644 --- a/src/main/java/flink/graphs/spargel/VertexCentricIteration.java +++ b/src/main/java/flink/graphs/spargel/VertexCentricIteration.java @@ -18,6 +18,7 @@ package flink.graphs.spargel; +import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -33,13 +34,15 @@ import org.apache.flink.api.java.operators.CoGroupOperator; import org.apache.flink.api.java.operators.CustomUnaryOperation; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; +import flink.graphs.Edge; +import flink.graphs.Vertex; + /** * This class represents iterative graph computations, programmed in a vertex-centric perspective. * It is a special case of Bulk Synchronous Parallel computation. The paradigm has also been @@ -70,16 +73,15 @@ * @param The type of the message sent between vertices along the edges. * @param The type of the values that are associated with the edges. */ -public class VertexCentricIteration, VertexValue, Message, EdgeValue> - implements CustomUnaryOperation, Tuple2> +public class VertexCentricIteration & Serializable, VertexValue extends Serializable, + Message, EdgeValue extends Serializable> + implements CustomUnaryOperation, Vertex> { private final VertexUpdateFunction updateFunction; private final MessagingFunction messagingFunction; - private final DataSet> edgesWithoutValue; - - private final DataSet> edgesWithValue; + private final DataSet> edgesWithValue; private final Map> aggregators; @@ -91,7 +93,7 @@ public class VertexCentricIteration, Ver private final TypeInformation messageType; - private DataSet> initialVertices; + private DataSet> initialVertices; private String name; @@ -101,64 +103,21 @@ public class VertexCentricIteration, Ver // ---------------------------------------------------------------------------------- - private VertexCentricIteration(VertexUpdateFunction uf, - MessagingFunction mf, - DataSet> edgesWithoutValue, - int maximumNumberOfIterations) - { - Validate.notNull(uf); - Validate.notNull(mf); - Validate.notNull(edgesWithoutValue); - Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); - - // check that the edges are actually a valid tuple set of vertex key types - TypeInformation> edgesType = edgesWithoutValue.getType(); - Validate.isTrue(edgesType.isTupleType() && edgesType.getArity() == 2, "The edges data set (for edges without edge values) must consist of 2-tuples."); - - TupleTypeInfo tupleInfo = (TupleTypeInfo) edgesType; - Validate.isTrue(tupleInfo.getTypeAt(0).equals(tupleInfo.getTypeAt(1)) - && Comparable.class.isAssignableFrom(tupleInfo.getTypeAt(0).getTypeClass()), - "Both tuple fields (source and target vertex id) must be of the data type that represents the vertex key and implement the java.lang.Comparable interface."); - - this.updateFunction = uf; - this.messagingFunction = mf; - this.edgesWithoutValue = edgesWithoutValue; - this.edgesWithValue = null; - this.maximumNumberOfIterations = maximumNumberOfIterations; - this.aggregators = new HashMap>(); - - this.messageType = getMessageType(mf); - } - private VertexCentricIteration(VertexUpdateFunction uf, MessagingFunction mf, - DataSet> edgesWithValue, - int maximumNumberOfIterations, - boolean edgeHasValueMarker) + DataSet> edgesWithValue, + int maximumNumberOfIterations, boolean edgeHasValueMarker) { Validate.notNull(uf); Validate.notNull(mf); Validate.notNull(edgesWithValue); Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); - - // check that the edges are actually a valid tuple set of vertex key types - TypeInformation> edgesType = edgesWithValue.getType(); - Validate.isTrue(edgesType.isTupleType() && edgesType.getArity() == 3, "The edges data set (for edges with edge values) must consist of 3-tuples."); - - TupleTypeInfo tupleInfo = (TupleTypeInfo) edgesType; - Validate.isTrue(tupleInfo.getTypeAt(0).equals(tupleInfo.getTypeAt(1)) - && Comparable.class.isAssignableFrom(tupleInfo.getTypeAt(0).getTypeClass()), - "The first two tuple fields (source and target vertex id) must be of the data type that represents the vertex key and implement the java.lang.Comparable interface."); - - Validate.isTrue(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one."); - + this.updateFunction = uf; this.messagingFunction = mf; - this.edgesWithoutValue = null; this.edgesWithValue = edgesWithValue; this.maximumNumberOfIterations = maximumNumberOfIterations; - this.aggregators = new HashMap>(); - + this.aggregators = new HashMap>(); this.messageType = getMessageType(mf); } @@ -271,20 +230,7 @@ public boolean isSolutionSetUnmanagedMemory() { * @see org.apache.flink.api.java.operators.CustomUnaryOperation#setInput(org.apache.flink.api.java.DataSet) */ @Override - public void setInput(DataSet> inputData) { - // sanity check that we really have two tuples - TypeInformation> inputType = inputData.getType(); - Validate.isTrue(inputType.isTupleType() && inputType.getArity() == 2, "The input data set (the initial vertices) must consist of 2-tuples."); - - // check that the key type here is the same as for the edges - TypeInformation keyType = ((TupleTypeInfo) inputType).getTypeAt(0); - TypeInformation edgeType = edgesWithoutValue != null ? edgesWithoutValue.getType() : edgesWithValue.getType(); - TypeInformation edgeKeyType = ((TupleTypeInfo) edgeType).getTypeAt(0); - - Validate.isTrue(keyType.equals(edgeKeyType), "The first tuple field (the vertex id) of the input data set (the initial vertices) " + - "must be the same data type as the first fields of the edge data set (the source vertex id). " + - "Here, the key type for the vertex ids is '%s' and the key type for the edges is '%s'.", keyType, edgeKeyType); - + public void setInput(DataSet> inputData) { this.initialVertices = inputData; } @@ -294,22 +240,22 @@ public void setInput(DataSet> inputData) { * @return The operator that represents this vertex-centric graph computation. */ @Override - public DataSet> createResult() { + public DataSet> createResult() { if (this.initialVertices == null) { throw new IllegalStateException("The input data set has not been set."); } // prepare some type information - TypeInformation> vertexTypes = initialVertices.getType(); + TypeInformation> vertexTypes = initialVertices.getType(); TypeInformation keyType = ((TupleTypeInfo) initialVertices.getType()).getTypeAt(0); - TypeInformation> messageTypeInfo = new TupleTypeInfo>(keyType, messageType); - + TypeInformation> messageTypeInfo = new TupleTypeInfo>(keyType, messageType); + // set up the iteration operator final String name = (this.name != null) ? this.name : "Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")"; final int[] zeroKeyPos = new int[] {0}; - final DeltaIteration, Tuple2> iteration = + final DeltaIteration, Vertex> iteration = this.initialVertices.iterateDelta(this.initialVertices, this.maximumNumberOfIterations, zeroKeyPos); iteration.name(name); iteration.parallelism(parallelism); @@ -322,14 +268,8 @@ public DataSet> createResult() { // build the messaging function (co group) CoGroupOperator> messages; - if (edgesWithoutValue != null) { - MessagingUdfNoEdgeValues messenger = new MessagingUdfNoEdgeValues(messagingFunction, messageTypeInfo); - messages = this.edgesWithoutValue.coGroup(iteration.getWorkset()).where(0).equalTo(0).with(messenger); - } - else { - MessagingUdfWithEdgeValues messenger = new MessagingUdfWithEdgeValues(messagingFunction, messageTypeInfo); - messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(0).equalTo(0).with(messenger); - } + MessagingUdfWithEdgeValues messenger = new MessagingUdfWithEdgeValues(messagingFunction, messageTypeInfo); + messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(0).equalTo(0).with(messenger); // configure coGroup message function with name and broadcast variables messages = messages.name("Messaging"); @@ -340,7 +280,7 @@ public DataSet> createResult() { VertexUpdateUdf updateUdf = new VertexUpdateUdf(updateFunction, vertexTypes); // build the update function (co group) - CoGroupOperator> updates = + CoGroupOperator> updates = messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf); // configure coGroup update function with name and broadcast variables @@ -355,43 +295,12 @@ public DataSet> createResult() { return iteration.closeWith(updates, updates); } - - // -------------------------------------------------------------------------------------------- - // Constructor builders to avoid signature conflicts with generic type erasure - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new vertex-centric iteration operator for graphs where the edges are not associated with a value. - * - * @param edgesWithoutValue The data set containing edges. Edges are represented as 2-tuples: (source-id, target-id) - * @param vertexUpdateFunction The function that updates the state of the vertices from the incoming messages. - * @param messagingFunction The function that turns changed vertex states into messages along the edges. - * - * @param The type of the vertex key (the vertex identifier). - * @param The type of the vertex value (the state of the vertex). - * @param The type of the message sent between vertices along the edges. - * - * @return An in stance of the vertex-centric graph computation operator. - */ - public static final , VertexValue, Message> - VertexCentricIteration withPlainEdges( - DataSet> edgesWithoutValue, - VertexUpdateFunction vertexUpdateFunction, - MessagingFunction messagingFunction, - int maximumNumberOfIterations) - { - @SuppressWarnings("unchecked") - MessagingFunction tmf = - (MessagingFunction) messagingFunction; - - return new VertexCentricIteration(vertexUpdateFunction, tmf, edgesWithoutValue, maximumNumberOfIterations); - } - + /** * Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as * a weight or distance). * - * @param edgesWithValue The data set containing edges. Edges are represented as 2-tuples: (source-id, target-id) + * @param edgesWithValue The data set containing edges. * @param uf The function that updates the state of the vertices from the incoming messages. * @param mf The function that turns changed vertex states into messages along the edges. * @@ -402,9 +311,10 @@ public DataSet> createResult() { * * @return An in stance of the vertex-centric graph computation operator. */ - public static final , VertexValue, Message, EdgeValue> - VertexCentricIteration withValuedEdges( - DataSet> edgesWithValue, + public static final & Serializable, VertexValue extends Serializable, + Message, EdgeValue extends Serializable> + VertexCentricIteration withEdges( + DataSet> edgesWithValue, VertexUpdateFunction uf, MessagingFunction mf, int maximumNumberOfIterations) @@ -416,9 +326,10 @@ VertexCentricIteration withValuedEdg // Wrapping UDFs // -------------------------------------------------------------------------------------------- - private static final class VertexUpdateUdf, VertexValue, Message> - extends RichCoGroupFunction, Tuple2, Tuple2> - implements ResultTypeQueryable> + private static final class VertexUpdateUdf & Serializable, + VertexValue extends Serializable, Message> + extends RichCoGroupFunction, Vertex, Vertex> + implements ResultTypeQueryable> { private static final long serialVersionUID = 1L; @@ -426,25 +337,25 @@ private static final class VertexUpdateUdf messageIter = new MessageIterator(); - private transient TypeInformation> resultType; + private transient TypeInformation> resultType; private VertexUpdateUdf(VertexUpdateFunction vertexUpdateFunction, - TypeInformation> resultType) + TypeInformation> resultType) { this.vertexUpdateFunction = vertexUpdateFunction; this.resultType = resultType; } @Override - public void coGroup(Iterable> messages, Iterable> vertex, - Collector> out) + public void coGroup(Iterable> messages, Iterable> vertex, + Collector> out) throws Exception { - final Iterator> vertexIter = vertex.iterator(); + final Iterator> vertexIter = vertex.iterator(); if (vertexIter.hasNext()) { - Tuple2 vertexState = vertexIter.next(); + Vertex vertexState = vertexIter.next(); @SuppressWarnings("unchecked") Iterator> downcastIter = (Iterator>) (Iterator) messages.iterator(); @@ -482,71 +393,17 @@ public void close() throws Exception { } @Override - public TypeInformation> getProducedType() { + public TypeInformation> getProducedType() { return this.resultType; } } - - /* - * UDF that encapsulates the message sending function for graphs where the edges have no associated values. - */ - private static final class MessagingUdfNoEdgeValues, VertexValue, Message> - extends RichCoGroupFunction, Tuple2, Tuple2> - implements ResultTypeQueryable> - { - private static final long serialVersionUID = 1L; - - private final MessagingFunction messagingFunction; - - private transient TypeInformation> resultType; - - - private MessagingUdfNoEdgeValues(MessagingFunction messagingFunction, - TypeInformation> resultType) - { - this.messagingFunction = messagingFunction; - this.resultType = resultType; - } - - @Override - public void coGroup(Iterable> edges, - Iterable> state, Collector> out) - throws Exception - { - final Iterator> stateIter = state.iterator(); - - if (stateIter.hasNext()) { - Tuple2 newVertexState = stateIter.next(); - messagingFunction.set((Iterator) edges.iterator(), out); - messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); - } - } - - @Override - public void open(Configuration parameters) throws Exception { - if (getIterationRuntimeContext().getSuperstepNumber() == 1) { - this.messagingFunction.init(getIterationRuntimeContext(), false); - } - - this.messagingFunction.preSuperstep(); - } - - @Override - public void close() throws Exception { - this.messagingFunction.postSuperstep(); - } - @Override - public TypeInformation> getProducedType() { - return this.resultType; - } - } - /* * UDF that encapsulates the message sending function for graphs where the edges have an associated value. */ - private static final class MessagingUdfWithEdgeValues, VertexValue, Message, EdgeValue> - extends RichCoGroupFunction, Tuple2, Tuple2> + private static final class MessagingUdfWithEdgeValues & Serializable, + VertexValue extends Serializable, Message, EdgeValue extends Serializable> + extends RichCoGroupFunction, Vertex, Tuple2> implements ResultTypeQueryable> { private static final long serialVersionUID = 1L; @@ -564,14 +421,14 @@ private MessagingUdfWithEdgeValues(MessagingFunction> edges, - Iterable> state, Collector> out) + public void coGroup(Iterable> edges, + Iterable> state, Collector> out) throws Exception { - final Iterator> stateIter = state.iterator(); + final Iterator> stateIter = state.iterator(); if (stateIter.hasNext()) { - Tuple2 newVertexState = stateIter.next(); + Vertex newVertexState = stateIter.next(); messagingFunction.set((Iterator) edges.iterator(), out); messagingFunction.sendMessages(newVertexState.f0, newVertexState.f1); } @@ -580,7 +437,7 @@ public void coGroup(Iterable> edges, @Override public void open(Configuration parameters) throws Exception { if (getIterationRuntimeContext().getSuperstepNumber() == 1) { - this.messagingFunction.init(getIterationRuntimeContext(), true); + this.messagingFunction.init(getIterationRuntimeContext()); } this.messagingFunction.preSuperstep(); diff --git a/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java b/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java index c3fd2b12b3ac0..e30451c4dcdb7 100644 --- a/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java +++ b/src/main/java/flink/graphs/spargel/VertexUpdateFunction.java @@ -23,10 +23,11 @@ import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.functions.IterationRuntimeContext; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Value; import org.apache.flink.util.Collector; +import flink.graphs.Vertex; + /** * This class must be extended by functions that compute the state of the vertex depending on the old state and the * incoming messages. The central method is {@link #updateVertex(Comparable, Object, MessageIterator)}, which is @@ -36,7 +37,8 @@ * The vertex value type. * The message type. */ -public abstract class VertexUpdateFunction, VertexValue, Message> implements Serializable { +public abstract class VertexUpdateFunction & Serializable, + VertexValue extends Serializable, Message> implements Serializable { private static final long serialVersionUID = 1L; @@ -129,16 +131,16 @@ public Collection getBroadcastSet(String name) { private IterationRuntimeContext runtimeContext; - private Collector> out; + private Collector> out; - private Tuple2 outVal; + private Vertex outVal; void init(IterationRuntimeContext context) { this.runtimeContext = context; } - void setOutput(Tuple2 val, Collector> out) { + void setOutput(Vertex val, Collector> out) { this.out = out; this.outVal = val; } From 23310d928e2eaf09a8233503bec539deca41273f Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 15 Jan 2015 21:38:43 +0100 Subject: [PATCH 098/112] removed 2-arg Edge constructor --- src/main/java/flink/graphs/Edge.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/main/java/flink/graphs/Edge.java b/src/main/java/flink/graphs/Edge.java index e8500b4a230c6..085d1afb71966 100644 --- a/src/main/java/flink/graphs/Edge.java +++ b/src/main/java/flink/graphs/Edge.java @@ -29,11 +29,6 @@ public class Edge & Serializable, V extends Serializable public Edge(){} - public Edge(K src, K trg) { - this.f0 = src; - this.f1 = trg; - } - public Edge(K src, K trg, V val) { this.f0 = src; this.f1 = trg; From 454a96c36706264c6de176d84db3a0393cd2ed96 Mon Sep 17 00:00:00 2001 From: Carsten Brandt Date: Sat, 17 Jan 2015 00:56:22 +0100 Subject: [PATCH 099/112] refactored methods for Graph creation - constructors are now private - create() methods have been renamed and refactored to fromDataSet() - fromCollection() methods have been refactored - overall consistency in creating new Graph objects, using the constructor now in all places fixes #53 --- src/main/java/flink/graphs/Graph.java | 347 ++++++++++++-------------- 1 file changed, 156 insertions(+), 191 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 425a377a63f05..307f177c29847 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -75,27 +75,27 @@ public class Graph & Serializable, VV extends Serializab private boolean isUndirected; /** - * Creates a graph from two datasets: vertices and edges + * Creates a graph from two DataSets: vertices and edges * * @param vertices a DataSet of vertices. * @param edges a DataSet of vertices. * @param context the flink execution environment. */ - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { + private Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { /** a graph is directed by default */ this(vertices, edges, context, false); } /** - * Creates a graph from two datasets: vertices and edges and allow setting the undirected property + * Creates a graph from two DataSets: vertices and edges and allow setting the undirected property * * @param vertices a DataSet of vertices. * @param edges a DataSet of vertices. * @param context the flink execution environment. * @param undirected whether this is an undirected graph */ - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, + private Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, boolean undirected) { this.vertices = vertices; this.edges = edges; @@ -103,6 +103,139 @@ public Graph(DataSet> vertices, DataSet> edges, Execut this.isUndirected = undirected; } + /** + * Creates a graph from a Collection of vertices and a Collection of edges. + * @param vertices a Collection of vertices. + * @param edges a Collection of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromCollection (Collection> vertices, + Collection> edges, + ExecutionEnvironment context) { + + return fromDataSet(context.fromCollection(vertices), context.fromCollection(edges), context); + } + + /** + * Creates a graph from a Collection of edges, vertices are induced from the edges. + * Vertices are created automatically and their values are set to NullValue. + * @param edges a Collection of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, EV extends Serializable> + Graph fromCollection (Collection> edges, ExecutionEnvironment context) { + + return fromDataSet(context.fromCollection(edges), context); + } + + /** + * Creates a graph from a Collection of edges, vertices are induced from the edges and + * vertex values are calculated by a mapper function. + * Vertices are created automatically and their values are set + * by applying the provided map function to the vertex ids. + * @param edges a Collection of vertices. + * @param mapper the mapper function. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromCollection (Collection> edges, + final MapFunction mapper, + ExecutionEnvironment context) { + + return fromDataSet(context.fromCollection(edges), mapper, context); + } + + /** + * Creates a graph from a DataSet of vertices and a DataSet of edges. + * @param vertices a DataSet of vertices. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromDataSet (DataSet> vertices, + DataSet> edges, + ExecutionEnvironment context) { + + return new Graph(vertices, edges, context); + } + + /** + * Creates a graph from a DataSet of edges, vertices are induced from the edges. + * Vertices are created automatically and their values are set to NullValue. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, EV extends Serializable> + Graph fromDataSet (DataSet> edges, + ExecutionEnvironment context) { + + DataSet> vertices = + edges.flatMap(new EmitSrcAndTarget()).distinct(); + + return new Graph(vertices, edges, context); + } + + private static final class EmitSrcAndTarget & Serializable, EV extends Serializable> + implements FlatMapFunction, Vertex> { + + public void flatMap(Edge edge, Collector> out) { + out.collect(new Vertex(edge.f0, NullValue.getInstance())); + out.collect(new Vertex(edge.f1, NullValue.getInstance())); + } + } + + /** + * Creates a graph from a DataSet of edges, vertices are induced from the edges and + * vertex values are calculated by a mapper function. + * Vertices are created automatically and their values are set + * by applying the provided map function to the vertex ids. + * @param edges a DataSet of vertices. + * @param mapper the mapper function. + * @param context the flink execution environment. + * @return the newly created graph. + */ + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromDataSet (DataSet> edges, + final MapFunction mapper, + ExecutionEnvironment context) { + + TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + + TypeInformation valueType = TypeExtractor + .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + TypeInformation> returnType = (TypeInformation>) + new TupleTypeInfo(Vertex.class, keyType, valueType); + + DataSet> vertices = + edges.flatMap(new EmitSrcAndTargetAsTuple1()) + .distinct() + .map(new MapFunction, Vertex>() { + public Vertex map(Tuple1 value) throws Exception { + return new Vertex(value.f0, mapper.map(value.f0)); + } + }) + .returns(returnType); + + return new Graph(vertices, edges, context); + } + + private static final class EmitSrcAndTargetAsTuple1 & Serializable, + EV extends Serializable> implements FlatMapFunction, Tuple1> { + + public void flatMap(Edge edge, Collector> out) { + out.collect(new Tuple1(edge.f0)); + out.collect(new Tuple1(edge.f1)); + } + } + /** * @return the flink execution environment. */ @@ -119,14 +252,14 @@ public DataSet validate(GraphValidator validator) { } /** - * @return the vertex dataset. + * @return the vertex DataSet. */ public DataSet> getVertices() { return vertices; } /** - * @return the edge dataset. + * @return the edge DataSet. */ public DataSet> getEdges() { return edges; @@ -156,7 +289,7 @@ public Vertex map(Vertex value) throws Exception { }) .returns(returnType); - return new Graph(mappedVertices, this.getEdges(), this.context); + return new Graph(mappedVertices, this.edges, this.context); } /** @@ -197,7 +330,7 @@ public Graph joinWithVertices(DataSet> inputDataSet, DataSet> resultedVertices = this.getVertices() .coGroup(inputDataSet).where(0).equalTo(0) .with(new ApplyCoGroupToVertexValues(mapper)); - return Graph.create(resultedVertices, this.getEdges(), this.getContext()); + return new Graph(resultedVertices, this.edges, this.context); } private static final class ApplyCoGroupToVertexValues & Serializable, @@ -243,7 +376,7 @@ public Graph joinWithEdges(DataSet> inputDataSet, DataSet> resultedEdges = this.getEdges() .coGroup(inputDataSet).where(0,1).equalTo(0,1) .with(new ApplyCoGroupToEdgeValues(mapper)); - return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + return new Graph(this.vertices, resultedEdges, this.context); } private static final class ApplyCoGroupToEdgeValues & Serializable, @@ -292,7 +425,7 @@ public Graph joinWithEdgesOnSource(DataSet> inputDat .coGroup(inputDataSet).where(0).equalTo(0) .with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(mapper)); - return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + return new Graph(this.vertices, resultedEdges, this.context); } private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget & Serializable, @@ -344,7 +477,7 @@ public Graph joinWithEdgesOnTarget(DataSet> inputDat .coGroup(inputDataSet).where(1).equalTo(0) .with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(mapper)); - return Graph.create(this.getVertices(), resultedEdges, this.getContext()); + return new Graph(this.vertices, resultedEdges, this.context); } /** @@ -672,124 +805,10 @@ public Graph reverse() throws UnsupportedOperationException { } else { DataSet> undirectedEdges = edges.map(new ReverseEdgesMap()); - return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); + return new Graph(this.vertices, undirectedEdges, this.context, true); } } - /** - * Creates a graph from a dataset of vertices and a dataset of edges - * @param vertices a DataSet of vertices. - * @param edges a DataSet of vertices. - * @param context the flink execution environment. - * @return the newly created graph - */ - public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph - create(DataSet> vertices, DataSet> edges, - ExecutionEnvironment context) { - return new Graph(vertices, edges, context); - } - - /** - * Creates a graph from a DataSet of edges. - * Vertices are created automatically and their values are set to NullValue. - * @param edges a DataSet of vertices. - * @param context the flink execution environment. - * @return the newly created graph - */ - public static & Serializable, EV extends Serializable> - Graph create(DataSet> edges, ExecutionEnvironment context) { - DataSet> vertices = - edges.flatMap(new EmitSrcAndTarget()).distinct(); - return new Graph(vertices, edges, context); - } - - /** - * Creates a graph from a DataSet of edges. - * Vertices are created automatically and their values are set - * by applying the provided map function to the vertex ids. - * @param edges the input edges - * @param mapper the map function to set the initial vertex value - * @return the newly created graph - */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph create(DataSet> edges, final MapFunction mapper, - ExecutionEnvironment context) { - TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); - - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); - - @SuppressWarnings({ "unchecked", "rawtypes" }) - TypeInformation> returnType = (TypeInformation>) - new TupleTypeInfo(Vertex.class, keyType, valueType); - - DataSet> vertices = - edges.flatMap(new EmitSrcAndTargetAsTuple1()) - .distinct().map(new MapFunction, Vertex>(){ - public Vertex map(Tuple1 value) throws Exception { - return new Vertex(value.f0, mapper.map(value.f0)); - } - }).returns(returnType); - return new Graph(vertices, edges, context); - } - - private static final class EmitSrcAndTarget & Serializable, EV extends Serializable> - implements FlatMapFunction, Vertex> { - public void flatMap(Edge edge, - Collector> out) { - - out.collect(new Vertex(edge.f0, NullValue.getInstance())); - out.collect(new Vertex(edge.f1, NullValue.getInstance())); - } - } - - private static final class EmitSrcAndTargetAsTuple1 & Serializable, - EV extends Serializable> implements FlatMapFunction, Tuple1> { - public void flatMap(Edge edge, Collector> out) { - - out.collect(new Tuple1(edge.f0)); - out.collect(new Tuple1(edge.f1)); - } - } - - /** - * Read and create the graph vertex Tuple2 DataSet from a csv file - * - * The CSV file should be of the following format: - * - * - * - * For example, with space delimiter: - * - * 1 57 - * 2 45 - * 3 77 - * 4 12 - * - * @param context the flink execution environment. - * @param filePath the path to the CSV file. - * @param delimiter the CSV delimiter. - * @param Tuple2IdClass The class to use for Vertex IDs - * @param Tuple2ValueClass The class to use for Vertex Values - * @return a set of vertices and their values. - */ - public static & Serializable, VV extends Serializable> - DataSet> - readTuple2CsvFile(ExecutionEnvironment context, String filePath, - char delimiter, Class Tuple2IdClass, Class Tuple2ValueClass) { - - CsvReader reader = new CsvReader(filePath, context); - DataSet> vertices = reader.fieldDelimiter(delimiter).types(Tuple2IdClass, Tuple2ValueClass) - .map(new MapFunction, Tuple2>() { - - public Tuple2 map(Tuple2 value) throws Exception { - return (Tuple2)value; - } - }); - return vertices; - } - /** * @return Singleton DataSet containing the vertex count */ @@ -914,14 +933,6 @@ public Boolean map(Integer n) { } } - public Graph fromCollection (Collection> vertices, Collection> edges) { - - DataSet> v = context.fromCollection(vertices); - DataSet> e = context.fromCollection(edges); - - return new Graph(v, e, context); - } - /** * Adds the input vertex and edges to the graph. * If the vertex already exists in the graph, it will not be added again, @@ -936,14 +947,14 @@ public Graph addVertex (final Vertex vertex, List> // Take care of empty edge set if (edges.isEmpty()) { - return Graph.create(getVertices().union(newVertex).distinct(), getEdges(), context); + return new Graph(this.vertices.union(newVertex).distinct(), this.edges, this.context); } // Add the vertex and its edges - DataSet> newVertices = getVertices().union(newVertex).distinct(); - DataSet> newEdges = getEdges().union(context.fromCollection(edges)); + DataSet> newVertices = this.vertices.union(newVertex).distinct(); + DataSet> newEdges = this.edges.union(context.fromCollection(edges)); - return Graph.create(newVertices, newEdges, context); + return new Graph(newVertices, newEdges, this.context); } /** @@ -956,8 +967,11 @@ public Graph addVertex (final Vertex vertex, List> * @return the new graph containing the existing vertices and edges plus the newly added edge */ public Graph addEdge (Vertex source, Vertex target, EV edgeValue) { - Graph partialGraph = this.fromCollection(Arrays.asList(source, target), - Arrays.asList(new Edge(source.f0, target.f0, edgeValue))); + Graph partialGraph = fromCollection( + Arrays.asList(source, target), + Arrays.asList(new Edge(source.f0, target.f0, edgeValue)), + this.context + ); return this.union(partialGraph); } @@ -1020,7 +1034,7 @@ public boolean filter(Edge edge) throws Exception { public Graph removeEdge (Edge edge) { DataSet> newEdges = getEdges().filter( new EdgeRemovalEdgeFilter(edge)); - return new Graph(this.getVertices(), newEdges, this.context); + return new Graph(this.vertices, newEdges, this.context); } private static final class EdgeRemovalEdgeFilter & Serializable, @@ -1062,58 +1076,9 @@ public Graph runVertexCentricIteration(VertexUpdateFunction> newVertices = vertices.runOperation( VertexCentricIteration.withEdges(edges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); - return new Graph(newVertices, edges, context); + return new Graph(newVertices, this.edges, this.context); } - /** - * Creates a graph from the given vertex and edge collections - * @param context the flink execution environment. - * @param v the collection of vertices - * @param e the collection of edges - * @return a new graph formed from the set of edges and vertices - */ - public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph - fromCollection(ExecutionEnvironment context, Collection> v, - Collection> e) throws Exception { - - DataSet> vertices = context.fromCollection(v); - DataSet> edges = context.fromCollection(e); - - return Graph.create(vertices, edges, context); - } - - /** - * Vertices may not have a value attached or may receive a value as a result of running the algorithm. - * @param context the flink execution environment. - * @param e the collection of edges - * @return a new graph formed from the edges, with no value for the vertices - */ - public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph - fromCollection(ExecutionEnvironment context, Collection> e) { - - DataSet> edges = context.fromCollection(e); - - return Graph.create(edges, context); - } - - /** - * Vertices may have an initial value defined by a function. - * @param context the flink execution environment. - * @param e the collection of edges - * @return a new graph formed from the edges, with a custom value for the vertices, - * determined by the mapping function - */ - public static & Serializable, VV extends Serializable, - EV extends Serializable> Graph - fromCollection(ExecutionEnvironment context, Collection> e, - final MapFunction mapper) { - - DataSet> edges = context.fromCollection(e); - return Graph.create(edges, mapper, context); - } - public Graph run (GraphAlgorithm algorithm) { return algorithm.run(this); } From 87fb98cb3c616c8e1bdb664d377749f64153077a Mon Sep 17 00:00:00 2001 From: Carsten Brandt Date: Sat, 17 Jan 2015 01:18:26 +0100 Subject: [PATCH 100/112] created fromTupleDataSet() methods issue #53 --- src/main/java/flink/graphs/Graph.java | 78 +++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 307f177c29847..65618039c3ed0 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -236,6 +236,68 @@ public void flatMap(Edge edge, Collector> out) { } } + /** + * Creates a graph from a DataSet of Tuple objects for vertices and edges. + * + * Vertices with value are created from Tuple2, + * Edges with value are created from Tuple3. + * + * @param vertices a DataSet of vertices. + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + @SuppressWarnings({ "unchecked" }) + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromTupleDataSet (DataSet> vertices, + DataSet> edges, + ExecutionEnvironment context) { + + DataSet> vertexDataSet = (DataSet>) (DataSet) vertices; + DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + return fromDataSet(vertexDataSet, edgeDataSet, context); + } + + /** + * Creates a graph from a DataSet of Tuple objects for edges, vertices are induced from the edges. + * + * Edges with value are created from Tuple3. + * Vertices are created automatically and their values are set to NullValue. + * + * @param edges a DataSet of vertices. + * @param context the flink execution environment. + * @return the newly created graph. + */ + @SuppressWarnings({ "unchecked" }) + public static & Serializable, EV extends Serializable> + Graph fromTupleDataSet (DataSet> edges, + ExecutionEnvironment context) { + + DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + return fromDataSet(edgeDataSet, context); + } + + /** + * Creates a graph from a DataSet of Tuple objects for edges, vertices are induced from the edges and + * vertex values are calculated by a mapper function. + * Edges with value are created from Tuple3. + * Vertices are created automatically and their values are set + * by applying the provided map function to the vertex ids. + * @param edges a DataSet of vertices. + * @param mapper the mapper function. + * @param context the flink execution environment. + * @return the newly created graph. + */ + @SuppressWarnings({ "unchecked" }) + public static & Serializable, VV extends Serializable, EV extends Serializable> + Graph fromTupleDataSet (DataSet> edges, + final MapFunction mapper, + ExecutionEnvironment context) { + + DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + return fromDataSet(edgeDataSet, mapper, context); + } + /** * @return the flink execution environment. */ @@ -265,6 +327,22 @@ public DataSet> getEdges() { return edges; } + /** + * @return the vertex DataSet as Tuple2. + */ + @SuppressWarnings({ "unchecked" }) + public DataSet> getVerticesAsTuple2() { + return (DataSet>) (DataSet) vertices; + } + + /** + * @return the edge DataSet as Tuple3. + */ + @SuppressWarnings({ "unchecked" }) + public DataSet> getEdgesAsTuple3() { + return (DataSet>) (DataSet) edges; + } + /** * Apply a function to the attribute of each vertex in the graph. * @param mapper the map function to apply. From 944c0c1e3db2f7fd9655a15465ce0622378cabb1 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 19 Jan 2015 16:18:08 +0100 Subject: [PATCH 101/112] remove isUndirected flag; methods will always consider the graph as directed --- src/main/java/flink/graphs/Graph.java | 52 +++++---------------------- 1 file changed, 9 insertions(+), 43 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index 425a377a63f05..3caf13d2ec14e 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -72,20 +72,6 @@ public class Graph & Serializable, VV extends Serializab private final ExecutionEnvironment context; private final DataSet> vertices; private final DataSet> edges; - private boolean isUndirected; - - /** - * Creates a graph from two datasets: vertices and edges - * - * @param vertices a DataSet of vertices. - * @param edges a DataSet of vertices. - * @param context the flink execution environment. - */ - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { - - /** a graph is directed by default */ - this(vertices, edges, context, false); - } /** * Creates a graph from two datasets: vertices and edges and allow setting the undirected property @@ -93,14 +79,11 @@ public Graph(DataSet> vertices, DataSet> edges, Execut * @param vertices a DataSet of vertices. * @param edges a DataSet of vertices. * @param context the flink execution environment. - * @param undirected whether this is an undirected graph */ - public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context, - boolean undirected) { + public Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { this.vertices = vertices; this.edges = edges; this.context = context; - this.isUndirected = undirected; } /** @@ -456,19 +439,14 @@ public DataSet> getDegrees() { } /** - * Convert the directed graph into an undirected graph - * by adding all inverse-direction edges. + * This operation adds all inverse-direction edges + * to the graph. * @return the undirected graph. */ public Graph getUndirected() throws UnsupportedOperationException { - if (this.isUndirected) { - throw new UnsupportedOperationException("The graph is already undirected."); - } - else { DataSet> undirectedEdges = edges.union(edges.map(new ReverseEdgesMap())); - return new Graph(vertices, undirectedEdges, this.context, true); - } + return new Graph(vertices, undirectedEdges, this.context); } /** @@ -667,13 +645,8 @@ public Edge map(Edge value) { * @throws UnsupportedOperationException */ public Graph reverse() throws UnsupportedOperationException { - if (this.isUndirected) { - throw new UnsupportedOperationException("The graph is already undirected."); - } - else { - DataSet> undirectedEdges = edges.map(new ReverseEdgesMap()); - return new Graph(vertices, (DataSet>) undirectedEdges, this.context, true); - } + DataSet> reversedEdges = edges.map(new ReverseEdgesMap()); + return new Graph(vertices, reversedEdges, this.context); } /** @@ -840,17 +813,10 @@ public Tuple2 map(Edge edge) throws Exception { * @return true if the graph is weakly connected. */ public DataSet isWeaklyConnected (int maxIterations) { - Graph graph; - - if (!(this.isUndirected)) { - // first, convert to an undirected graph - graph = this.getUndirected(); - } - else { - graph = this; - } + // first, convert to an undirected graph + Graph graph = this.getUndirected(); - DataSet vertexIds = graph.getVertexIds(); + DataSet vertexIds = graph.getVertexIds(); DataSet> verticesWithInitialIds = vertexIds .map(new DuplicateVertexIDMapper()); From 76d0ce3e1a174a151655ef0ed963631fa41d8290 Mon Sep 17 00:00:00 2001 From: vasia Date: Sun, 14 Dec 2014 22:44:24 +0100 Subject: [PATCH 102/112] filtering bad records and top track per user --- .../flink/graphs/example/MusicProfiles.java | 102 ++++++++++++++++++ .../flink/graphs/utils/Tuple3ToEdgeMap.java | 25 +++++ 2 files changed, 127 insertions(+) create mode 100644 src/main/java/flink/graphs/example/MusicProfiles.java create mode 100644 src/main/java/flink/graphs/utils/Tuple3ToEdgeMap.java diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java new file mode 100644 index 0000000000000..cd425a306cf0c --- /dev/null +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -0,0 +1,102 @@ +package flink.graphs.example; + +import java.util.Iterator; + +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.CoGroupFunction; +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.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.types.NullValue; +import org.apache.flink.util.Collector; + +import flink.graphs.Edge; +import flink.graphs.EdgeDirection; +import flink.graphs.EdgesFunction; +import flink.graphs.Graph; +import flink.graphs.utils.Tuple3ToEdgeMap; + +public class MusicProfiles implements ProgramDescription { + + @SuppressWarnings("serial") + public static void main (String [] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + /** read the user-song-play triplets + * The format is \t\t + */ + DataSet> triplets = env.readCsvFile(args[0]) + .lineDelimiter("\n").fieldDelimiter('\t').types(String.class, String.class, Integer.class); + + /** + * read the mismatches dataset and extract the songIDs + * The format is "ERROR: song_title" + */ + DataSet> mismatches = env.readTextFile(args[1]).map( + new MapFunction>() { + public Tuple1 map(String value) { + // TODO Auto-generated method stub + return null; + } + }); + + // filter out the mismatches from the triplets dataset + DataSet> validTriplets = triplets.coGroup(mismatches) + .where(1).equalTo(0).with(new CoGroupFunction, Tuple1, + Tuple3>() { + public void coGroup( + Iterable> triplets, + Iterable> invalidSongs, + Collector> out) { + if (!invalidSongs.iterator().hasNext()) { + // this is a valid triplet + out.collect(triplets.iterator().next()); + } + } + }); + + // Create a user -> song weighted bipartite graph + // where the edge weights correspond to play counts + DataSet> userSongEdges = validTriplets.map( + new Tuple3ToEdgeMap()); + + Graph userSongGraph = Graph.create(userSongEdges, env); + + // get the top track (most listened) for each user + DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges( + new EdgesFunction() { + public Tuple2 iterateEdges( + Iterable>> edges) { + int maxPlaycount = 0; + String userId = ""; + String topSong = ""; + + final Iterator>> edgesIterator = + edges.iterator(); + if (edgesIterator.hasNext()) { + Tuple2> first = edgesIterator.next(); + userId = first.f0; + topSong = first.f1.getTarget(); + } + while (edgesIterator.hasNext()) { + Tuple2> edge = edgesIterator.next(); + if (edge.f1.getValue() > maxPlaycount) { + maxPlaycount = edge.f1.getValue(); + topSong = edge.f1.getTarget(); + } + } + return new Tuple2 (userId, topSong); + } + }, EdgeDirection.OUT); + } + + @Override + public String getDescription() { + return null; + } + +} diff --git a/src/main/java/flink/graphs/utils/Tuple3ToEdgeMap.java b/src/main/java/flink/graphs/utils/Tuple3ToEdgeMap.java new file mode 100644 index 0000000000000..2a6cb230cfa0d --- /dev/null +++ b/src/main/java/flink/graphs/utils/Tuple3ToEdgeMap.java @@ -0,0 +1,25 @@ +package flink.graphs.utils; + +import java.io.Serializable; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple3; + +import flink.graphs.Edge; + +/** + * create an Edge DataSetfrom a Tuple3 dataset + * + * @param + * @param + */ +public class Tuple3ToEdgeMap & Serializable, + EV extends Serializable> implements MapFunction, Edge> { + + private static final long serialVersionUID = 1L; + + public Edge map(Tuple3 tuple) { + return new Edge(tuple.f0, tuple.f1, tuple.f2); + } + +} From d7184ce6465e385b707733b01206b8f389b03b07 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 5 Jan 2015 17:16:51 +0100 Subject: [PATCH 103/112] create similar users graph --- .../flink/graphs/example/MusicProfiles.java | 145 +++++++++++------- 1 file changed, 88 insertions(+), 57 deletions(-) diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java index cd425a306cf0c..d74e3392bbbb0 100644 --- a/src/main/java/flink/graphs/example/MusicProfiles.java +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -1,9 +1,11 @@ package flink.graphs.example; -import java.util.Iterator; +import java.util.ArrayList; +import java.util.List; import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -15,85 +17,114 @@ import flink.graphs.Edge; import flink.graphs.EdgeDirection; -import flink.graphs.EdgesFunction; +import flink.graphs.EdgesFunctionWithVertexValue; import flink.graphs.Graph; +import flink.graphs.Vertex; import flink.graphs.utils.Tuple3ToEdgeMap; public class MusicProfiles implements ProgramDescription { - @SuppressWarnings("serial") public static void main (String [] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - /** read the user-song-play triplets + /** + * Read the user-song-play triplets * The format is \t\t */ DataSet> triplets = env.readCsvFile(args[0]) .lineDelimiter("\n").fieldDelimiter('\t').types(String.class, String.class, Integer.class); /** - * read the mismatches dataset and extract the songIDs + * Read the mismatches dataset and extract the songIDs * The format is "ERROR: song_title" */ - DataSet> mismatches = env.readTextFile(args[1]).map( - new MapFunction>() { - public Tuple1 map(String value) { - // TODO Auto-generated method stub - return null; - } - }); - - // filter out the mismatches from the triplets dataset + DataSet> mismatches = env.readTextFile(args[1]).map(new ExtractMismatchSongIds()); + + /** + * Filter out the mismatches from the triplets dataset + */ DataSet> validTriplets = triplets.coGroup(mismatches) - .where(1).equalTo(0).with(new CoGroupFunction, Tuple1, - Tuple3>() { - public void coGroup( - Iterable> triplets, - Iterable> invalidSongs, - Collector> out) { - if (!invalidSongs.iterator().hasNext()) { - // this is a valid triplet - out.collect(triplets.iterator().next()); - } - } - }); - - // Create a user -> song weighted bipartite graph - // where the edge weights correspond to play counts - DataSet> userSongEdges = validTriplets.map( - new Tuple3ToEdgeMap()); + .where(1).equalTo(0).with(new FilterOutMismatches()); + /** + * Create a user -> song weighted bipartite graph + * where the edge weights correspond to play counts + */ + DataSet> userSongEdges = validTriplets.map(new Tuple3ToEdgeMap()); Graph userSongGraph = Graph.create(userSongEdges, env); - // get the top track (most listened) for each user - DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges( - new EdgesFunction() { - public Tuple2 iterateEdges( - Iterable>> edges) { - int maxPlaycount = 0; - String userId = ""; - String topSong = ""; - - final Iterator>> edgesIterator = - edges.iterator(); - if (edgesIterator.hasNext()) { - Tuple2> first = edgesIterator.next(); - userId = first.f0; - topSong = first.f1.getTarget(); - } - while (edgesIterator.hasNext()) { - Tuple2> edge = edgesIterator.next(); - if (edge.f1.getValue() > maxPlaycount) { - maxPlaycount = edge.f1.getValue(); - topSong = edge.f1.getTarget(); - } - } - return new Tuple2 (userId, topSong); - } - }, EdgeDirection.OUT); + /** + * Get the top track (most listened) for each user + */ + DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges(new GetTopSongPerUser(), + EdgeDirection.OUT); + + /** + * Create a user-user similarity graph, based on common songs, + * i.e. two users that listen to the same song are connected. + * For each song, we create an edge between each pair of its in-neighbors. + */ + DataSet> similarUsers = userSongGraph.getEdges().groupBy(1) + .reduceGroup(new CreateSimilarUserEdges()).distinct(); + Graph similarUsersGraph = Graph.create(similarUsers, env).getUndirected(); + } + @SuppressWarnings("serial") + public static final class ExtractMismatchSongIds implements MapFunction> { + public Tuple1 map(String value) { + String[] tokens = value.split("\\s+"); + String songId = tokens[1].substring(1); + return new Tuple1(songId); + } + } + + @SuppressWarnings("serial") + public static final class FilterOutMismatches implements CoGroupFunction, + Tuple1, Tuple3> { + public void coGroup( + Iterable> triplets, + Iterable> invalidSongs, + Collector> out) { + if (!invalidSongs.iterator().hasNext()) { + // this is a valid triplet + out.collect(triplets.iterator().next()); + } + } + } + + @SuppressWarnings("serial") + public static final class GetTopSongPerUser implements EdgesFunctionWithVertexValue + > { + public Tuple2 iterateEdges(Vertex vertex, + Iterable> edges) { + int maxPlaycount = 0; + String topSong = ""; + for (Edge edge: edges) { + if (edge.getValue() > maxPlaycount) { + maxPlaycount = edge.getValue(); + topSong = edge.getTarget(); + } + } + return new Tuple2 (vertex.getId(), topSong); + } + } + + @SuppressWarnings("serial") + public static final class CreateSimilarUserEdges implements GroupReduceFunction, + Edge> { + public void reduce(Iterable> edges, Collector> out) { + List listeners = new ArrayList(); + for (Edge edge : edges) { + listeners.add(edge.getSource()); + } + for (int i=0; i < listeners.size()-1; i++) { + out.collect(new Edge(listeners.get(i), listeners.get(i+1))); + } + } + } + @Override public String getDescription() { return null; From b2bde26d02bdf17ebff64ec2fe059ce6e79984b8 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 5 Jan 2015 17:50:29 +0100 Subject: [PATCH 104/112] added label propagation in library --- .../java/flink/graphs/GraphAlgorithm.java | 5 ++ .../graphs/library/LabelPropagation.java | 82 +++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 src/main/java/flink/graphs/library/LabelPropagation.java diff --git a/src/main/java/flink/graphs/GraphAlgorithm.java b/src/main/java/flink/graphs/GraphAlgorithm.java index c2cbd71793c48..2f5de9560d90b 100644 --- a/src/main/java/flink/graphs/GraphAlgorithm.java +++ b/src/main/java/flink/graphs/GraphAlgorithm.java @@ -3,6 +3,11 @@ import java.io.Serializable; +/** + * @param key type + * @param vertex value type + * @param edge value type + */ public interface GraphAlgorithm & Serializable, VV extends Serializable, EV extends Serializable> { diff --git a/src/main/java/flink/graphs/library/LabelPropagation.java b/src/main/java/flink/graphs/library/LabelPropagation.java new file mode 100644 index 0000000000000..430ccbba12ecf --- /dev/null +++ b/src/main/java/flink/graphs/library/LabelPropagation.java @@ -0,0 +1,82 @@ +package flink.graphs.library; + +import flink.graphs.*; + +import org.apache.flink.spargel.java.MessageIterator; +import org.apache.flink.spargel.java.MessagingFunction; +import org.apache.flink.spargel.java.VertexUpdateFunction; +import org.apache.flink.types.NullValue; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +@SuppressWarnings("serial") +public class LabelPropagation & Serializable> implements GraphAlgorithm { + + private final int maxIterations; + + public LabelPropagation(int maxIterations) { + this.maxIterations = maxIterations; + } + + @Override + public Graph run(Graph input) { + + // iteratively adopt the most frequent label among the neighbors + // of each vertex + return input.runVertexCentricIteration( + new UpdateVertexLabel(), + new SendNewLabelToNeighbors(), + maxIterations + ); + } + + /** + * Function that updates the value of a vertex by adopting the most frequent label + * among its in-neighbors + */ + public static final class UpdateVertexLabel & Serializable> + extends VertexUpdateFunction { + + public void updateVertex(K vertexKey, Long vertexValue, MessageIterator inMessages) { + Map labelsWithFrequencies = new HashMap(); + + long maxFrequency = 1; + long mostFrequentLabel = vertexValue; + + // store the labels with their frequencies + for (Long msg : inMessages) { + if (labelsWithFrequencies.containsKey(msg)) { + long currentFreq = labelsWithFrequencies.get(msg); + labelsWithFrequencies.put(msg, currentFreq + 1); + } + else { + labelsWithFrequencies.put(msg, 1L); + } + } + // select the most frequent label + for (Entry entry : labelsWithFrequencies.entrySet()) { + if (entry.getValue() > maxFrequency) { + maxFrequency = entry.getValue(); + mostFrequentLabel = entry.getKey(); + } + } + + // set the new vertex value + setNewVertexValue(mostFrequentLabel); + } + } + + /** + * Sends the vertex label to all out-neighbors + */ + public static final class SendNewLabelToNeighbors & Serializable> + extends MessagingFunction { + + public void sendMessages(K vertexKey, Long newLabel) { + sendMessageToAllNeighbors(newLabel); + } + } +} \ No newline at end of file From 958631e066baaa2371260d6656cbdb602455fa53 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 5 Jan 2015 19:50:02 +0100 Subject: [PATCH 105/112] label propagation example --- .../example/LabelPropagationExample.java | 69 +++++++++++++++++++ .../graphs/library/LabelPropagation.java | 8 +++ 2 files changed, 77 insertions(+) create mode 100644 src/main/java/flink/graphs/example/LabelPropagationExample.java diff --git a/src/main/java/flink/graphs/example/LabelPropagationExample.java b/src/main/java/flink/graphs/example/LabelPropagationExample.java new file mode 100644 index 0000000000000..ca857779c3968 --- /dev/null +++ b/src/main/java/flink/graphs/example/LabelPropagationExample.java @@ -0,0 +1,69 @@ +package flink.graphs.example; + + +import java.util.Random; + +import flink.graphs.*; +import flink.graphs.library.LabelPropagation; + +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.api.common.functions.*; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.types.NullValue; +import org.apache.flink.util.Collector; + +public class LabelPropagationExample implements ProgramDescription { + + public static void main (String [] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> vertices = getVertexDataSet(env); + DataSet> edges = getEdgeDataSet(env); + + Graph graph = new Graph(vertices, edges, env); + + DataSet> verticesWithCommunity = + graph.run(new LabelPropagation(maxIterations)).getVertices(); + + verticesWithCommunity.print(); + + env.execute(); + } + + @Override + public String getDescription() { + return "Label Propagation Example"; + } + + private static long numVertices = 20; + private static int maxIterations = 10; + private static int numberOfLabels = 3; + + @SuppressWarnings("serial") + private static DataSet> getVertexDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numVertices) + .map(new MapFunction>() { + public Vertex map(Long l) throws Exception { + Random randomGenerator = new Random(); + return new Vertex(l, (long) randomGenerator.nextInt((int) numberOfLabels)); + } + }); + } + + @SuppressWarnings("serial") + private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numVertices) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(Long key, Collector> out) { + int numOutEdges = (int) (Math.random() * (numVertices / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numVertices) + 1; + out.collect(new Edge(key, target, NullValue.getInstance())); + } + } + }); + } +} \ No newline at end of file diff --git a/src/main/java/flink/graphs/library/LabelPropagation.java b/src/main/java/flink/graphs/library/LabelPropagation.java index 430ccbba12ecf..65c911347615d 100644 --- a/src/main/java/flink/graphs/library/LabelPropagation.java +++ b/src/main/java/flink/graphs/library/LabelPropagation.java @@ -12,6 +12,14 @@ import java.util.Map; import java.util.Map.Entry; +/** + * An implementation of the label propagation algorithm. + * The iterative algorithm detects communities by propagating labels. + * In each iteration, a vertex adopts the label that is most frequent among its neighbors' labels. + * The algorithm converges when no vertex changes its value or the maximum number of iterations have been reached. + * Note that different initializations might lead to different results. + * + */ @SuppressWarnings("serial") public class LabelPropagation & Serializable> implements GraphAlgorithm { From 70ffc21d8913a62574014ec1b21e4143843326d2 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 5 Jan 2015 20:58:14 +0100 Subject: [PATCH 106/112] added label propagation step in MusicProfiles --- .../flink/graphs/example/MusicProfiles.java | 66 ++++++++++++++++--- .../example/utils/MusicProfilesData.java | 65 ++++++++++++++++++ 2 files changed, 122 insertions(+), 9 deletions(-) create mode 100644 src/main/java/flink/graphs/example/utils/MusicProfilesData.java diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java index d74e3392bbbb0..02f6554ba7350 100644 --- a/src/main/java/flink/graphs/example/MusicProfiles.java +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -2,9 +2,11 @@ import java.util.ArrayList; import java.util.List; +import java.util.Random; import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; @@ -20,26 +22,41 @@ import flink.graphs.EdgesFunctionWithVertexValue; import flink.graphs.Graph; import flink.graphs.Vertex; +import flink.graphs.example.utils.MusicProfilesData; +import flink.graphs.library.LabelPropagation; import flink.graphs.utils.Tuple3ToEdgeMap; public class MusicProfiles implements ProgramDescription { + /** + * This example demonstrates how to mix the "record" Flink API with the graph API. + * The input is a set triplets and a set of + * bad records,i.e. song ids that should not be trusted. + * Initially, we use the record API to filter out the bad records. + * Then, we use the graph API to create a user -> song weighted bipartite graph + * and compute the top song (most listened) per user. + * Then, we use the record API again, to create a user-user similarity graph, + * based on common songs, where two users that listen to the same song are connected. + * Finally, we use the graph API to run the label propagation community detection algorithm + * on the similarity graph. + */ public static void main (String [] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final long numberOfLabels = 3; + final int numIterations = 10; /** * Read the user-song-play triplets * The format is \t\t */ - DataSet> triplets = env.readCsvFile(args[0]) - .lineDelimiter("\n").fieldDelimiter('\t').types(String.class, String.class, Integer.class); - + DataSet> triplets = MusicProfilesData.getUserSongTriplets(env); + /** * Read the mismatches dataset and extract the songIDs * The format is "ERROR: song_title" */ - DataSet> mismatches = env.readTextFile(args[1]).map(new ExtractMismatchSongIds()); + DataSet> mismatches = MusicProfilesData.getMismatches(env).map(new ExtractMismatchSongIds()); /** * Filter out the mismatches from the triplets dataset @@ -58,7 +75,8 @@ public static void main (String [] args) throws Exception { * Get the top track (most listened) for each user */ DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges(new GetTopSongPerUser(), - EdgeDirection.OUT); + EdgeDirection.OUT).filter(new FilterSongNodes()); + usersWithTopTrack.print(); /** * Create a user-user similarity graph, based on common songs, @@ -69,6 +87,15 @@ public static void main (String [] args) throws Exception { .reduceGroup(new CreateSimilarUserEdges()).distinct(); Graph similarUsersGraph = Graph.create(similarUsers, env).getUndirected(); + /** + * Detect user communities using the label propagation library method + */ + DataSet> verticesWithCommunity = similarUsersGraph.mapVertices( + new InitVertexLabels(numberOfLabels)) + .run(new LabelPropagation(numIterations)).getVertices(); + verticesWithCommunity.print(); + + env.execute(); } @SuppressWarnings("serial") @@ -89,11 +116,20 @@ public void coGroup( Collector> out) { if (!invalidSongs.iterator().hasNext()) { // this is a valid triplet - out.collect(triplets.iterator().next()); + for (Tuple3 triplet : triplets) { + out.collect(triplet); + } } } } + @SuppressWarnings("serial") + public static final class FilterSongNodes implements FilterFunction> { + public boolean filter(Tuple2 value) throws Exception { + return !value.f1.equals(""); + } + } + @SuppressWarnings("serial") public static final class GetTopSongPerUser implements EdgesFunctionWithVertexValue > { @@ -120,14 +156,26 @@ public void reduce(Iterable> edges, Collector(listeners.get(i), listeners.get(i+1))); + out.collect(new Edge(listeners.get(i), listeners.get(i+1), + NullValue.getInstance())); } } } + + @SuppressWarnings("serial") + public static final class InitVertexLabels implements MapFunction, Long> { + private long numberOfLabels; + public InitVertexLabels(long labels) { + this.numberOfLabels = labels; + } + public Long map(Vertex value) { + Random randomGenerator = new Random(); + return (long) randomGenerator.nextInt((int) numberOfLabels); + } + } @Override public String getDescription() { - return null; + return "Music Profiles Example"; } - } diff --git a/src/main/java/flink/graphs/example/utils/MusicProfilesData.java b/src/main/java/flink/graphs/example/utils/MusicProfilesData.java new file mode 100644 index 0000000000000..cfe9c8847d892 --- /dev/null +++ b/src/main/java/flink/graphs/example/utils/MusicProfilesData.java @@ -0,0 +1,65 @@ +package flink.graphs.example.utils; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple3; + +public class MusicProfilesData { + + public static DataSet> getUserSongTriplets(ExecutionEnvironment env) { + List> triplets = new ArrayList>(); + + triplets.add(new Tuple3("user_1", "song_1", 100)); + triplets.add(new Tuple3("user_1", "song_2", 10)); + triplets.add(new Tuple3("user_1", "song_3", 20)); + triplets.add(new Tuple3("user_1", "song_4", 30)); + triplets.add(new Tuple3("user_1", "song_5", 1)); + + triplets.add(new Tuple3("user_2", "song_6", 40)); + triplets.add(new Tuple3("user_2", "song_7", 10)); + triplets.add(new Tuple3("user_2", "song_8", 3)); + + triplets.add(new Tuple3("user_3", "song_1", 100)); + triplets.add(new Tuple3("user_3", "song_2", 10)); + triplets.add(new Tuple3("user_3", "song_3", 20)); + triplets.add(new Tuple3("user_3", "song_8", 30)); + triplets.add(new Tuple3("user_3", "song_9", 1)); + triplets.add(new Tuple3("user_3", "song_10", 8)); + triplets.add(new Tuple3("user_3", "song_11", 90)); + triplets.add(new Tuple3("user_3", "song_12", 30)); + triplets.add(new Tuple3("user_3", "song_13", 34)); + triplets.add(new Tuple3("user_3", "song_14", 17)); + + triplets.add(new Tuple3("user_4", "song_1", 100)); + triplets.add(new Tuple3("user_4", "song_6", 10)); + triplets.add(new Tuple3("user_4", "song_8", 20)); + triplets.add(new Tuple3("user_4", "song_12", 30)); + triplets.add(new Tuple3("user_4", "song_13", 1)); + triplets.add(new Tuple3("user_4", "song_15", 1)); + + triplets.add(new Tuple3("user_5", "song_3", 300)); + triplets.add(new Tuple3("user_5", "song_4", 4)); + triplets.add(new Tuple3("user_5", "song_5", 5)); + triplets.add(new Tuple3("user_5", "song_8", 8)); + triplets.add(new Tuple3("user_5", "song_9", 9)); + triplets.add(new Tuple3("user_5", "song_10", 10)); + triplets.add(new Tuple3("user_5", "song_12", 12)); + triplets.add(new Tuple3("user_5", "song_13", 13)); + triplets.add(new Tuple3("user_5", "song_15", 15)); + + triplets.add(new Tuple3("user_6", "song_6", 30)); + + return env.fromCollection(triplets); + } + + public static DataSet getMismatches(ExecutionEnvironment env) { + List errors = new ArrayList(); + errors.add("ERROR: Sever"); + errors.add("ERROR: Black Trees"); + return env.fromCollection(errors); + } +} + From aae63fe9231a44c14d8f8cb2a03216de489d881f Mon Sep 17 00:00:00 2001 From: vasia Date: Fri, 16 Jan 2015 12:56:16 +0100 Subject: [PATCH 107/112] LP example update: nodes get their id as initial label; if 2 labels have the same freq, select the one with the highest value --- .../example/LabelPropagationExample.java | 22 ++++++++++--------- .../graphs/library/LabelPropagation.java | 18 ++++++++++----- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/src/main/java/flink/graphs/example/LabelPropagationExample.java b/src/main/java/flink/graphs/example/LabelPropagationExample.java index ca857779c3968..7acabb90c48b0 100644 --- a/src/main/java/flink/graphs/example/LabelPropagationExample.java +++ b/src/main/java/flink/graphs/example/LabelPropagationExample.java @@ -1,8 +1,5 @@ package flink.graphs.example; - -import java.util.Random; - import flink.graphs.*; import flink.graphs.library.LabelPropagation; @@ -13,6 +10,13 @@ import org.apache.flink.types.NullValue; import org.apache.flink.util.Collector; +/** + * This example uses the label propagation algorithm to detect communities by propagating labels. + * Initially, each vertex is assigned its id as its label. + * The vertices iteratively propagate their labels to their neighbors and adopt the most frequent label + * among their neighbors. + * The algorithm converges when no vertex changes value or the maximum number of iterations have been reached. + */ public class LabelPropagationExample implements ProgramDescription { public static void main (String [] args) throws Exception { @@ -37,23 +41,21 @@ public String getDescription() { return "Label Propagation Example"; } - private static long numVertices = 20; - private static int maxIterations = 10; - private static int numberOfLabels = 3; + private static long numVertices = 100; + private static int maxIterations = 20; - @SuppressWarnings("serial") + @SuppressWarnings("serial") private static DataSet> getVertexDataSet(ExecutionEnvironment env) { return env.generateSequence(1, numVertices) .map(new MapFunction>() { public Vertex map(Long l) throws Exception { - Random randomGenerator = new Random(); - return new Vertex(l, (long) randomGenerator.nextInt((int) numberOfLabels)); + return new Vertex(l, l); } }); } @SuppressWarnings("serial") - private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { + private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { return env.generateSequence(1, numVertices) .flatMap(new FlatMapFunction>() { @Override diff --git a/src/main/java/flink/graphs/library/LabelPropagation.java b/src/main/java/flink/graphs/library/LabelPropagation.java index 65c911347615d..fc77d0be30471 100644 --- a/src/main/java/flink/graphs/library/LabelPropagation.java +++ b/src/main/java/flink/graphs/library/LabelPropagation.java @@ -1,10 +1,10 @@ package flink.graphs.library; import flink.graphs.*; +import flink.graphs.spargel.MessageIterator; +import flink.graphs.spargel.MessagingFunction; +import flink.graphs.spargel.VertexUpdateFunction; -import org.apache.flink.spargel.java.MessageIterator; -import org.apache.flink.spargel.java.MessagingFunction; -import org.apache.flink.spargel.java.VertexUpdateFunction; import org.apache.flink.types.NullValue; import java.io.Serializable; @@ -16,6 +16,7 @@ * An implementation of the label propagation algorithm. * The iterative algorithm detects communities by propagating labels. * In each iteration, a vertex adopts the label that is most frequent among its neighbors' labels. + * Labels are represented by Longs and we assume a total ordering among them, in order to break ties. * The algorithm converges when no vertex changes its value or the maximum number of iterations have been reached. * Note that different initializations might lead to different results. * @@ -64,9 +65,16 @@ public void updateVertex(K vertexKey, Long vertexValue, MessageIterator in labelsWithFrequencies.put(msg, 1L); } } - // select the most frequent label + // select the most frequent label: if two or more labels have the same frequency, + // the node adopts the label with the highest value for (Entry entry : labelsWithFrequencies.entrySet()) { - if (entry.getValue() > maxFrequency) { + if (entry.getValue() == maxFrequency) { + // check the label value to break ties + if (entry.getKey() > mostFrequentLabel) { + mostFrequentLabel = entry.getKey(); + } + } + else if (entry.getValue() > maxFrequency) { maxFrequency = entry.getValue(); mostFrequentLabel = entry.getKey(); } From 7bfe627c4d0b78b5f76e4025b8041adb3714f3c1 Mon Sep 17 00:00:00 2001 From: vasia Date: Fri, 16 Jan 2015 15:53:26 +0100 Subject: [PATCH 108/112] updated music profiles example --- .../flink/graphs/example/MusicProfiles.java | 52 +++++++++++-------- 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java index 02f6554ba7350..91830a44d8b73 100644 --- a/src/main/java/flink/graphs/example/MusicProfiles.java +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -2,7 +2,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Random; import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.CoGroupFunction; @@ -26,6 +25,7 @@ import flink.graphs.library.LabelPropagation; import flink.graphs.utils.Tuple3ToEdgeMap; +@SuppressWarnings("serial") public class MusicProfiles implements ProgramDescription { /** @@ -43,7 +43,6 @@ public class MusicProfiles implements ProgramDescription { public static void main (String [] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - final long numberOfLabels = 3; final int numIterations = 10; /** @@ -69,6 +68,7 @@ public static void main (String [] args) throws Exception { * where the edge weights correspond to play counts */ DataSet> userSongEdges = validTriplets.map(new Tuple3ToEdgeMap()); + Graph userSongGraph = Graph.create(userSongEdges, env); /** @@ -76,6 +76,7 @@ public static void main (String [] args) throws Exception { */ DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT).filter(new FilterSongNodes()); + usersWithTopTrack.print(); /** @@ -85,20 +86,34 @@ public static void main (String [] args) throws Exception { */ DataSet> similarUsers = userSongGraph.getEdges().groupBy(1) .reduceGroup(new CreateSimilarUserEdges()).distinct(); - Graph similarUsersGraph = Graph.create(similarUsers, env).getUndirected(); + + Graph similarUsersGraph = Graph.create(similarUsers, + + new MapFunction() { + public Long map(String value) { return 1l; } + + }, env).getUndirected(); /** * Detect user communities using the label propagation library method */ - DataSet> verticesWithCommunity = similarUsersGraph.mapVertices( - new InitVertexLabels(numberOfLabels)) + + // Initialize each vertex with a unique numeric label + DataSet> idsWithInitialLabels = similarUsersGraph.getVertices() + .reduceGroup(new AssignInitialLabelReducer()); + + // update the vertex values and run the label propagation algorithm + DataSet> verticesWithCommunity = similarUsersGraph.joinWithVertices(idsWithInitialLabels, + new MapFunction, Long>() { + public Long map(Tuple2 value) { return value.f1; } + }) .run(new LabelPropagation(numIterations)).getVertices(); + verticesWithCommunity.print(); env.execute(); } - @SuppressWarnings("serial") public static final class ExtractMismatchSongIds implements MapFunction> { public Tuple1 map(String value) { String[] tokens = value.split("\\s+"); @@ -107,7 +122,6 @@ public Tuple1 map(String value) { } } - @SuppressWarnings("serial") public static final class FilterOutMismatches implements CoGroupFunction, Tuple1, Tuple3> { public void coGroup( @@ -123,14 +137,12 @@ public void coGroup( } } - @SuppressWarnings("serial") public static final class FilterSongNodes implements FilterFunction> { public boolean filter(Tuple2 value) throws Exception { return !value.f1.equals(""); } } - @SuppressWarnings("serial") public static final class GetTopSongPerUser implements EdgesFunctionWithVertexValue > { public Tuple2 iterateEdges(Vertex vertex, @@ -147,7 +159,6 @@ public Tuple2 iterateEdges(Vertex vertex, } } - @SuppressWarnings("serial") public static final class CreateSimilarUserEdges implements GroupReduceFunction, Edge> { public void reduce(Iterable> edges, Collector> out) { @@ -162,18 +173,17 @@ public void reduce(Iterable> edges, Collector, Long> { - private long numberOfLabels; - public InitVertexLabels(long labels) { - this.numberOfLabels = labels; - } - public Long map(Vertex value) { - Random randomGenerator = new Random(); - return (long) randomGenerator.nextInt((int) numberOfLabels); + public static final class AssignInitialLabelReducer implements GroupReduceFunction, + Tuple2> { + public void reduce(Iterable> vertices, Collector> out) { + long label = 0; + for (Vertex vertex : vertices) { + out.collect(new Tuple2(vertex.getId(), label)); + label++; + } } - } - + } + @Override public String getDescription() { return "Music Profiles Example"; From 8809196cd45486f917baa0072139947195e64eb6 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 19 Jan 2015 17:38:20 +0100 Subject: [PATCH 109/112] replaced create with fromDataSet --- .../java/flink/graphs/example/LabelPropagationExample.java | 2 +- src/main/java/flink/graphs/example/MusicProfiles.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/flink/graphs/example/LabelPropagationExample.java b/src/main/java/flink/graphs/example/LabelPropagationExample.java index 7acabb90c48b0..50c9ae562195d 100644 --- a/src/main/java/flink/graphs/example/LabelPropagationExample.java +++ b/src/main/java/flink/graphs/example/LabelPropagationExample.java @@ -26,7 +26,7 @@ public static void main (String [] args) throws Exception { DataSet> vertices = getVertexDataSet(env); DataSet> edges = getEdgeDataSet(env); - Graph graph = new Graph(vertices, edges, env); + Graph graph = Graph.fromDataSet(vertices, edges, env); DataSet> verticesWithCommunity = graph.run(new LabelPropagation(maxIterations)).getVertices(); diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java index 91830a44d8b73..44f226f3fa3f7 100644 --- a/src/main/java/flink/graphs/example/MusicProfiles.java +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -69,7 +69,7 @@ public static void main (String [] args) throws Exception { */ DataSet> userSongEdges = validTriplets.map(new Tuple3ToEdgeMap()); - Graph userSongGraph = Graph.create(userSongEdges, env); + Graph userSongGraph = Graph.fromDataSet(userSongEdges, env); /** * Get the top track (most listened) for each user @@ -87,7 +87,7 @@ public static void main (String [] args) throws Exception { DataSet> similarUsers = userSongGraph.getEdges().groupBy(1) .reduceGroup(new CreateSimilarUserEdges()).distinct(); - Graph similarUsersGraph = Graph.create(similarUsers, + Graph similarUsersGraph = Graph.fromDataSet(similarUsers, new MapFunction() { public Long map(String value) { return 1l; } From b541d096379b04e027ac4df3f52944428ec8f733 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 19 Jan 2015 18:20:19 +0100 Subject: [PATCH 110/112] use fromTupleDataset method instead of Tuple3ToEdgeMap --- src/main/java/flink/graphs/Graph.java | 13 ++++++------- .../java/flink/graphs/example/MusicProfiles.java | 5 +---- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/main/java/flink/graphs/Graph.java b/src/main/java/flink/graphs/Graph.java index b6e6d8759e655..a2364785b2b01 100644 --- a/src/main/java/flink/graphs/Graph.java +++ b/src/main/java/flink/graphs/Graph.java @@ -52,6 +52,8 @@ import flink.graphs.spargel.VertexCentricIteration; import flink.graphs.spargel.VertexUpdateFunction; import flink.graphs.utils.GraphUtils; +import flink.graphs.utils.Tuple2ToVertexMap; +import flink.graphs.utils.Tuple3ToEdgeMap; import flink.graphs.validation.GraphValidator; /** @@ -229,14 +231,13 @@ public void flatMap(Edge edge, Collector> out) { * @param context the flink execution environment. * @return the newly created graph. */ - @SuppressWarnings({ "unchecked" }) public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromTupleDataSet (DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { - DataSet> vertexDataSet = (DataSet>) (DataSet) vertices; - DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + DataSet> vertexDataSet = vertices.map(new Tuple2ToVertexMap()); + DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); return fromDataSet(vertexDataSet, edgeDataSet, context); } @@ -250,12 +251,11 @@ Graph fromTupleDataSet (DataSet> vertices, * @param context the flink execution environment. * @return the newly created graph. */ - @SuppressWarnings({ "unchecked" }) public static & Serializable, EV extends Serializable> Graph fromTupleDataSet (DataSet> edges, ExecutionEnvironment context) { - DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); return fromDataSet(edgeDataSet, context); } @@ -270,13 +270,12 @@ Graph fromTupleDataSet (DataSet> edges, * @param context the flink execution environment. * @return the newly created graph. */ - @SuppressWarnings({ "unchecked" }) public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromTupleDataSet (DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { - DataSet> edgeDataSet = (DataSet>) (DataSet) edges; + DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); return fromDataSet(edgeDataSet, mapper, context); } diff --git a/src/main/java/flink/graphs/example/MusicProfiles.java b/src/main/java/flink/graphs/example/MusicProfiles.java index 44f226f3fa3f7..668c7659e4848 100644 --- a/src/main/java/flink/graphs/example/MusicProfiles.java +++ b/src/main/java/flink/graphs/example/MusicProfiles.java @@ -23,7 +23,6 @@ import flink.graphs.Vertex; import flink.graphs.example.utils.MusicProfilesData; import flink.graphs.library.LabelPropagation; -import flink.graphs.utils.Tuple3ToEdgeMap; @SuppressWarnings("serial") public class MusicProfiles implements ProgramDescription { @@ -67,9 +66,7 @@ public static void main (String [] args) throws Exception { * Create a user -> song weighted bipartite graph * where the edge weights correspond to play counts */ - DataSet> userSongEdges = validTriplets.map(new Tuple3ToEdgeMap()); - - Graph userSongGraph = Graph.fromDataSet(userSongEdges, env); + Graph userSongGraph = Graph.fromTupleDataSet(validTriplets, env); /** * Get the top track (most listened) for each user From cdad1030fa9b3fa551bdc8a54fa8466b32fe9592 Mon Sep 17 00:00:00 2001 From: vasia Date: Mon, 19 Jan 2015 22:50:40 +0100 Subject: [PATCH 111/112] [FLINK-1201][gelly] added missing licence headers --- .../org/apache/flink/gelly/EdgeDirection.java | 18 +++ .../org/apache/flink/gelly/EdgesFunction.java | 18 +++ .../gelly/EdgesFunctionWithVertexValue.java | 18 +++ .../apache/flink/gelly/GraphAlgorithm.java | 20 ++- .../apache/flink/gelly/NeighborsFunction.java | 18 +++ .../NeighborsFunctionWithVertexValue.java | 18 +++ .../flink/gelly/example/GraphMetrics.java | 32 ++++- .../example/LabelPropagationExample.java | 18 +++ .../flink/gelly/example/MusicProfiles.java | 18 +++ .../flink/gelly/example/PageRankExample.java | 18 +++ .../SingleSourceShortestPathsExample.java | 18 +++ .../utils/EdgeWithLongIdNullValueParser.java | 18 +++ .../gelly/example/utils/ExampleUtils.java | 20 ++- .../example/utils/MusicProfilesData.java | 18 +++ .../flink/gelly/library/LabelPropagation.java | 18 +++ .../apache/flink/gelly/library/PageRank.java | 18 +++ .../library/SingleSourceShortestPaths.java | 18 +++ .../org/apache/flink/gelly/package-info.java | 4 - .../gelly/spargel/VertexCentricIteration.java | 4 +- .../flink/gelly/utils/EdgeToTuple3Map.java | 18 +++ .../apache/flink/gelly/utils/GraphUtils.java | 18 +++ .../flink/gelly/utils/Tuple2ToVertexMap.java | 18 +++ .../flink/gelly/utils/Tuple3ToEdgeMap.java | 18 +++ .../flink/gelly/utils/VertexToTuple2Map.java | 18 +++ .../gelly/validation/GraphValidator.java | 18 +++ .../validation/InvalidVertexIdsValidator.java | 18 +++ .../src/test/java/flink/graphs/TestPGA.java | 122 ------------------ .../apache/flink/gelly/test}/TestDegrees.java | 20 ++- .../flink/gelly/test}/TestFromCollection.java | 29 ++++- .../flink/gelly/test}/TestGraphCreation.java | 23 +++- .../test}/TestGraphCreationWithMapper.java | 23 +++- .../flink/gelly/test}/TestGraphMutations.java | 20 ++- .../gelly/test}/TestGraphOperations.java | 20 ++- .../flink/gelly/test}/TestGraphUtils.java | 26 +++- .../flink/gelly/test}/TestJoinWithEdges.java | 20 ++- .../gelly/test}/TestJoinWithVertices.java | 23 +++- .../flink/gelly/test}/TestMapEdges.java | 25 +++- .../flink/gelly/test}/TestMapVertices.java | 25 +++- .../gelly/test}/TestReduceOnEdgesMethods.java | 20 ++- .../test}/TestReduceOnNeighborMethods.java | 20 ++- .../TestVertexCentricConnectedComponents.java | 2 +- .../gelly/test}/TestWeaklyConnected.java | 20 ++- 42 files changed, 722 insertions(+), 174 deletions(-) delete mode 100644 flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/package-info.java delete mode 100644 flink-addons/flink-gelly/src/test/java/flink/graphs/TestPGA.java rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestDegrees.java (87%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestFromCollection.java (80%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestGraphCreation.java (85%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestGraphCreationWithMapper.java (83%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestGraphMutations.java (89%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestGraphOperations.java (90%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestGraphUtils.java (92%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestJoinWithEdges.java (96%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestJoinWithVertices.java (91%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestMapEdges.java (86%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestMapVertices.java (87%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestReduceOnEdgesMethods.java (91%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestReduceOnNeighborMethods.java (91%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestVertexCentricConnectedComponents.java (99%) rename flink-addons/flink-gelly/src/test/java/{flink/graphs => org/apache/flink/gelly/test}/TestWeaklyConnected.java (81%) diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgeDirection.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgeDirection.java index 8ef08c09ef6c9..8f9932a726726 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgeDirection.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgeDirection.java @@ -1,3 +1,21 @@ +/* + * 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.gelly; public enum EdgeDirection { diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunction.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunction.java index 23d0047d458a0..f3b5dee4ec263 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunction.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunction.java @@ -1,3 +1,21 @@ +/* + * 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.gelly; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunctionWithVertexValue.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunctionWithVertexValue.java index ed543accc2899..c10edc385a5a2 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunctionWithVertexValue.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/EdgesFunctionWithVertexValue.java @@ -1,3 +1,21 @@ +/* + * 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.gelly; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java index 87c83c2e90184..27df4e8dd8f7f 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java @@ -1,5 +1,22 @@ -package org.apache.flink.gelly; +/* + * 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.gelly; import java.io.Serializable; @@ -12,5 +29,4 @@ public interface GraphAlgorithm & Serializable, VV exten EV extends Serializable> { public Graph run (Graph input); - } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunction.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunction.java index 423d95a3018e2..70abb498bf9fa 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunction.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunction.java @@ -1,3 +1,21 @@ +/* + * 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.gelly; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunctionWithVertexValue.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunctionWithVertexValue.java index c8525e902cfa5..85722f31c4531 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunctionWithVertexValue.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/NeighborsFunctionWithVertexValue.java @@ -1,3 +1,21 @@ +/* + * 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.gelly; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/GraphMetrics.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/GraphMetrics.java index f97411efb1e7f..cb79c3164e230 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/GraphMetrics.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/GraphMetrics.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example; import java.util.Collection; @@ -70,13 +88,13 @@ public static void main(String[] args) throws Exception { DataSet minOutDegreeVertex = graph.outDegrees().minBy(1).map(new ProjectVertexId()); /** print the results **/ - ExampleUtils.printResult(numVertices, "Total number of vertices", env); - ExampleUtils.printResult(numEdges, "Total number of edges", env); - ExampleUtils.printResult(avgNodeDegree, "Average node degree", env); - ExampleUtils.printResult(maxInDegreeVertex, "Vertex with Max in-degree", env); - ExampleUtils.printResult(minInDegreeVertex, "Vertex with Min in-degree", env); - ExampleUtils.printResult(maxOutDegreeVertex, "Vertex with Max out-degree", env); - ExampleUtils.printResult(minOutDegreeVertex, "Vertex with Min out-degree", env); + ExampleUtils.printResult(numVertices, "Total number of vertices"); + ExampleUtils.printResult(numEdges, "Total number of edges"); + ExampleUtils.printResult(avgNodeDegree, "Average node degree"); + ExampleUtils.printResult(maxInDegreeVertex, "Vertex with Max in-degree"); + ExampleUtils.printResult(minInDegreeVertex, "Vertex with Min in-degree"); + ExampleUtils.printResult(maxOutDegreeVertex, "Vertex with Max out-degree"); + ExampleUtils.printResult(minOutDegreeVertex, "Vertex with Min out-degree"); env.execute(); } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java index dce11ad0fb825..67c60c94a162b 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example; import org.apache.flink.api.common.ProgramDescription; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java index 558cde9e8da56..4eb749f82f3a8 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example; import java.util.ArrayList; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java index e161c9d09398d..d6c002c6f78b2 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example; import org.apache.flink.api.common.ProgramDescription; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java index a8e739bc64723..760a4a0e3c638 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example; import org.apache.flink.api.common.ProgramDescription; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/EdgeWithLongIdNullValueParser.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/EdgeWithLongIdNullValueParser.java index 56f49bb21d413..26c0be79a7cd0 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/EdgeWithLongIdNullValueParser.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/EdgeWithLongIdNullValueParser.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example.utils; import org.apache.flink.api.common.functions.RichMapFunction; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java index 2c4923e903f22..b8c79cc647ea9 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example.utils; import java.io.PrintStream; @@ -18,7 +36,7 @@ public class ExampleUtils { @SuppressWarnings({ "serial", "unchecked", "rawtypes" }) - public static void printResult(DataSet set, String msg, ExecutionEnvironment env) { + public static void printResult(DataSet set, String msg) { set.output(new PrintingOutputFormatWithMessage(msg) { }); } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/MusicProfilesData.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/MusicProfilesData.java index 8c1c5f9950dd2..10fc017943cda 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/MusicProfilesData.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/MusicProfilesData.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.example.utils; import java.util.ArrayList; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java index 909fec73e60fb..cbcbb208c9e9a 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.library; import org.apache.flink.gelly.*; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java index 2bb27cd1af9c6..2408d63fbf34c 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.library; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java index eed428d079331..d4e41bb377d96 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.library; import org.apache.flink.api.common.functions.MapFunction; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/package-info.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/package-info.java deleted file mode 100644 index 431b649b2c9dc..0000000000000 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/package-info.java +++ /dev/null @@ -1,4 +0,0 @@ -/** - * - */ -package org.apache.flink.gelly; \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/spargel/VertexCentricIteration.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/spargel/VertexCentricIteration.java index f97e368448eda..ff186e14be852 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/spargel/VertexCentricIteration.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/spargel/VertexCentricIteration.java @@ -105,7 +105,7 @@ public class VertexCentricIteration & Se private VertexCentricIteration(VertexUpdateFunction uf, MessagingFunction mf, DataSet> edgesWithValue, - int maximumNumberOfIterations, boolean edgeHasValueMarker) + int maximumNumberOfIterations) { Validate.notNull(uf); Validate.notNull(mf); @@ -318,7 +318,7 @@ VertexCentricIteration withEdges( MessagingFunction mf, int maximumNumberOfIterations) { - return new VertexCentricIteration(uf, mf, edgesWithValue, maximumNumberOfIterations, true); + return new VertexCentricIteration(uf, mf, edgesWithValue, maximumNumberOfIterations); } // -------------------------------------------------------------------------------------------- diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/EdgeToTuple3Map.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/EdgeToTuple3Map.java index 422111eb69fa7..79479f2b7108e 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/EdgeToTuple3Map.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/EdgeToTuple3Map.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.utils; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java index ed21c00026378..563a2ce9f7287 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.utils; import java.util.ArrayList; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple2ToVertexMap.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple2ToVertexMap.java index 01b63d978a9fc..6b8376b12a16b 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple2ToVertexMap.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple2ToVertexMap.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.utils; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple3ToEdgeMap.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple3ToEdgeMap.java index 88c984ad528df..346a29bdefbc9 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple3ToEdgeMap.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/Tuple3ToEdgeMap.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.utils; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/VertexToTuple2Map.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/VertexToTuple2Map.java index 7e4f215d5ba7e..2ea4eba2370b5 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/VertexToTuple2Map.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/VertexToTuple2Map.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.utils; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java index f100e9fb9d816..08ff5ecf9867f 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.validation; import java.io.Serializable; diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java index 63aab0e8f53f5..bd7a6dc450684 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java @@ -1,3 +1,21 @@ +/* + * 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.gelly.validation; import org.apache.flink.api.common.functions.CoGroupFunction; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestPGA.java b/flink-addons/flink-gelly/src/test/java/flink/graphs/TestPGA.java deleted file mode 100644 index 4d68604cc0125..0000000000000 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestPGA.java +++ /dev/null @@ -1,122 +0,0 @@ -//package flink.graphs; -// -//import java.util.ArrayList; -//import java.util.List; -// -//import org.apache.flink.api.common.functions.CoGroupFunction; -//import org.apache.flink.api.common.functions.FlatJoinFunction; -//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.tuple.Tuple2; -//import org.apache.flink.api.java.tuple.Tuple3; -//import org.apache.flink.util.Collector; -//import org.junit.Before; -//import org.junit.Test; -// -//public class TestPGA { -// -// // Assume existing graph object -// // Tuple2 ids and values: 0,1,2,3 -// // Edges: 0->1, 1->3, 0->3, 1->2 -// -// static Graph graph; -// static ExecutionEnvironment env; -// -// @Before -// public void testSetUp() { -// env = ExecutionEnvironment.getExecutionEnvironment(); -// setUpGraph(); -// } -// -// -// public static void setUpGraph() { -// -// List> Tuple2List = new ArrayList>(); -// -// for (int i = 0; i < 4; i++) { -// Tuple2 v = new Tuple2(i, i); -// Tuple2List.add(v); -// } -// -// -// List> edgeList = new ArrayList<>(); -// -// edgeList.add(new Tuple3(0, 1, 0)); -// edgeList.add(new Tuple3(1, 3, 0)); -// edgeList.add(new Tuple3(0, 3, 0)); -// edgeList.add(new Tuple3(1, 2, 0)); -// -// DataSet> vertices = env.fromCollection(Tuple2List); -// DataSet> edges = env.fromCollection(edgeList); -// -// graph = new Graph(vertices, edges, env); -// } -// @SuppressWarnings("serial") -// @Test -// public void testPga() throws Exception { -// // Test pga by running connected components -// // Expected output is that all vertices end up with the same attribute, 0 -// -// // Send the vertex attribute to all neighbors -// CoGroupFunction, Tuple3, Tuple2> -// sendAttribute = -// new CoGroupFunction, Tuple3, Tuple2>() { -// @Override -// public void coGroup(Iterable> vertices, -// Iterable> edges, -// Collector> tuple2Collector) throws Exception { -// for (Tuple2 vertex : vertices) { -// for (Tuple3 edge: edges) { -// tuple2Collector.collect(new Tuple2(edge.f1, vertex.f1)); -// } -// } -// } -// }; -// -// // Gather all messages and keep the message with the smallest attribute -// GroupReduceFunction, Tuple2> -// gatherAttributes = -// new GroupReduceFunction, Tuple2>() { -// @Override -// public void reduce(Iterable> messages, -// Collector> msgCollector) throws Exception { -// -// Tuple2 minTuple = new Tuple2(Integer.MAX_VALUE, Integer.MAX_VALUE); -// for (Tuple2 message : messages) { -// if (message.f1 < minTuple.f1) { -// minTuple = message.copy(); -// } -// } -// msgCollector.collect(minTuple); -// } -// }; -// -// // Check if the produced message is smaller than the current vertex attribute, if yes change attribute -// FlatJoinFunction, Tuple2, Tuple2> -// apply = -// new FlatJoinFunction, Tuple2, Tuple2>() { -// @Override -// public void join(Tuple2 msg, -// Tuple2 vertex, -// Collector> vertexCollector) throws Exception { -// if (msg.f1 < vertex.f1) { -// vertexCollector.collect(msg.copy()); -// } -// } -// }; -// -// -// // Run the pga iterations -// Graph connected = graph.pga(sendAttribute, gatherAttributes, apply, 100); -// -// DataSet> conVerts = connected.getVertices(); -// -// // All vertices should end up with attribute 0 -// conVerts.print(); -// //TODO(thvasilo): Automate correctness testing -// -// env.execute(); -// -// } -//} diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestDegrees.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestDegrees.java similarity index 87% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestDegrees.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestDegrees.java index cc5ead0cedb19..337873eb22391 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestDegrees.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestDegrees.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestFromCollection.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestFromCollection.java similarity index 80% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestFromCollection.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestFromCollection.java index adb1f44de40fb..af5d72f8d5737 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestFromCollection.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestFromCollection.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; @@ -66,8 +84,8 @@ public static String runProgram(int progId, String resultPath) throws Exception * Test fromCollection(vertices, edges): */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(env), - TestGraphUtils.getLongLongEdges(env), env); + Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), + TestGraphUtils.getLongLongEdges(), env); graph.getEdges().writeAsCsv(resultPath); env.execute(); @@ -84,7 +102,7 @@ public static String runProgram(int progId, String resultPath) throws Exception * Test fromCollection(edges) with no initial value for the vertices */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(env), + Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(), env); graph.getVertices().writeAsCsv(resultPath); @@ -101,7 +119,7 @@ public static String runProgram(int progId, String resultPath) throws Exception * function that takes the id and doubles it */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(env), + Graph graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(), new MapFunction() { public Long map(Long vertexId) { return vertexId * 2; @@ -121,5 +139,4 @@ public Long map(Long vertexId) { } } } - } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreation.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreation.java similarity index 85% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreation.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreation.java index dcb3a3e0c4f40..c389961146641 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreation.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreation.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; @@ -12,6 +30,7 @@ import org.apache.flink.gelly.Edge; import org.apache.flink.gelly.Graph; import org.apache.flink.gelly.Vertex; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomParameterizedType; import org.apache.flink.gelly.validation.InvalidVertexIdsValidator; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.types.NullValue; @@ -19,8 +38,6 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; - @RunWith(Parameterized.class) public class TestGraphCreation extends JavaProgramTestBase { diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreationWithMapper.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreationWithMapper.java similarity index 83% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreationWithMapper.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreationWithMapper.java index 2c487a6164d6f..84d8df16eb848 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphCreationWithMapper.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphCreationWithMapper.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; @@ -10,13 +28,12 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomType; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import flink.graphs.TestGraphUtils.DummyCustomType; - @RunWith(Parameterized.class) public class TestGraphCreationWithMapper extends JavaProgramTestBase { diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphMutations.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphMutations.java similarity index 89% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphMutations.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphMutations.java index 2e0f15e644dcc..d66c18dbb1994 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphMutations.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphMutations.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphOperations.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphOperations.java similarity index 90% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphOperations.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphOperations.java index bb728851c960d..ab96eef147252 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphOperations.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphOperations.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphUtils.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphUtils.java similarity index 92% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphUtils.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphUtils.java index 92854c0eda3d4..85b96fe2e6bf0 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestGraphUtils.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestGraphUtils.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.Serializable; import java.util.ArrayList; @@ -204,8 +222,7 @@ public static final DataSet> getLongLongEdgeDataWithZeroDegree( /** * Function that produces an ArrayList of vertices */ - public static final List> getLongLongVertices( - ExecutionEnvironment env) { + public static final List> getLongLongVertices() { List> vertices = new ArrayList>(); vertices.add(new Vertex(1L, 1L)); vertices.add(new Vertex(2L, 2L)); @@ -230,8 +247,7 @@ public static final DataSet> getDisconnectedLongLongEdgeData( /** * Function that produces an ArrayList of edges */ - public static final List> getLongLongEdges( - ExecutionEnvironment env) { + public static final List> getLongLongEdges() { List> edges = new ArrayList>(); edges.add(new Edge(1L, 2L, 12L)); edges.add(new Edge(1L, 3L, 13L)); diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithEdges.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithEdges.java similarity index 96% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithEdges.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithEdges.java index 45db1e7be6f01..054d41d5aa962 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithEdges.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithEdges.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithVertices.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithVertices.java similarity index 91% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithVertices.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithVertices.java index 2f3e9dcd3ebe1..1ee7ee4121f18 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestJoinWithVertices.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestJoinWithVertices.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.ExecutionEnvironment; @@ -6,12 +24,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.gelly.Graph; import org.apache.flink.gelly.Vertex; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomParameterizedType; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; - import java.io.IOException; import java.util.Collection; import java.util.LinkedList; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapEdges.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapEdges.java similarity index 86% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapEdges.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapEdges.java index b48064c92a551..6d20c7c07769a 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapEdges.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapEdges.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; @@ -12,14 +30,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.gelly.Edge; import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomParameterizedType; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomType; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; -import flink.graphs.TestGraphUtils.DummyCustomType; - @RunWith(Parameterized.class) public class TestMapEdges extends JavaProgramTestBase { diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapVertices.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapVertices.java similarity index 87% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapVertices.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapVertices.java index e41b43dc05006..933d1a413677c 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestMapVertices.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestMapVertices.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; @@ -12,14 +30,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.gelly.Graph; import org.apache.flink.gelly.Vertex; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomParameterizedType; +import org.apache.flink.gelly.test.TestGraphUtils.DummyCustomType; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import flink.graphs.TestGraphUtils.DummyCustomParameterizedType; -import flink.graphs.TestGraphUtils.DummyCustomType; - @RunWith(Parameterized.class) public class TestMapVertices extends JavaProgramTestBase { diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnEdgesMethods.java similarity index 91% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnEdgesMethods.java index 42e100a71cb6b..f9518934b62c5 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnEdgesMethods.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnEdgesMethods.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnNeighborMethods.java similarity index 91% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnNeighborMethods.java index 69df4c548dc5e..fd06c36cd3793 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestReduceOnNeighborMethods.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestReduceOnNeighborMethods.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestVertexCentricConnectedComponents.java similarity index 99% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestVertexCentricConnectedComponents.java index 9429fb37ebac3..fd9b70e28f3ee 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestVertexCentricConnectedComponents.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestVertexCentricConnectedComponents.java @@ -17,7 +17,7 @@ */ -package flink.graphs; +package org.apache.flink.gelly.test; import java.io.BufferedReader; diff --git a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestWeaklyConnected.java b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestWeaklyConnected.java similarity index 81% rename from flink-addons/flink-gelly/src/test/java/flink/graphs/TestWeaklyConnected.java rename to flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestWeaklyConnected.java index 4f2791d0fdeb1..abe164a0628f5 100644 --- a/flink-addons/flink-gelly/src/test/java/flink/graphs/TestWeaklyConnected.java +++ b/flink-addons/flink-gelly/src/test/java/org/apache/flink/gelly/test/TestWeaklyConnected.java @@ -1,4 +1,22 @@ -package flink.graphs; +/* + * 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.gelly.test; import java.io.FileNotFoundException; import java.io.IOException; From 2a14f7794bb6fa8254bbab7883a0e07d334a01e8 Mon Sep 17 00:00:00 2001 From: vasia Date: Tue, 20 Jan 2015 08:46:51 +0100 Subject: [PATCH 112/112] [FLINK-1201][gelly] fixed checkstyle errors --- .../java/org/apache/flink/gelly/Graph.java | 1219 +++++++++-------- .../apache/flink/gelly/GraphAlgorithm.java | 5 +- .../example/LabelPropagationExample.java | 92 +- .../flink/gelly/example/MusicProfiles.java | 233 ++-- .../flink/gelly/example/PageRankExample.java | 124 +- .../SingleSourceShortestPathsExample.java | 33 +- .../gelly/example/utils/ExampleUtils.java | 83 +- .../flink/gelly/library/LabelPropagation.java | 90 +- .../apache/flink/gelly/library/PageRank.java | 109 +- .../library/SingleSourceShortestPaths.java | 141 +- .../apache/flink/gelly/utils/GraphUtils.java | 37 +- .../gelly/validation/GraphValidator.java | 9 +- .../validation/InvalidVertexIdsValidator.java | 97 +- 13 files changed, 1156 insertions(+), 1116 deletions(-) diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/Graph.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/Graph.java index 601c1de3f3710..b527cdeae2da2 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/Graph.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/Graph.java @@ -48,118 +48,125 @@ import org.apache.flink.gelly.spargel.MessagingFunction; import org.apache.flink.gelly.spargel.VertexCentricIteration; import org.apache.flink.gelly.spargel.VertexUpdateFunction; +import org.apache.flink.gelly.utils.EdgeToTuple3Map; import org.apache.flink.gelly.utils.GraphUtils; import org.apache.flink.gelly.utils.Tuple2ToVertexMap; import org.apache.flink.gelly.utils.Tuple3ToEdgeMap; +import org.apache.flink.gelly.utils.VertexToTuple2Map; import org.apache.flink.gelly.validation.GraphValidator; import org.apache.flink.util.Collector; import org.apache.flink.types.NullValue; /** - * Represents a Graph consisting of {@link Edge edges} and {@link Vertex vertices}. - * - * + * Represents a Graph consisting of {@link Edge edges} and {@link Vertex + * vertices}. + * + * * @see org.apache.flink.gelly.Edge * @see org.apache.flink.gelly.Vertex - * + * * @param the key type for edge and vertex identifiers * @param the value type for vertexes * @param the value type for edges */ @SuppressWarnings("serial") -public class Graph & Serializable, VV extends Serializable, EV extends Serializable> { +public class Graph & Serializable, VV extends Serializable, EV extends Serializable> { - private final ExecutionEnvironment context; + private final ExecutionEnvironment context; private final DataSet> vertices; private final DataSet> edges; /** - * Creates a graph from two DataSets: vertices and edges and allow setting the undirected property - * + * Creates a graph from two DataSets: vertices and edges and allow setting + * the undirected property + * * @param vertices a DataSet of vertices. - * @param edges a DataSet of vertices. + * @param edges a DataSet of edges. * @param context the flink execution environment. */ private Graph(DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { this.vertices = vertices; this.edges = edges; - this.context = context; + this.context = context; } /** * Creates a graph from a Collection of vertices and a Collection of edges. + * * @param vertices a Collection of vertices. - * @param edges a Collection of vertices. + * @param edges a Collection of edges. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromCollection (Collection> vertices, - Collection> edges, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromCollection( + Collection> vertices, Collection> edges, + ExecutionEnvironment context) { - return fromDataSet(context.fromCollection(vertices), context.fromCollection(edges), context); + return fromDataSet(context.fromCollection(vertices), + context.fromCollection(edges), context); } /** - * Creates a graph from a Collection of edges, vertices are induced from the edges. - * Vertices are created automatically and their values are set to NullValue. + * Creates a graph from a Collection of edges, vertices are induced from the + * edges. Vertices are created automatically and their values are set to + * NullValue. + * * @param edges a Collection of vertices. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, EV extends Serializable> - Graph fromCollection (Collection> edges, ExecutionEnvironment context) { + public static & Serializable, EV extends Serializable> Graph fromCollection( + Collection> edges, ExecutionEnvironment context) { return fromDataSet(context.fromCollection(edges), context); } /** - * Creates a graph from a Collection of edges, vertices are induced from the edges and - * vertex values are calculated by a mapper function. - * Vertices are created automatically and their values are set - * by applying the provided map function to the vertex ids. - * @param edges a Collection of vertices. + * Creates a graph from a Collection of edges, vertices are induced from the + * edges and vertex values are calculated by a mapper function. Vertices are + * created automatically and their values are set by applying the provided + * map function to the vertex ids. + * + * @param edges a Collection of edges. * @param mapper the mapper function. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromCollection (Collection> edges, - final MapFunction mapper, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromCollection( + Collection> edges, final MapFunction mapper, + ExecutionEnvironment context) { return fromDataSet(context.fromCollection(edges), mapper, context); } /** * Creates a graph from a DataSet of vertices and a DataSet of edges. + * * @param vertices a DataSet of vertices. - * @param edges a DataSet of vertices. + * @param edges a DataSet of edges. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromDataSet (DataSet> vertices, - DataSet> edges, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromDataSet( + DataSet> vertices, DataSet> edges, + ExecutionEnvironment context) { return new Graph(vertices, edges, context); } /** - * Creates a graph from a DataSet of edges, vertices are induced from the edges. - * Vertices are created automatically and their values are set to NullValue. - * @param edges a DataSet of vertices. + * Creates a graph from a DataSet of edges, vertices are induced from the + * edges. Vertices are created automatically and their values are set to + * NullValue. + * + * @param edges a DataSet of edges. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, EV extends Serializable> - Graph fromDataSet (DataSet> edges, - ExecutionEnvironment context) { + public static & Serializable, EV extends Serializable> Graph fromDataSet( + DataSet> edges, ExecutionEnvironment context) { - DataSet> vertices = - edges.flatMap(new EmitSrcAndTarget()).distinct(); + DataSet> vertices = edges.flatMap(new EmitSrcAndTarget()).distinct(); return new Graph(vertices, edges, context); } @@ -174,44 +181,41 @@ public void flatMap(Edge edge, Collector> out) { } /** - * Creates a graph from a DataSet of edges, vertices are induced from the edges and - * vertex values are calculated by a mapper function. - * Vertices are created automatically and their values are set - * by applying the provided map function to the vertex ids. - * @param edges a DataSet of vertices. + * Creates a graph from a DataSet of edges, vertices are induced from the + * edges and vertex values are calculated by a mapper function. Vertices are + * created automatically and their values are set by applying the provided + * map function to the vertex ids. + * + * @param edges a DataSet of edges. * @param mapper the mapper function. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromDataSet (DataSet> edges, - final MapFunction mapper, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromDataSet( + DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + TypeInformation valueType = TypeExtractor.createTypeInfo( + MapFunction.class, mapper.getClass(), 1, null, null); @SuppressWarnings({ "unchecked", "rawtypes" }) - TypeInformation> returnType = (TypeInformation>) - new TupleTypeInfo(Vertex.class, keyType, valueType); - - DataSet> vertices = - edges.flatMap(new EmitSrcAndTargetAsTuple1()) - .distinct() - .map(new MapFunction, Vertex>() { - public Vertex map(Tuple1 value) throws Exception { - return new Vertex(value.f0, mapper.map(value.f0)); - } - }) - .returns(returnType); + TypeInformation> returnType = (TypeInformation>) new TupleTypeInfo( + Vertex.class, keyType, valueType); + + DataSet> vertices = edges + .flatMap(new EmitSrcAndTargetAsTuple1()).distinct() + .map(new MapFunction, Vertex>() { + public Vertex map(Tuple1 value) throws Exception { + return new Vertex(value.f0, mapper.map(value.f0)); + } + }).returns(returnType); return new Graph(vertices, edges, context); } - private static final class EmitSrcAndTargetAsTuple1 & Serializable, - EV extends Serializable> implements FlatMapFunction, Tuple1> { + private static final class EmitSrcAndTargetAsTuple1 & Serializable, EV extends Serializable> + implements FlatMapFunction, Tuple1> { public void flatMap(Edge edge, Collector> out) { out.collect(new Tuple1(edge.f0)); @@ -221,19 +225,17 @@ public void flatMap(Edge edge, Collector> out) { /** * Creates a graph from a DataSet of Tuple objects for vertices and edges. - * - * Vertices with value are created from Tuple2, - * Edges with value are created from Tuple3. - * - * @param vertices a DataSet of vertices. - * @param edges a DataSet of vertices. + * + * Vertices with value are created from Tuple2, Edges with value are created + * from Tuple3. + * + * @param vertices a DataSet of Tuple2. + * @param edges a DataSet of Tuple3. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromTupleDataSet (DataSet> vertices, - DataSet> edges, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromTupleDataSet( + DataSet> vertices, DataSet> edges, ExecutionEnvironment context) { DataSet> vertexDataSet = vertices.map(new Tuple2ToVertexMap()); DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); @@ -241,38 +243,37 @@ Graph fromTupleDataSet (DataSet> vertices, } /** - * Creates a graph from a DataSet of Tuple objects for edges, vertices are induced from the edges. - * - * Edges with value are created from Tuple3. - * Vertices are created automatically and their values are set to NullValue. - * - * @param edges a DataSet of vertices. + * Creates a graph from a DataSet of Tuple objects for edges, vertices are + * induced from the edges. + * + * Edges with value are created from Tuple3. Vertices are created + * automatically and their values are set to NullValue. + * + * @param edges a DataSet of Tuple3. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, EV extends Serializable> - Graph fromTupleDataSet (DataSet> edges, - ExecutionEnvironment context) { + public static & Serializable, EV extends Serializable> Graph fromTupleDataSet( + DataSet> edges, ExecutionEnvironment context) { DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); return fromDataSet(edgeDataSet, context); } /** - * Creates a graph from a DataSet of Tuple objects for edges, vertices are induced from the edges and - * vertex values are calculated by a mapper function. - * Edges with value are created from Tuple3. - * Vertices are created automatically and their values are set - * by applying the provided map function to the vertex ids. - * @param edges a DataSet of vertices. + * Creates a graph from a DataSet of Tuple objects for edges, vertices are + * induced from the edges and vertex values are calculated by a mapper + * function. Edges with value are created from Tuple3. Vertices are created + * automatically and their values are set by applying the provided map + * function to the vertex ids. + * + * @param edges a DataSet of Tuple3. * @param mapper the mapper function. * @param context the flink execution environment. * @return the newly created graph. */ - public static & Serializable, VV extends Serializable, EV extends Serializable> - Graph fromTupleDataSet (DataSet> edges, - final MapFunction mapper, - ExecutionEnvironment context) { + public static & Serializable, VV extends Serializable, EV extends Serializable> Graph fromTupleDataSet( + DataSet> edges, final MapFunction mapper, ExecutionEnvironment context) { DataSet> edgeDataSet = edges.map(new Tuple3ToEdgeMap()); return fromDataSet(edgeDataSet, mapper, context); @@ -287,6 +288,7 @@ public ExecutionEnvironment getContext() { /** * Function that checks whether a graph's ids are valid + * * @return true if the graph's ids are valid, false if not. */ public DataSet validate(GraphValidator validator) { @@ -310,79 +312,79 @@ public DataSet> getEdges() { /** * @return the vertex DataSet as Tuple2. */ - @SuppressWarnings({ "unchecked" }) public DataSet> getVerticesAsTuple2() { - return (DataSet>) (DataSet) vertices; + return vertices.map(new VertexToTuple2Map()); } /** * @return the edge DataSet as Tuple3. */ - @SuppressWarnings({ "unchecked" }) public DataSet> getEdgesAsTuple3() { - return (DataSet>) (DataSet) edges; + return edges.map(new EdgeToTuple3Map()); } - /** - * Apply a function to the attribute of each vertex in the graph. - * @param mapper the map function to apply. - * @return a new graph - */ - @SuppressWarnings({ "unchecked", "rawtypes" }) + /** + * Apply a function to the attribute of each vertex in the graph. + * + * @param mapper the map function to apply. + * @return a new graph + */ + @SuppressWarnings({ "unchecked", "rawtypes" }) public Graph mapVertices(final MapFunction, NV> mapper) { - TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); + TypeInformation keyType = ((TupleTypeInfo) vertices.getType()).getTypeAt(0); - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + TypeInformation valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); - TypeInformation> returnType = (TypeInformation>) - new TupleTypeInfo(Vertex.class, keyType, valueType); + TypeInformation> returnType = (TypeInformation>) new TupleTypeInfo( + Vertex.class, keyType, valueType); - DataSet> mappedVertices = vertices - .map(new MapFunction, Vertex>() { - public Vertex map(Vertex value) throws Exception { - return new Vertex(value.f0, mapper.map(value)); - } - }) - .returns(returnType); + DataSet> mappedVertices = vertices.map( + new MapFunction, Vertex>() { + public Vertex map(Vertex value) throws Exception { + return new Vertex(value.f0, mapper.map(value)); + } + }).returns(returnType); - return new Graph(mappedVertices, this.edges, this.context); - } + return new Graph(mappedVertices, this.edges, this.context); + } - /** - * Apply a function to the attribute of each edge in the graph. - * @param mapper the map function to apply. - * @return a new graph - */ - @SuppressWarnings({ "unchecked", "rawtypes" }) + /** + * Apply a function to the attribute of each edge in the graph. + * + * @param mapper the map function to apply. + * @return a new graph + */ + @SuppressWarnings({ "unchecked", "rawtypes" }) public Graph mapEdges(final MapFunction, NV> mapper) { - TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); + TypeInformation keyType = ((TupleTypeInfo) edges.getType()).getTypeAt(0); - TypeInformation valueType = TypeExtractor - .createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); + TypeInformation valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null); - TypeInformation> returnType = (TypeInformation>) - new TupleTypeInfo(Edge.class, keyType, keyType, valueType); + TypeInformation> returnType = (TypeInformation>) new TupleTypeInfo( + Edge.class, keyType, keyType, valueType); - DataSet> mappedEdges = edges.map(new MapFunction, Edge>() { - public Edge map(Edge value) throws Exception { - return new Edge(value.f0, value.f1, mapper.map(value)); - } - }) - .returns(returnType); + DataSet> mappedEdges = edges.map( + new MapFunction, Edge>() { + public Edge map(Edge value) throws Exception { + return new Edge(value.f0, value.f1, mapper + .map(value)); + } + }).returns(returnType); - return new Graph(this.vertices, mappedEdges, this.context); - } + return new Graph(this.vertices, mappedEdges, this.context); + } /** - * Joins the vertex DataSet of this graph with an input DataSet and applies a UDF on the resulted values. + * Joins the vertex DataSet of this graph with an input DataSet and applies + * a UDF on the resulted values. + * * @param inputDataSet the DataSet to join with. * @param mapper the UDF map function to apply. * @return a new graph where the vertex values have been updated. */ - public Graph joinWithVertices(DataSet> inputDataSet, + public Graph joinWithVertices(DataSet> inputDataSet, final MapFunction, VV> mapper) { DataSet> resultedVertices = this.getVertices() @@ -391,75 +393,78 @@ public Graph joinWithVertices(DataSet> inputDataSet, return new Graph(resultedVertices, this.edges, this.context); } - private static final class ApplyCoGroupToVertexValues & Serializable, - VV extends Serializable, T> implements CoGroupFunction, Tuple2, Vertex> { + private static final class ApplyCoGroupToVertexValues & Serializable, VV extends Serializable, T> + implements CoGroupFunction, Tuple2, Vertex> { private MapFunction, VV> mapper; + public ApplyCoGroupToVertexValues(MapFunction, VV> mapper) { this.mapper = mapper; } @Override - public void coGroup(Iterable> vertices, Iterable> input, - Collector> collector) throws Exception { + public void coGroup(Iterable> vertices, + Iterable> input, Collector> collector) throws Exception { final Iterator> vertexIterator = vertices.iterator(); final Iterator> inputIterator = input.iterator(); if (vertexIterator.hasNext()) { - if(inputIterator.hasNext()) { + if (inputIterator.hasNext()) { final Tuple2 inputNext = inputIterator.next(); collector.collect(new Vertex(inputNext.f0, mapper - .map(new Tuple2(vertexIterator.next().f1, inputNext.f1)))); + .map(new Tuple2(vertexIterator.next().f1, + inputNext.f1)))); } else { collector.collect(vertexIterator.next()); } - + } } } /** - * Joins the edge DataSet with an input DataSet on a composite key of both source and target - * and applies a UDF on the resulted values. + * Joins the edge DataSet with an input DataSet on a composite key of both + * source and target and applies a UDF on the resulted values. + * * @param inputDataSet the DataSet to join with. * @param mapper the UDF map function to apply. - * @param + * @param the return type * @return a new graph where the edge values have been updated. */ - public Graph joinWithEdges(DataSet> inputDataSet, + public Graph joinWithEdges(DataSet> inputDataSet, final MapFunction, EV> mapper) { DataSet> resultedEdges = this.getEdges() - .coGroup(inputDataSet).where(0,1).equalTo(0,1) + .coGroup(inputDataSet).where(0, 1).equalTo(0, 1) .with(new ApplyCoGroupToEdgeValues(mapper)); return new Graph(this.vertices, resultedEdges, this.context); } - private static final class ApplyCoGroupToEdgeValues & Serializable, - EV extends Serializable, T> + private static final class ApplyCoGroupToEdgeValues & Serializable, EV extends Serializable, T> implements CoGroupFunction, Tuple3, Edge> { private MapFunction, EV> mapper; + public ApplyCoGroupToEdgeValues(MapFunction, EV> mapper) { this.mapper = mapper; } @Override - public void coGroup(Iterable> edges, - Iterable> input, - Collector> collector) throws Exception { + public void coGroup(Iterable> edges, Iterable> input, + Collector> collector) throws Exception { final Iterator> edgesIterator = edges.iterator(); final Iterator> inputIterator = input.iterator(); if (edgesIterator.hasNext()) { - if(inputIterator.hasNext()) { + if (inputIterator.hasNext()) { final Tuple3 inputNext = inputIterator.next(); - collector.collect(new Edge(inputNext.f0, inputNext.f1, mapper - .map(new Tuple2(edgesIterator.next().f2, inputNext.f2)))); + collector.collect(new Edge(inputNext.f0, + inputNext.f1, mapper.map(new Tuple2( + edgesIterator.next().f2, inputNext.f2)))); } else { collector.collect(edgesIterator.next()); } @@ -468,12 +473,14 @@ public void coGroup(Iterable> edges, } /** - * Joins the edge DataSet with an input DataSet on the source key of the edges and the first attribute - * of the input DataSet and applies a UDF on the resulted values. - * In case the inputDataSet contains the same key more than once, only the first value will be considered. + * Joins the edge DataSet with an input DataSet on the source key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. In case the inputDataSet contains the same key more + * than once, only the first value will be considered. + * * @param inputDataSet the DataSet to join with. * @param mapper the UDF map function to apply. - * @param + * @param the return type * @return a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnSource(DataSet> inputDataSet, @@ -486,33 +493,36 @@ public Graph joinWithEdgesOnSource(DataSet> inputDat return new Graph(this.vertices, resultedEdges, this.context); } - private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget & Serializable, - EV extends Serializable, T> implements CoGroupFunction, Tuple2, Edge> { + private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget & Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple2, Edge> { private MapFunction, EV> mapper; - public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(MapFunction, EV> mapper) { + + public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget( + MapFunction, EV> mapper) { this.mapper = mapper; } @Override - public void coGroup(Iterable> edges, Iterable> input, - Collector> collector) throws Exception { + public void coGroup(Iterable> edges, + Iterable> input, Collector> collector) throws Exception { final Iterator> edgesIterator = edges.iterator(); final Iterator> inputIterator = input.iterator(); - if(inputIterator.hasNext()) { + if (inputIterator.hasNext()) { final Tuple2 inputNext = inputIterator.next(); - while(edgesIterator.hasNext()) { + while (edgesIterator.hasNext()) { Edge edgesNext = edgesIterator.next(); - collector.collect(new Edge(edgesNext.f0, edgesNext.f1, mapper - .map(new Tuple2(edgesNext.f2, inputNext.f1)))); + collector.collect(new Edge(edgesNext.f0, + edgesNext.f1, mapper.map(new Tuple2( + edgesNext.f2, inputNext.f1)))); } } else { - while(edgesIterator.hasNext()) { + while (edgesIterator.hasNext()) { collector.collect(edgesIterator.next()); } } @@ -520,12 +530,14 @@ public void coGroup(Iterable> edges, Iterable> input, } /** - * Joins the edge DataSet with an input DataSet on the target key of the edges and the first attribute - * of the input DataSet and applies a UDF on the resulted values. - * Should the inputDataSet contain the same key more than once, only the first value will be considered. + * Joins the edge DataSet with an input DataSet on the target key of the + * edges and the first attribute of the input DataSet and applies a UDF on + * the resulted values. Should the inputDataSet contain the same key more + * than once, only the first value will be considered. + * * @param inputDataSet the DataSet to join with. * @param mapper the UDF map function to apply. - * @param + * @param the return type * @return a new graph where the edge values have been updated. */ public Graph joinWithEdgesOnTarget(DataSet> inputDataSet, @@ -539,32 +551,32 @@ public Graph joinWithEdgesOnTarget(DataSet> inputDat } /** - * Apply filtering functions to the graph - * and return a sub-graph that satisfies the predicates - * for both vertices and edges. + * Apply filtering functions to the graph and return a sub-graph that + * satisfies the predicates for both vertices and edges. + * * @param vertexFilter the filter function for vertices. * @param edgeFilter the filter function for edges. * @return the resulting sub-graph. */ public Graph subgraph(FilterFunction> vertexFilter, FilterFunction> edgeFilter) { - DataSet> filteredVertices = this.vertices.filter(vertexFilter); + DataSet> filteredVertices = this.vertices.filter(vertexFilter); - DataSet> remainingEdges = this.edges.join(filteredVertices) - .where(0).equalTo(0) - .with(new ProjectEdge()) - .join(filteredVertices).where(1).equalTo(0) - .with(new ProjectEdge()); + DataSet> remainingEdges = this.edges.join(filteredVertices) + .where(0).equalTo(0).with(new ProjectEdge()) + .join(filteredVertices).where(1).equalTo(0) + .with(new ProjectEdge()); - DataSet> filteredEdges = remainingEdges.filter(edgeFilter); + DataSet> filteredEdges = remainingEdges.filter(edgeFilter); - return new Graph(filteredVertices, filteredEdges, this.context); - } + return new Graph(filteredVertices, filteredEdges, + this.context); + } /** - * Apply a filtering function to the graph - * and return a sub-graph that satisfies the predicates - * only for the vertices. + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the vertices. + * * @param vertexFilter the filter function for vertices. * @return the resulting sub-graph. */ @@ -573,8 +585,7 @@ public Graph filterOnVertices(FilterFunction> vertexFil DataSet> filteredVertices = this.vertices.filter(vertexFilter); DataSet> remainingEdges = this.edges.join(filteredVertices) - .where(0).equalTo(0) - .with(new ProjectEdge()) + .where(0).equalTo(0).with(new ProjectEdge()) .join(filteredVertices).where(1).equalTo(0) .with(new ProjectEdge()); @@ -582,9 +593,9 @@ public Graph filterOnVertices(FilterFunction> vertexFil } /** - * Apply a filtering function to the graph - * and return a sub-graph that satisfies the predicates - * only for the edges. + * Apply a filtering function to the graph and return a sub-graph that + * satisfies the predicates only for the edges. + * * @param edgeFilter the filter function for edges. * @return the resulting sub-graph. */ @@ -594,32 +605,29 @@ public Graph filterOnEdges(FilterFunction> edgeFilter) { return new Graph(this.vertices, filteredEdges, this.context); } - @ConstantFieldsFirst("0->0;1->1;2->2") - private static final class ProjectEdge & Serializable, - VV extends Serializable, EV extends Serializable> implements FlatJoinFunction, Vertex, - Edge> { - public void join(Edge first, - Vertex second, Collector> out) { + @ConstantFieldsFirst("0->0;1->1;2->2") + private static final class ProjectEdge & Serializable, VV extends Serializable, EV extends Serializable> + implements FlatJoinFunction, Vertex, Edge> { + public void join(Edge first, Vertex second, Collector> out) { out.collect(first); } - } - - /** - * Return the out-degree of all vertices in the graph - * @return A DataSet of Tuple2 - */ + } + + /** + * Return the out-degree of all vertices in the graph + * + * @return A DataSet of Tuple2 + */ public DataSet> outDegrees() { - return vertices.coGroup(edges).where(0).equalTo(0) - .with(new CountNeighborsCoGroup()); + return vertices.coGroup(edges).where(0).equalTo(0).with(new CountNeighborsCoGroup()); } - private static final class CountNeighborsCoGroup & Serializable, - VV extends Serializable, EV extends Serializable> implements CoGroupFunction, - Edge, Tuple2> { + private static final class CountNeighborsCoGroup & Serializable, VV extends Serializable, EV extends Serializable> + implements CoGroupFunction, Edge, Tuple2> { @SuppressWarnings("unused") - public void coGroup(Iterable> vertex, - Iterable> outEdges, Collector> out) { + public void coGroup(Iterable> vertex, Iterable> outEdges, + Collector> out) { long count = 0; for (Edge edge : outEdges) { count++; @@ -627,19 +635,20 @@ public void coGroup(Iterable> vertex, out.collect(new Tuple2(vertex.iterator().next().f0, count)); } } - + /** * Return the in-degree of all vertices in the graph + * * @return A DataSet of Tuple2 */ public DataSet> inDegrees() { - return vertices.coGroup(edges).where(0).equalTo(1) - .with(new CountNeighborsCoGroup()); + return vertices.coGroup(edges).where(0).equalTo(1).with(new CountNeighborsCoGroup()); } /** * Return the degree of all vertices in the graph + * * @return A DataSet of Tuple2 */ public DataSet> getDegrees() { @@ -647,22 +656,26 @@ public DataSet> getDegrees() { } /** - * This operation adds all inverse-direction edges - * to the graph. + * This operation adds all inverse-direction edges to the graph. + * * @return the undirected graph. */ - public Graph getUndirected() throws UnsupportedOperationException { - DataSet> undirectedEdges = - edges.union(edges.map(new ReverseEdgesMap())); - return new Graph(vertices, undirectedEdges, this.context); + public Graph getUndirected() { + + DataSet> undirectedEdges = edges.union(edges.map(new ReverseEdgesMap())); + return new Graph(vertices, undirectedEdges, this.context); } - + /** - * Compute an aggregate over the edges of each vertex. - * The function applied on the edges has access to the vertex value. - * @param edgesFunction the function to apply to the neighborhood - * @param direction the edge direction (in-, out-, all-) - * @param the output type + * Compute an aggregate over the edges of each vertex. The function applied + * on the edges has access to the vertex value. + * + * @param edgesFunction + * the function to apply to the neighborhood + * @param direction + * the edge direction (in-, out-, all-) + * @param + * the output type * @return a dataset of a T * @throws IllegalArgumentException */ @@ -671,27 +684,29 @@ public DataSet reduceOnEdges(EdgesFunctionWithVertexValue e switch (direction) { case IN: - return vertices.coGroup(edges).where(0).equalTo(1).with( - new ApplyCoGroupFunction(edgesFunction)); + return vertices.coGroup(edges).where(0).equalTo(1) + .with(new ApplyCoGroupFunction(edgesFunction)); case OUT: - return vertices.coGroup(edges).where(0).equalTo(0).with( - new ApplyCoGroupFunction(edgesFunction)); + return vertices.coGroup(edges).where(0).equalTo(0) + .with(new ApplyCoGroupFunction(edgesFunction)); case ALL: return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode())) - .where(0).equalTo(0) - .with(new ApplyCoGroupFunctionOnAllEdges(edgesFunction)); + .where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges(edgesFunction)); default: throw new IllegalArgumentException("Illegal edge direction"); } } /** - * Compute an aggregate over the edges of each vertex. - * The function applied on the edges only has access to the vertex id - * (not the vertex value). - * @param edgesFunction the function to apply to the neighborhood - * @param direction the edge direction (in-, out-, all-) - * @param the output type + * Compute an aggregate over the edges of each vertex. The function applied + * on the edges only has access to the vertex id (not the vertex value). + * + * @param edgesFunction + * the function to apply to the neighborhood + * @param direction + * the edge direction (in-, out-, all-) + * @param + * the output type * @return a dataset of T * @throws IllegalArgumentException */ @@ -706,15 +721,15 @@ public DataSet reduceOnEdges(EdgesFunction edgesFunction, return edges.map(new ProjectVertexIdMap(0)) .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); case ALL: - return edges.flatMap(new EmitOneEdgePerNode()).groupBy(0) - .reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); + return edges.flatMap(new EmitOneEdgePerNode()) + .groupBy(0).reduceGroup(new ApplyGroupReduceFunction(edgesFunction)); default: throw new IllegalArgumentException("Illegal edge direction"); } } - private static final class ProjectVertexIdMap & Serializable, - EV extends Serializable> implements MapFunction, Tuple2>> { + private static final class ProjectVertexIdMap & Serializable, EV extends Serializable> + implements MapFunction, Tuple2>> { private int fieldPosition; @@ -724,13 +739,12 @@ public ProjectVertexIdMap(int position) { @SuppressWarnings("unchecked") public Tuple2> map(Edge edge) { - return new Tuple2>((K) edge.getField(fieldPosition), edge); + return new Tuple2>((K) edge.getField(fieldPosition), edge); } } - private static final class ApplyGroupReduceFunction & Serializable, - EV extends Serializable, T> implements GroupReduceFunction>, T>, - ResultTypeQueryable { + private static final class ApplyGroupReduceFunction & Serializable, EV extends Serializable, T> + implements GroupReduceFunction>, T>, ResultTypeQueryable { private EdgesFunction function; @@ -738,8 +752,7 @@ public ApplyGroupReduceFunction(EdgesFunction fun) { this.function = fun; } - public void reduce(Iterable>> edges, - Collector out) throws Exception { + public void reduce(Iterable>> edges, Collector out) throws Exception { out.collect(function.iterateEdges(edges)); } @@ -749,34 +762,31 @@ public TypeInformation getProducedType() { } } - private static final class EmitOneEdgePerNode & Serializable, - VV extends Serializable, EV extends Serializable> implements FlatMapFunction< - Edge, Tuple2>> { + private static final class EmitOneEdgePerNode & Serializable, VV extends Serializable, EV extends Serializable> + implements FlatMapFunction, Tuple2>> { public void flatMap(Edge edge, Collector>> out) { out.collect(new Tuple2>(edge.getSource(), edge)); out.collect(new Tuple2>(edge.getTarget(), edge)); } } - private static final class EmitOneEdgeWithNeighborPerNode & Serializable, - VV extends Serializable, EV extends Serializable> implements FlatMapFunction< - Edge, Tuple3>> { + private static final class EmitOneEdgeWithNeighborPerNode & Serializable, VV extends Serializable, EV extends Serializable> + implements FlatMapFunction, Tuple3>> { public void flatMap(Edge edge, Collector>> out) { out.collect(new Tuple3>(edge.getSource(), edge.getTarget(), edge)); out.collect(new Tuple3>(edge.getTarget(), edge.getSource(), edge)); } } - private static final class ApplyCoGroupFunction & Serializable, - VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Edge, T>, - ResultTypeQueryable { - + private static final class ApplyCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Edge, T>, ResultTypeQueryable { + private EdgesFunctionWithVertexValue function; - - public ApplyCoGroupFunction (EdgesFunctionWithVertexValue fun) { + + public ApplyCoGroupFunction(EdgesFunctionWithVertexValue fun) { this.function = fun; } + public void coGroup(Iterable> vertex, Iterable> edges, Collector out) throws Exception { out.collect(function.iterateEdges(vertex.iterator().next(), edges)); @@ -784,63 +794,62 @@ public void coGroup(Iterable> vertex, @Override public TypeInformation getProducedType() { - return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null); + return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, + null, null); } } - private static final class ApplyCoGroupFunctionOnAllEdges & Serializable, - VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple2>, T>, - ResultTypeQueryable { + private static final class ApplyCoGroupFunctionOnAllEdges & Serializable, VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple2>, T>, ResultTypeQueryable { - private EdgesFunctionWithVertexValue function; + private EdgesFunctionWithVertexValue function; - public ApplyCoGroupFunctionOnAllEdges (EdgesFunctionWithVertexValue fun) { - this.function = fun; - } + public ApplyCoGroupFunctionOnAllEdges(EdgesFunctionWithVertexValue fun) { + this.function = fun; + } - public void coGroup(Iterable> vertex, final Iterable>> keysWithEdges, - Collector out) throws Exception { + public void coGroup(Iterable> vertex, final Iterable>> keysWithEdges, + Collector out) throws Exception { - final Iterator> edgesIterator = new Iterator>() { + final Iterator> edgesIterator = new Iterator>() { - final Iterator>> keysWithEdgesIterator = keysWithEdges.iterator(); + final Iterator>> keysWithEdgesIterator = keysWithEdges.iterator(); - @Override - public boolean hasNext() { - return keysWithEdgesIterator.hasNext(); - } + @Override + public boolean hasNext() { + return keysWithEdgesIterator.hasNext(); + } - @Override - public Edge next() { - return keysWithEdgesIterator.next().f1; - } + @Override + public Edge next() { + return keysWithEdgesIterator.next().f1; + } - @Override - public void remove() { - keysWithEdgesIterator.remove(); - } - }; + @Override + public void remove() { + keysWithEdgesIterator.remove(); + } + }; - Iterable> edgesIterable = new Iterable>() { - public Iterator> iterator() { - return edgesIterator; - } - }; + Iterable> edgesIterable = new Iterable>() { + public Iterator> iterator() { + return edgesIterator; + } + }; - out.collect(function.iterateEdges(vertex.iterator().next(), edgesIterable)); - } + out.collect(function.iterateEdges(vertex.iterator().next(), edgesIterable)); + } - @Override - public TypeInformation getProducedType() { - return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, null, null); + @Override + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3, + null, null); + } } -} @ConstantFields("0->1;1->0;2->2") - private static final class ReverseEdgesMap & Serializable, - EV extends Serializable> implements MapFunction, - Edge> { + private static final class ReverseEdgesMap & Serializable, EV extends Serializable> + implements MapFunction, Edge> { public Edge map(Edge value) { return new Edge(value.f1, value.f0, value.f2); @@ -849,6 +858,7 @@ public Edge map(Edge value) { /** * Reverse the direction of the edges in the graph + * * @return a new graph with all edges reversed * @throws UnsupportedOperationException */ @@ -860,274 +870,278 @@ public Graph reverse() throws UnsupportedOperationException { /** * @return Singleton DataSet containing the vertex count */ - public DataSet numberOfVertices () { - return GraphUtils.count(vertices, context); - } + public DataSet numberOfVertices() { + return GraphUtils.count(vertices, context); + } /** * @return Singleton DataSet containing the edge count */ - public DataSet numberOfEdges () { - return GraphUtils.count(edges, context); - } - - /** - * @return The IDs of the vertices as DataSet - */ - public DataSet getVertexIds () { - return vertices.map(new ExtractVertexIDMapper()); - } - - private static final class ExtractVertexIDMapper & Serializable, - VV extends Serializable> implements MapFunction, K> { - @Override - public K map(Vertex vertex) { - return vertex.f0; - } - } - - /** - * @return The IDs of the edges as DataSet - */ - public DataSet> getEdgeIds () { - return edges.map(new ExtractEdgeIDsMapper()); - } - - private static final class ExtractEdgeIDsMapper & Serializable, - EV extends Serializable> implements MapFunction, Tuple2> { - @Override - public Tuple2 map(Edge edge) throws Exception { - return new Tuple2(edge.f0, edge.f1); - } - } + public DataSet numberOfEdges() { + return GraphUtils.count(edges, context); + } + + /** + * @return The IDs of the vertices as DataSet + */ + public DataSet getVertexIds() { + return vertices.map(new ExtractVertexIDMapper()); + } + + private static final class ExtractVertexIDMapper & Serializable, VV extends Serializable> + implements MapFunction, K> { + @Override + public K map(Vertex vertex) { + return vertex.f0; + } + } + + /** + * @return The IDs of the edges as DataSet + */ + public DataSet> getEdgeIds() { + return edges.map(new ExtractEdgeIDsMapper()); + } + + private static final class ExtractEdgeIDsMapper & Serializable, EV extends Serializable> + implements MapFunction, Tuple2> { + @Override + public Tuple2 map(Edge edge) throws Exception { + return new Tuple2(edge.f0, edge.f1); + } + } /** * Checks the weak connectivity of a graph. - * @param maxIterations the maximum number of iterations for the inner delta iteration + * + * @param maxIterations + * the maximum number of iterations for the inner delta iteration * @return true if the graph is weakly connected. */ - public DataSet isWeaklyConnected (int maxIterations) { + public DataSet isWeaklyConnected(int maxIterations) { // first, convert to an undirected graph Graph graph = this.getUndirected(); DataSet vertexIds = graph.getVertexIds(); - DataSet> verticesWithInitialIds = vertexIds - .map(new DuplicateVertexIDMapper()); - - DataSet> edgeIds = graph.getEdgeIds(); - - DeltaIteration, Tuple2> iteration = verticesWithInitialIds - .iterateDelta(verticesWithInitialIds, maxIterations, 0); - - DataSet> changes = iteration.getWorkset() - .join(edgeIds, JoinHint.REPARTITION_SORT_MERGE) - .where(0).equalTo(0) - .with(new FindNeighborsJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet(), JoinHint.REPARTITION_SORT_MERGE) - .where(0).equalTo(0) - .with(new VertexWithNewComponentJoin()); - - DataSet> components = iteration.closeWith(changes, changes); - DataSet result = GraphUtils.count(components.groupBy(1).reduceGroup( - new EmitFirstReducer()), context).map(new CheckIfOneComponentMapper()); - return result; - } - + DataSet> verticesWithInitialIds = vertexIds + .map(new DuplicateVertexIDMapper()); + + DataSet> edgeIds = graph.getEdgeIds(); + + DeltaIteration, Tuple2> iteration = verticesWithInitialIds + .iterateDelta(verticesWithInitialIds, maxIterations, 0); + + DataSet> changes = iteration.getWorkset() + .join(edgeIds, JoinHint.REPARTITION_SORT_MERGE) + .where(0).equalTo(0).with(new FindNeighborsJoin()) + .groupBy(0).aggregate(Aggregations.MIN, 1) + .join(iteration.getSolutionSet(), JoinHint.REPARTITION_SORT_MERGE).where(0).equalTo(0) + .with(new VertexWithNewComponentJoin()); + + DataSet> components = iteration.closeWith(changes, changes); + DataSet result = GraphUtils.count(components.groupBy(1).reduceGroup(new EmitFirstReducer()), + context).map(new CheckIfOneComponentMapper()); + return result; + } + private static final class DuplicateVertexIDMapper implements MapFunction> { - @Override - public Tuple2 map(K k) { - return new Tuple2(k, k); - } - } - - private static final class FindNeighborsJoin implements JoinFunction, Tuple2, - Tuple2> { - @Override - public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) { - return new Tuple2(edge.f1, vertexWithComponent.f1); - } - } - - private static final class VertexWithNewComponentJoin> - implements FlatJoinFunction, Tuple2, Tuple2> { - @Override - public void join(Tuple2 candidate, Tuple2 old, Collector> out) { - if (candidate.f1.compareTo(old.f1) < 0) { - out.collect(candidate); - } - } - } - - private static final class EmitFirstReducer implements - GroupReduceFunction, Tuple2> { + @Override + public Tuple2 map(K k) { + return new Tuple2(k, k); + } + } + + private static final class FindNeighborsJoin implements JoinFunction, Tuple2, Tuple2> { + @Override + public Tuple2 join(Tuple2 vertexWithComponent, Tuple2 edge) { + return new Tuple2(edge.f1, vertexWithComponent.f1); + } + } + + private static final class VertexWithNewComponentJoin> + implements FlatJoinFunction, Tuple2, Tuple2> { + @Override + public void join(Tuple2 candidate, Tuple2 old, Collector> out) { + if (candidate.f1.compareTo(old.f1) < 0) { + out.collect(candidate); + } + } + } + + private static final class EmitFirstReducer implements GroupReduceFunction, Tuple2> { public void reduce(Iterable> values, Collector> out) { - out.collect(values.iterator().next()); + out.collect(values.iterator().next()); } } - - private static final class CheckIfOneComponentMapper implements MapFunction { - @Override - public Boolean map(Integer n) { - return (n == 1); - } + + private static final class CheckIfOneComponentMapper implements MapFunction { + @Override + public Boolean map(Integer n) { + return (n == 1); + } } - + /** - * Adds the input vertex and edges to the graph. - * If the vertex already exists in the graph, it will not be added again, - * but the given edges will. + * Adds the input vertex and edges to the graph. If the vertex already + * exists in the graph, it will not be added again, but the given edges + * will. + * * @param vertex the vertex to add to the graph * @param edges a list of edges to add to the grap - * @return the new graph containing the existing and newly added vertices and edges + * @return the new graph containing the existing and newly added vertices + * and edges */ @SuppressWarnings("unchecked") - public Graph addVertex (final Vertex vertex, List> edges) { - DataSet> newVertex = this.context.fromElements(vertex); - - // Take care of empty edge set - if (edges.isEmpty()) { - return new Graph(this.vertices.union(newVertex).distinct(), this.edges, this.context); - } - - // Add the vertex and its edges - DataSet> newVertices = this.vertices.union(newVertex).distinct(); - DataSet> newEdges = this.edges.union(context.fromCollection(edges)); - - return new Graph(newVertices, newEdges, this.context); - } - - /** - * Adds the given edge to the graph. - * If the source and target vertices do not exist in the graph, - * they will also be added. - * @param source the source vertex of the edge - * @param target the target vertex of the edge - * @param edgeValue the edge value - * @return the new graph containing the existing vertices and edges plus the newly added edge - */ - @SuppressWarnings("unchecked") - public Graph addEdge (Vertex source, Vertex target, EV edgeValue) { - Graph partialGraph = fromCollection( - Arrays.asList(source, target), + public Graph addVertex(final Vertex vertex, List> edges) { + DataSet> newVertex = this.context.fromElements(vertex); + + // Take care of empty edge set + if (edges.isEmpty()) { + return new Graph(this.vertices.union(newVertex) + .distinct(), this.edges, this.context); + } + + // Add the vertex and its edges + DataSet> newVertices = this.vertices.union(newVertex).distinct(); + DataSet> newEdges = this.edges.union(context.fromCollection(edges)); + + return new Graph(newVertices, newEdges, this.context); + } + + /** + * Adds the given edge to the graph. If the source and target vertices do + * not exist in the graph, they will also be added. + * + * @param source the source vertex of the edge + * @param target the target vertex of the edge + * @param edgeValue the edge value + * @return the new graph containing the existing vertices and edges plus the + * newly added edge + */ + @SuppressWarnings("unchecked") + public Graph addEdge(Vertex source, Vertex target, EV edgeValue) { + Graph partialGraph = fromCollection(Arrays.asList(source, target), Arrays.asList(new Edge(source.f0, target.f0, edgeValue)), - this.context - ); - return this.union(partialGraph); - } + this.context); + return this.union(partialGraph); + } /** * Removes the given vertex and its edges from the graph. + * * @param vertex the vertex to remove - * @return the new graph containing the existing vertices and edges without the removed vertex and its edges + * @return the new graph containing the existing vertices and edges without + * the removed vertex and its edges */ - public Graph removeVertex (Vertex vertex) { - - DataSet> newVertices = getVertices().filter( - new RemoveVertexFilter(vertex)); - DataSet> newEdges = getEdges().filter( - new VertexRemovalEdgeFilter(vertex)); - return new Graph(newVertices, newEdges, this.context); - } - - private static final class RemoveVertexFilter & Serializable, - VV extends Serializable> implements FilterFunction> { - - private Vertex vertexToRemove; - - public RemoveVertexFilter(Vertex vertex) { - vertexToRemove = vertex; + public Graph removeVertex(Vertex vertex) { + + DataSet> newVertices = getVertices().filter(new RemoveVertexFilter(vertex)); + DataSet> newEdges = getEdges().filter(new VertexRemovalEdgeFilter(vertex)); + return new Graph(newVertices, newEdges, this.context); + } + + private static final class RemoveVertexFilter & Serializable, VV extends Serializable> + implements FilterFunction> { + + private Vertex vertexToRemove; + + public RemoveVertexFilter(Vertex vertex) { + vertexToRemove = vertex; } - @Override - public boolean filter(Vertex vertex) throws Exception { - return !vertex.f0.equals(vertexToRemove.f0); - } - } - - private static final class VertexRemovalEdgeFilter & Serializable, - VV extends Serializable, EV extends Serializable> implements FilterFunction> { + @Override + public boolean filter(Vertex vertex) throws Exception { + return !vertex.f0.equals(vertexToRemove.f0); + } + } + + private static final class VertexRemovalEdgeFilter & Serializable, VV extends Serializable, EV extends Serializable> + implements FilterFunction> { - private Vertex vertexToRemove; + private Vertex vertexToRemove; - public VertexRemovalEdgeFilter(Vertex vertex) { + public VertexRemovalEdgeFilter(Vertex vertex) { vertexToRemove = vertex; } - @Override - public boolean filter(Edge edge) throws Exception { - - if (edge.f0.equals(vertexToRemove.f0)) { - return false; - } - if (edge.f1.equals(vertexToRemove.f0)) { - return false; - } - return true; - } - } - - /** - * Removes all edges that match the given edge from the graph. - * @param edge the edge to remove - * @return the new graph containing the existing vertices and edges without the removed edges - */ - public Graph removeEdge (Edge edge) { - DataSet> newEdges = getEdges().filter( - new EdgeRemovalEdgeFilter(edge)); - return new Graph(this.vertices, newEdges, this.context); - } - - private static final class EdgeRemovalEdgeFilter & Serializable, - EV extends Serializable> implements FilterFunction> { - private Edge edgeToRemove; - - public EdgeRemovalEdgeFilter(Edge edge) { + @Override + public boolean filter(Edge edge) throws Exception { + + if (edge.f0.equals(vertexToRemove.f0)) { + return false; + } + if (edge.f1.equals(vertexToRemove.f0)) { + return false; + } + return true; + } + } + + /** + * Removes all edges that match the given edge from the graph. + * + * @param edge the edge to remove + * @return the new graph containing the existing vertices and edges without + * the removed edges + */ + public Graph removeEdge(Edge edge) { + DataSet> newEdges = getEdges().filter(new EdgeRemovalEdgeFilter(edge)); + return new Graph(this.vertices, newEdges, this.context); + } + + private static final class EdgeRemovalEdgeFilter & Serializable, EV extends Serializable> + implements FilterFunction> { + private Edge edgeToRemove; + + public EdgeRemovalEdgeFilter(Edge edge) { edgeToRemove = edge; } - @Override - public boolean filter(Edge edge) { - return (!(edge.f0.equals(edgeToRemove.f0) - && edge.f1.equals(edgeToRemove.f1))); - } - } - - /** - * Performs union on the vertices and edges sets of the input graphs - * removing duplicate vertices but maintaining duplicate edges. - * @param graph the graph to perform union with - * @return a new graph - */ - public Graph union (Graph graph) { - DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); - DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); - return new Graph(unionedVertices, unionedEdges, this.context); - } + @Override + public boolean filter(Edge edge) { + return (!(edge.f0.equals(edgeToRemove.f0) && edge.f1 + .equals(edgeToRemove.f1))); + } + } + + /** + * Performs union on the vertices and edges sets of the input graphs + * removing duplicate vertices but maintaining duplicate edges. + * + * @param graph the graph to perform union with + * @return a new graph + */ + public Graph union(Graph graph) { + + DataSet> unionedVertices = graph.getVertices().union(this.getVertices()).distinct(); + DataSet> unionedEdges = graph.getEdges().union(this.getEdges()); + return new Graph(unionedVertices, unionedEdges, this.context); + } /** * Runs a Vertex-Centric iteration on the graph. + * * @param vertexUpdateFunction the vertex update function * @param messagingFunction the messaging function * @param maximumNumberOfIterations maximum number of iterations to perform * @return */ - public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, - MessagingFunction messagingFunction, int maximumNumberOfIterations) { - DataSet> newVertices = vertices.runOperation( - VertexCentricIteration.withEdges(edges, - vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); + public Graph runVertexCentricIteration(VertexUpdateFunction vertexUpdateFunction, + MessagingFunction messagingFunction, int maximumNumberOfIterations) { + DataSet> newVertices = vertices.runOperation(VertexCentricIteration + .withEdges(edges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations)); return new Graph(newVertices, this.edges, this.context); - } + } - public Graph run (GraphAlgorithm algorithm) { + public Graph run(GraphAlgorithm algorithm) { return algorithm.run(this); } /** - * Compute an aggregate over the neighbors (edges and vertices) of each vertex. - * The function applied on the neighbors has access to the vertex value. + * Compute an aggregate over the neighbors (edges and vertices) of each + * vertex. The function applied on the neighbors has access to the vertex + * value. + * * @param neighborsFunction the function to apply to the neighborhood * @param direction the edge direction (in-, out-, all-) * @param the output type @@ -1139,23 +1153,27 @@ public DataSet reduceOnNeighbors(NeighborsFunctionWithVertexValue pairs - DataSet, Vertex>> edgesWithSources = edges.join(this.vertices) - .where(0).equalTo(0); - return vertices.coGroup(edgesWithSources).where(0).equalTo("f0.f1").with( - new ApplyNeighborCoGroupFunction(neighborsFunction)); + DataSet, Vertex>> edgesWithSources = edges + .join(this.vertices).where(0).equalTo(0); + return vertices.coGroup(edgesWithSources) + .where(0).equalTo("f0.f1") + .with(new ApplyNeighborCoGroupFunction(neighborsFunction)); case OUT: // create pairs - DataSet, Vertex>> edgesWithTargets = edges.join(this.vertices) - .where(1).equalTo(0); - return vertices.coGroup(edgesWithTargets).where(0).equalTo("f0.f0").with( - new ApplyNeighborCoGroupFunction(neighborsFunction)); + DataSet, Vertex>> edgesWithTargets = edges + .join(this.vertices).where(1).equalTo(0); + return vertices.coGroup(edgesWithTargets) + .where(0).equalTo("f0.f0") + .with(new ApplyNeighborCoGroupFunction(neighborsFunction)); case ALL: // create pairs - DataSet, Vertex>> edgesWithNeighbors = edges.flatMap( - new EmitOneEdgeWithNeighborPerNode()).join(this.vertices) - .where(1).equalTo(0).with(new ProjectEdgeWithNeighbor()); + DataSet, Vertex>> edgesWithNeighbors = edges + .flatMap(new EmitOneEdgeWithNeighborPerNode()) + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectEdgeWithNeighbor()); - return vertices.coGroup(edgesWithNeighbors).where(0).equalTo(0) + return vertices.coGroup(edgesWithNeighbors) + .where(0).equalTo(0) .with(new ApplyCoGroupFunctionOnAllNeighbors(neighborsFunction)); default: throw new IllegalArgumentException("Illegal edge direction"); @@ -1163,9 +1181,10 @@ public DataSet reduceOnNeighbors(NeighborsFunctionWithVertexValue the output type @@ -1177,21 +1196,24 @@ public DataSet reduceOnNeighbors(NeighborsFunction neighbor switch (direction) { case IN: // create pairs - DataSet, Vertex>> edgesWithSources = edges.join(this.vertices) - .where(0).equalTo(0).with(new ProjectVertexIdJoin(1)); + DataSet, Vertex>> edgesWithSources = edges + .join(this.vertices).where(0).equalTo(0) + .with(new ProjectVertexIdJoin(1)); return edgesWithSources.groupBy(0).reduceGroup( new ApplyNeighborGroupReduceFunction(neighborsFunction)); case OUT: // create pairs - DataSet, Vertex>> edgesWithTargets = edges.join(this.vertices) - .where(1).equalTo(0).with(new ProjectVertexIdJoin(0)); - return edgesWithTargets.groupBy(0).reduceGroup( - new ApplyNeighborGroupReduceFunction(neighborsFunction)); + DataSet, Vertex>> edgesWithTargets = edges + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectVertexIdJoin(0)); + return edgesWithTargets.groupBy(0).reduceGroup( + new ApplyNeighborGroupReduceFunction(neighborsFunction)); case ALL: // create pairs - DataSet, Vertex>> edgesWithNeighbors = edges.flatMap( - new EmitOneEdgeWithNeighborPerNode()).join(this.vertices) - .where(1).equalTo(0).with(new ProjectEdgeWithNeighbor()); + DataSet, Vertex>> edgesWithNeighbors = edges + .flatMap(new EmitOneEdgeWithNeighborPerNode()) + .join(this.vertices).where(1).equalTo(0) + .with(new ProjectEdgeWithNeighbor()); return edgesWithNeighbors.groupBy(0).reduceGroup( new ApplyNeighborGroupReduceFunction(neighborsFunction)); @@ -1200,125 +1222,118 @@ public DataSet reduceOnNeighbors(NeighborsFunction neighbor } } - private static final class ApplyNeighborGroupReduceFunction & Serializable, - VV extends Serializable, EV extends Serializable, T> implements GroupReduceFunction< - Tuple3, Vertex>, T>, ResultTypeQueryable { - + private static final class ApplyNeighborGroupReduceFunction & Serializable, VV extends Serializable, EV extends Serializable, T> + implements GroupReduceFunction, Vertex>, T>, ResultTypeQueryable { + private NeighborsFunction function; - + public ApplyNeighborGroupReduceFunction(NeighborsFunction fun) { this.function = fun; } - - public void reduce(Iterable, Vertex>> edges, - Collector out) throws Exception { + + public void reduce(Iterable, Vertex>> edges, Collector out) throws Exception { out.collect(function.iterateNeighbors(edges)); - + } @Override public TypeInformation getProducedType() { return TypeExtractor.createTypeInfo(NeighborsFunction.class, function.getClass(), 3, null, null); - } + } } - private static final class ProjectVertexIdJoin & Serializable, - VV extends Serializable, EV extends Serializable> implements FlatJoinFunction, - Vertex, Tuple3, Vertex>> { + private static final class ProjectVertexIdJoin & Serializable, VV extends Serializable, EV extends Serializable> + implements FlatJoinFunction, Vertex, Tuple3, Vertex>> { private int fieldPosition; public ProjectVertexIdJoin(int position) { this.fieldPosition = position; } + @SuppressWarnings("unchecked") - public void join(Edge edge, Vertex otherVertex, + public void join(Edge edge, Vertex otherVertex, Collector, Vertex>> out) { - out.collect(new Tuple3, Vertex>( - (K)edge.getField(fieldPosition), edge, otherVertex)); + out.collect(new Tuple3, Vertex>((K) edge.getField(fieldPosition), edge, otherVertex)); } } - private static final class ProjectEdgeWithNeighbor & Serializable, - VV extends Serializable, EV extends Serializable> implements - FlatJoinFunction>, Vertex, Tuple3, Vertex>> { + private static final class ProjectEdgeWithNeighbor & Serializable, VV extends Serializable, EV extends Serializable> + implements FlatJoinFunction>, Vertex, Tuple3, Vertex>> { public void join(Tuple3> keysWithEdge, Vertex neighbor, Collector, Vertex>> out) { - out.collect(new Tuple3, Vertex>(keysWithEdge.f0, - keysWithEdge.f2, neighbor)); + + out.collect(new Tuple3, Vertex>(keysWithEdge.f0, keysWithEdge.f2, neighbor)); } } - private static final class ApplyNeighborCoGroupFunction & Serializable, - VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple2, Vertex>, T>, - ResultTypeQueryable { - + private static final class ApplyNeighborCoGroupFunction & Serializable, VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple2, Vertex>, T>, ResultTypeQueryable { + private NeighborsFunctionWithVertexValue function; - - public ApplyNeighborCoGroupFunction (NeighborsFunctionWithVertexValue fun) { + + public ApplyNeighborCoGroupFunction(NeighborsFunctionWithVertexValue fun) { this.function = fun; } - public void coGroup(Iterable> vertex, - Iterable, Vertex>> neighbors, Collector out) throws Exception { - out.collect(function.iterateNeighbors(vertex.iterator().next(), neighbors)); + + public void coGroup(Iterable> vertex, Iterable, Vertex>> neighbors, + Collector out) throws Exception { + out.collect(function.iterateNeighbors(vertex.iterator().next(), neighbors)); } + @Override public TypeInformation getProducedType() { - return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, - function.getClass(), 3, null, null); + return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null); } } - private static final class ApplyCoGroupFunctionOnAllNeighbors & Serializable, - VV extends Serializable, EV extends Serializable, T> - implements CoGroupFunction, Tuple3, Vertex>, T>, - ResultTypeQueryable { + private static final class ApplyCoGroupFunctionOnAllNeighbors & Serializable, VV extends Serializable, EV extends Serializable, T> + implements CoGroupFunction, Tuple3, Vertex>, T>, ResultTypeQueryable { private NeighborsFunctionWithVertexValue function; - - public ApplyCoGroupFunctionOnAllNeighbors (NeighborsFunctionWithVertexValue fun) { + + public ApplyCoGroupFunctionOnAllNeighbors(NeighborsFunctionWithVertexValue fun) { this.function = fun; } - public void coGroup(Iterable> vertex, final Iterable, Vertex>> keysWithNeighbors, + public void coGroup(Iterable> vertex, + final Iterable, Vertex>> keysWithNeighbors, Collector out) throws Exception { - + final Iterator, Vertex>> neighborsIterator = new Iterator, Vertex>>() { - - final Iterator, Vertex>> keysWithEdgesIterator = - keysWithNeighbors.iterator(); - + + final Iterator, Vertex>> keysWithEdgesIterator = keysWithNeighbors.iterator(); + @Override public boolean hasNext() { return keysWithEdgesIterator.hasNext(); } - + @Override public Tuple2, Vertex> next() { - Tuple3, Vertex> next = keysWithEdgesIterator.next(); + Tuple3, Vertex> next = keysWithEdgesIterator.next(); return new Tuple2, Vertex>(next.f1, next.f2); } @Override public void remove() { keysWithEdgesIterator.remove(); - } + } }; - + Iterable, Vertex>> neighborsIterable = new Iterable, Vertex>>() { public Iterator, Vertex>> iterator() { return neighborsIterator; } }; - - out.collect(function.iterateNeighbors(vertex.iterator().next(), neighborsIterable)); - } + + out.collect(function.iterateNeighbors(vertex.iterator().next(), + neighborsIterable)); + } @Override public TypeInformation getProducedType() { - return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, - function.getClass(), 3, null, null); + return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null); } } -} +} \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java index 27df4e8dd8f7f..526b5a4b1ae7f 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/GraphAlgorithm.java @@ -25,8 +25,7 @@ * @param vertex value type * @param edge value type */ -public interface GraphAlgorithm & Serializable, VV extends Serializable, - EV extends Serializable> { +public interface GraphAlgorithm & Serializable, VV extends Serializable, EV extends Serializable> { - public Graph run (Graph input); + public Graph run(Graph input); } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java index 67c60c94a162b..50604620b79ad 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/LabelPropagationExample.java @@ -19,70 +19,76 @@ package org.apache.flink.gelly.example; import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.*; +import org.apache.flink.api.common.functions.FlatMapFunction; +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.gelly.*; +import org.apache.flink.gelly.Edge; +import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.Vertex; import org.apache.flink.gelly.library.LabelPropagation; import org.apache.flink.types.NullValue; import org.apache.flink.util.Collector; /** - * This example uses the label propagation algorithm to detect communities by propagating labels. - * Initially, each vertex is assigned its id as its label. - * The vertices iteratively propagate their labels to their neighbors and adopt the most frequent label - * among their neighbors. - * The algorithm converges when no vertex changes value or the maximum number of iterations have been reached. + * This example uses the label propagation algorithm to detect communities by + * propagating labels. Initially, each vertex is assigned its id as its label. + * The vertices iteratively propagate their labels to their neighbors and adopt + * the most frequent label among their neighbors. The algorithm converges when + * no vertex changes value or the maximum number of iterations have been + * reached. */ public class LabelPropagationExample implements ProgramDescription { - public static void main (String [] args) throws Exception { + public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> vertices = getVertexDataSet(env); - DataSet> edges = getEdgeDataSet(env); + DataSet> vertices = getVertexDataSet(env); + DataSet> edges = getEdgeDataSet(env); - Graph graph = Graph.fromDataSet(vertices, edges, env); + Graph graph = Graph.fromDataSet(vertices, edges, env); - DataSet> verticesWithCommunity = - graph.run(new LabelPropagation(maxIterations)).getVertices(); + DataSet> verticesWithCommunity = graph.run( + new LabelPropagation(maxIterations)).getVertices(); - verticesWithCommunity.print(); + verticesWithCommunity.print(); - env.execute(); - } + env.execute(); + } - @Override - public String getDescription() { - return "Label Propagation Example"; - } + @Override + public String getDescription() { + return "Label Propagation Example"; + } - private static long numVertices = 100; - private static int maxIterations = 20; + private static long numVertices = 100; + private static int maxIterations = 20; @SuppressWarnings("serial") private static DataSet> getVertexDataSet(ExecutionEnvironment env) { - return env.generateSequence(1, numVertices) - .map(new MapFunction>() { - public Vertex map(Long l) throws Exception { - return new Vertex(l, l); - } - }); - } + return env.generateSequence(1, numVertices).map( + new MapFunction>() { + public Vertex map(Long l) throws Exception { + return new Vertex(l, l); + } + }); + } - @SuppressWarnings("serial") + @SuppressWarnings("serial") private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { - return env.generateSequence(1, numVertices) - .flatMap(new FlatMapFunction>() { - @Override - public void flatMap(Long key, Collector> out) { - int numOutEdges = (int) (Math.random() * (numVertices / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numVertices) + 1; - out.collect(new Edge(key, target, NullValue.getInstance())); - } - } - }); - } + return env.generateSequence(1, numVertices).flatMap( + new FlatMapFunction>() { + @Override + public void flatMap(Long key, + Collector> out) { + int numOutEdges = (int) (Math.random() * (numVertices / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numVertices) + 1; + out.collect(new Edge(key, target, + NullValue.getInstance())); + } + } + }); + } } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java index 4eb749f82f3a8..b97681160b610 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/MusicProfiles.java @@ -45,161 +45,172 @@ public class MusicProfiles implements ProgramDescription { /** - * This example demonstrates how to mix the "record" Flink API with the graph API. - * The input is a set triplets and a set of - * bad records,i.e. song ids that should not be trusted. - * Initially, we use the record API to filter out the bad records. - * Then, we use the graph API to create a user -> song weighted bipartite graph - * and compute the top song (most listened) per user. - * Then, we use the record API again, to create a user-user similarity graph, - * based on common songs, where two users that listen to the same song are connected. - * Finally, we use the graph API to run the label propagation community detection algorithm - * on the similarity graph. + * This example demonstrates how to mix the "record" Flink API with the + * graph API. The input is a set triplets and + * a set of bad records,i.e. song ids that should not be trusted. Initially, + * we use the record API to filter out the bad records. Then, we use the + * graph API to create a user -> song weighted bipartite graph and compute + * the top song (most listened) per user. Then, we use the record API again, + * to create a user-user similarity graph, based on common songs, where two + * users that listen to the same song are connected. Finally, we use the + * graph API to run the label propagation community detection algorithm on + * the similarity graph. */ - public static void main (String [] args) throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - final int numIterations = 10; - - /** - * Read the user-song-play triplets - * The format is \t\t - */ - DataSet> triplets = MusicProfilesData.getUserSongTriplets(env); - - /** - * Read the mismatches dataset and extract the songIDs - * The format is "ERROR: song_title" - */ - DataSet> mismatches = MusicProfilesData.getMismatches(env).map(new ExtractMismatchSongIds()); - - /** - * Filter out the mismatches from the triplets dataset - */ - DataSet> validTriplets = triplets.coGroup(mismatches) - .where(1).equalTo(0).with(new FilterOutMismatches()); - - /** - * Create a user -> song weighted bipartite graph - * where the edge weights correspond to play counts - */ - Graph userSongGraph = Graph.fromTupleDataSet(validTriplets, env); - - /** - * Get the top track (most listened) for each user - */ - DataSet> usersWithTopTrack = userSongGraph.reduceOnEdges(new GetTopSongPerUser(), - EdgeDirection.OUT).filter(new FilterSongNodes()); - - usersWithTopTrack.print(); - - /** - * Create a user-user similarity graph, based on common songs, - * i.e. two users that listen to the same song are connected. - * For each song, we create an edge between each pair of its in-neighbors. - */ - DataSet> similarUsers = userSongGraph.getEdges().groupBy(1) - .reduceGroup(new CreateSimilarUserEdges()).distinct(); - - Graph similarUsersGraph = Graph.fromDataSet(similarUsers, - - new MapFunction() { - public Long map(String value) { return 1l; } - - }, env).getUndirected(); - - /** - * Detect user communities using the label propagation library method - */ - - // Initialize each vertex with a unique numeric label - DataSet> idsWithInitialLabels = similarUsersGraph.getVertices() - .reduceGroup(new AssignInitialLabelReducer()); - - // update the vertex values and run the label propagation algorithm - DataSet> verticesWithCommunity = similarUsersGraph.joinWithVertices(idsWithInitialLabels, - new MapFunction, Long>() { - public Long map(Tuple2 value) { return value.f1; } - }) - .run(new LabelPropagation(numIterations)).getVertices(); - - verticesWithCommunity.print(); - - env.execute(); - } + public static void main(String[] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final int numIterations = 10; + + /** + * Read the user-song-play triplets The format is + * \t\t + */ + DataSet> triplets = MusicProfilesData.getUserSongTriplets(env); + + /** + * Read the mismatches dataset and extract the songIDs The format is + * "ERROR: song_title" + */ + DataSet> mismatches = MusicProfilesData.getMismatches(env).map(new ExtractMismatchSongIds()); + + /** + * Filter out the mismatches from the triplets dataset + */ + DataSet> validTriplets = triplets + .coGroup(mismatches).where(1).equalTo(0) + .with(new FilterOutMismatches()); + + /** + * Create a user -> song weighted bipartite graph where the edge weights + * correspond to play counts + */ + Graph userSongGraph = Graph.fromTupleDataSet(validTriplets, env); + + /** + * Get the top track (most listened) for each user + */ + DataSet> usersWithTopTrack = userSongGraph + .reduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT) + .filter(new FilterSongNodes()); + + usersWithTopTrack.print(); + + /** + * Create a user-user similarity graph, based on common songs, i.e. two + * users that listen to the same song are connected. For each song, we + * create an edge between each pair of its in-neighbors. + */ + DataSet> similarUsers = userSongGraph + .getEdges().groupBy(1) + .reduceGroup(new CreateSimilarUserEdges()).distinct(); + + Graph similarUsersGraph = Graph.fromDataSet(similarUsers, + new MapFunction() { + public Long map(String value) { + return 1l; + } + }, env).getUndirected(); + + /** + * Detect user communities using the label propagation library method + */ + + // Initialize each vertex with a unique numeric label + DataSet> idsWithInitialLabels = similarUsersGraph + .getVertices().reduceGroup(new AssignInitialLabelReducer()); + + // update the vertex values and run the label propagation algorithm + DataSet> verticesWithCommunity = similarUsersGraph + .joinWithVertices(idsWithInitialLabels, + new MapFunction, Long>() { + public Long map(Tuple2 value) { + return value.f1; + } + }).run(new LabelPropagation(numIterations)) + .getVertices(); + + verticesWithCommunity.print(); + + env.execute(); + } public static final class ExtractMismatchSongIds implements MapFunction> { + public Tuple1 map(String value) { - String[] tokens = value.split("\\s+"); + String[] tokens = value.split("\\s+"); String songId = tokens[1].substring(1); return new Tuple1(songId); } - } - - public static final class FilterOutMismatches implements CoGroupFunction, - Tuple1, Tuple3> { - public void coGroup( - Iterable> triplets, - Iterable> invalidSongs, - Collector> out) { + } + + public static final class FilterOutMismatches implements CoGroupFunction, + Tuple1, Tuple3> { + + public void coGroup(Iterable> triplets, + Iterable> invalidSongs, Collector> out) { + if (!invalidSongs.iterator().hasNext()) { // this is a valid triplet for (Tuple3 triplet : triplets) { - out.collect(triplet); + out.collect(triplet); } } } - } + } - public static final class FilterSongNodes implements FilterFunction> { + public static final class FilterSongNodes implements FilterFunction> { public boolean filter(Tuple2 value) throws Exception { return !value.f1.equals(""); } - } + } - public static final class GetTopSongPerUser implements EdgesFunctionWithVertexValue - > { - public Tuple2 iterateEdges(Vertex vertex, + public static final class GetTopSongPerUser implements EdgesFunctionWithVertexValue> { + + public Tuple2 iterateEdges(Vertex vertex, Iterable> edges) { + int maxPlaycount = 0; String topSong = ""; - for (Edge edge: edges) { + for (Edge edge : edges) { if (edge.getValue() > maxPlaycount) { maxPlaycount = edge.getValue(); topSong = edge.getTarget(); } } - return new Tuple2 (vertex.getId(), topSong); + return new Tuple2(vertex.getId(), topSong); } - } + } + + public static final class CreateSimilarUserEdges implements GroupReduceFunction, + Edge> { - public static final class CreateSimilarUserEdges implements GroupReduceFunction, - Edge> { public void reduce(Iterable> edges, Collector> out) { List listeners = new ArrayList(); for (Edge edge : edges) { listeners.add(edge.getSource()); } - for (int i=0; i < listeners.size()-1; i++) { - out.collect(new Edge(listeners.get(i), listeners.get(i+1), - NullValue.getInstance())); + for (int i = 0; i < listeners.size() - 1; i++) { + out.collect(new Edge(listeners.get(i), + listeners.get(i + 1), NullValue.getInstance())); } } - } + } - public static final class AssignInitialLabelReducer implements GroupReduceFunction, - Tuple2> { - public void reduce(Iterable> vertices, Collector> out) { + public static final class AssignInitialLabelReducer implements GroupReduceFunction, + Tuple2> { + + public void reduce(Iterable> vertices, Collector> out) { long label = 0; for (Vertex vertex : vertices) { out.collect(new Tuple2(vertex.getId(), label)); label++; } } - } + } @Override public String getDescription() { return "Music Profiles Example"; } -} +} \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java index d6c002c6f78b2..2b55613448e7b 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/PageRankExample.java @@ -19,78 +19,84 @@ package org.apache.flink.gelly.example; import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.api.common.functions.*; +import org.apache.flink.api.common.functions.FlatMapFunction; +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.tuple.Tuple2; -import org.apache.flink.gelly.*; +import org.apache.flink.gelly.Edge; +import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.Vertex; import org.apache.flink.gelly.library.PageRank; import org.apache.flink.util.Collector; public class PageRankExample implements ProgramDescription { - @SuppressWarnings("serial") - public static void main (String [] args) throws Exception { + @SuppressWarnings("serial") + public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> pages = getPagesDataSet(env); + DataSet> pages = getPagesDataSet(env); - DataSet> links = getLinksDataSet(env); + DataSet> links = getLinksDataSet(env); - Graph network = Graph.fromDataSet(pages, links, env); - - DataSet> vertexOutDegrees = network.outDegrees(); - - // assign the transition probabilities as the edge weights - Graph networkWithWeights = network.joinWithEdgesOnSource(vertexOutDegrees, - new MapFunction, Double>() { - public Double map(Tuple2 value) { - return value.f0 / value.f1; + Graph network = Graph.fromDataSet(pages, links, env); + + DataSet> vertexOutDegrees = network.outDegrees(); + + // assign the transition probabilities as the edge weights + Graph networkWithWeights = network + .joinWithEdgesOnSource(vertexOutDegrees, + new MapFunction, Double>() { + public Double map(Tuple2 value) { + return value.f0 / value.f1; + } + }); + + DataSet> pageRanks = networkWithWeights.run( + new PageRank(numPages, DAMPENING_FACTOR, maxIterations)) + .getVertices(); + + pageRanks.print(); + + env.execute(); + } + + @Override + public String getDescription() { + return "PageRank"; + } + + private static final double DAMPENING_FACTOR = 0.85; + private static long numPages = 10; + private static int maxIterations = 10; + + @SuppressWarnings("serial") + private static DataSet> getPagesDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numPages).map( + new MapFunction>() { + @Override + public Vertex map(Long l) throws Exception { + return new Vertex(l, 1.0 / numPages); } }); - DataSet> pageRanks = - networkWithWeights.run(new PageRank(numPages, DAMPENING_FACTOR, maxIterations)).getVertices(); - - pageRanks.print(); - - env.execute(); - } - - @Override - public String getDescription() { - return "PageRank"; - } - - private static final double DAMPENING_FACTOR = 0.85; - private static long numPages = 10; - private static int maxIterations = 10; - - @SuppressWarnings("serial") - private static DataSet> getPagesDataSet(ExecutionEnvironment env) { - return env.generateSequence(1, numPages) - .map(new MapFunction>() { - @Override - public Vertex map(Long l) throws Exception { - return new Vertex(l, 1.0 / numPages); - } - }); - - } - - @SuppressWarnings("serial") - private static DataSet> getLinksDataSet(ExecutionEnvironment env) { - return env.generateSequence(1, numPages) - .flatMap(new FlatMapFunction>() { - @Override - public void flatMap(Long key, Collector> out) throws Exception { - int numOutEdges = (int) (Math.random() * (numPages / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numPages) + 1; - out.collect(new Edge(key, target, 1.0)); - } - } - }); - } + } + + @SuppressWarnings("serial") + private static DataSet> getLinksDataSet(ExecutionEnvironment env) { + return env.generateSequence(1, numPages).flatMap( + new FlatMapFunction>() { + @Override + public void flatMap(Long key, + Collector> out) throws Exception { + int numOutEdges = (int) (Math.random() * (numPages / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numPages) + 1; + out.collect(new Edge(key, target, 1.0)); + } + } + }); + } } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java index 760a4a0e3c638..32a84ca60b05f 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/SingleSourceShortestPathsExample.java @@ -29,30 +29,31 @@ public class SingleSourceShortestPathsExample implements ProgramDescription { - private static int maxIterations = 5; + private static int maxIterations = 5; - public static void main (String [] args) throws Exception { + public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> vertices = ExampleUtils.getLongDoubleVertexData(env); + DataSet> vertices = ExampleUtils.getLongDoubleVertexData(env); - DataSet> edges = ExampleUtils.getLongDoubleEdgeData(env); + DataSet> edges = ExampleUtils.getLongDoubleEdgeData(env); - Long srcVertexId = 1L; + Long srcVertexId = 1L; - Graph graph = Graph.fromDataSet(vertices, edges, env); + Graph graph = Graph.fromDataSet(vertices, edges, env); - DataSet> singleSourceShortestPaths = - graph.run(new SingleSourceShortestPaths(srcVertexId, maxIterations)).getVertices(); + DataSet> singleSourceShortestPaths = graph + .run(new SingleSourceShortestPaths(srcVertexId, + maxIterations)).getVertices(); - singleSourceShortestPaths.print(); + singleSourceShortestPaths.print(); - env.execute(); - } + env.execute(); + } - @Override - public String getDescription() { - return "Single Source Shortest Paths"; - } + @Override + public String getDescription() { + return "Single Source Shortest Paths"; + } } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java index b8c79cc647ea9..a8d0ed0e62d6a 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/example/utils/ExampleUtils.java @@ -40,33 +40,35 @@ public static void printResult(DataSet set, String msg) { set.output(new PrintingOutputFormatWithMessage(msg) { }); } - - public static class PrintingOutputFormatWithMessage implements OutputFormat { + + public static class PrintingOutputFormatWithMessage implements + OutputFormat { private static final long serialVersionUID = 1L; - + private transient PrintStream stream; - + private transient String prefix; - + private String message; - + // -------------------------------------------------------------------------------------------- - + /** * Instantiates a printing output format that prints to standard out. */ - public PrintingOutputFormatWithMessage() {} - + public PrintingOutputFormatWithMessage() { + } + public PrintingOutputFormatWithMessage(String msg) { - this.message = msg; + this.message = msg; } @Override public void open(int taskNumber, int numTasks) { // get the target stream this.stream = System.out; - + // set the prefix to message this.prefix = message + ": "; } @@ -75,8 +77,7 @@ public void open(int taskNumber, int numTasks) { public void writeRecord(T record) { if (this.prefix != null) { this.stream.println(this.prefix + record.toString()); - } - else { + } else { this.stream.println(record.toString()); } } @@ -86,41 +87,46 @@ public void close() { this.stream = null; this.prefix = null; } - + @Override public String toString() { return "Print to System.out"; } @Override - public void configure(Configuration parameters) {} + public void configure(Configuration parameters) { + } } @SuppressWarnings("serial") - public static DataSet> getVertexIds(ExecutionEnvironment env, - final long numVertices) { - return env.generateSequence(1, numVertices) - .map(new MapFunction>() { - public Vertex map(Long l) { - return new Vertex(l, NullValue.getInstance()); - } - }); + public static DataSet> getVertexIds( + ExecutionEnvironment env, final long numVertices) { + return env.generateSequence(1, numVertices).map( + new MapFunction>() { + public Vertex map(Long l) { + return new Vertex(l, NullValue + .getInstance()); + } + }); } @SuppressWarnings("serial") - public static DataSet> getRandomEdges(ExecutionEnvironment env, - final long numVertices) { - return env.generateSequence(1, numVertices) - .flatMap(new FlatMapFunction>() { - @Override - public void flatMap(Long key, Collector> out) throws Exception { - int numOutEdges = (int) (Math.random() * (numVertices / 2)); - for (int i = 0; i < numOutEdges; i++) { - long target = (long) (Math.random() * numVertices) + 1; - out.collect(new Edge(key, target, NullValue.getInstance())); - } - } - }); + public static DataSet> getRandomEdges( + ExecutionEnvironment env, final long numVertices) { + return env.generateSequence(1, numVertices).flatMap( + new FlatMapFunction>() { + @Override + public void flatMap(Long key, + Collector> out) + throws Exception { + int numOutEdges = (int) (Math.random() * (numVertices / 2)); + for (int i = 0; i < numOutEdges; i++) { + long target = (long) (Math.random() * numVertices) + 1; + out.collect(new Edge(key, target, + NullValue.getInstance())); + } + } + }); } public static final DataSet> getLongDoubleVertexData( @@ -134,7 +140,7 @@ public static final DataSet> getLongDoubleVertexData( return env.fromCollection(vertices); } - + public static final DataSet> getLongDoubleEdgeData( ExecutionEnvironment env) { List> edges = new ArrayList>(); @@ -145,8 +151,7 @@ public static final DataSet> getLongDoubleEdgeData( edges.add(new Edge(3L, 5L, 35.0)); edges.add(new Edge(4L, 5L, 45.0)); edges.add(new Edge(5L, 1L, 51.0)); - + return env.fromCollection(edges); } } - diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java index cbcbb208c9e9a..8291b0363683c 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/LabelPropagation.java @@ -18,7 +18,8 @@ package org.apache.flink.gelly.library; -import org.apache.flink.gelly.*; +import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.GraphAlgorithm; import org.apache.flink.gelly.spargel.MessageIterator; import org.apache.flink.gelly.spargel.MessagingFunction; import org.apache.flink.gelly.spargel.VertexUpdateFunction; @@ -30,43 +31,43 @@ import java.util.Map.Entry; /** - * An implementation of the label propagation algorithm. - * The iterative algorithm detects communities by propagating labels. - * In each iteration, a vertex adopts the label that is most frequent among its neighbors' labels. - * Labels are represented by Longs and we assume a total ordering among them, in order to break ties. - * The algorithm converges when no vertex changes its value or the maximum number of iterations have been reached. - * Note that different initializations might lead to different results. - * + * An implementation of the label propagation algorithm. The iterative algorithm + * detects communities by propagating labels. In each iteration, a vertex adopts + * the label that is most frequent among its neighbors' labels. Labels are + * represented by Longs and we assume a total ordering among them, in order to + * break ties. The algorithm converges when no vertex changes its value or the + * maximum number of iterations have been reached. Note that different + * initializations might lead to different results. + * */ @SuppressWarnings("serial") -public class LabelPropagation & Serializable> implements GraphAlgorithm { +public class LabelPropagation & Serializable> + implements GraphAlgorithm { - private final int maxIterations; + private final int maxIterations; - public LabelPropagation(int maxIterations) { - this.maxIterations = maxIterations; - } + public LabelPropagation(int maxIterations) { + this.maxIterations = maxIterations; + } - @Override - public Graph run(Graph input) { + @Override + public Graph run(Graph input) { - // iteratively adopt the most frequent label among the neighbors - // of each vertex - return input.runVertexCentricIteration( - new UpdateVertexLabel(), - new SendNewLabelToNeighbors(), - maxIterations - ); - } + // iteratively adopt the most frequent label among the neighbors + // of each vertex + return input.runVertexCentricIteration(new UpdateVertexLabel(), + new SendNewLabelToNeighbors(), maxIterations); + } - /** - * Function that updates the value of a vertex by adopting the most frequent label - * among its in-neighbors - */ - public static final class UpdateVertexLabel & Serializable> - extends VertexUpdateFunction { + /** + * Function that updates the value of a vertex by adopting the most frequent + * label among its in-neighbors + */ + public static final class UpdateVertexLabel & Serializable> + extends VertexUpdateFunction { - public void updateVertex(K vertexKey, Long vertexValue, MessageIterator inMessages) { + public void updateVertex(K vertexKey, Long vertexValue, + MessageIterator inMessages) { Map labelsWithFrequencies = new HashMap(); long maxFrequency = 1; @@ -77,12 +78,12 @@ public void updateVertex(K vertexKey, Long vertexValue, MessageIterator in if (labelsWithFrequencies.containsKey(msg)) { long currentFreq = labelsWithFrequencies.get(msg); labelsWithFrequencies.put(msg, currentFreq + 1); - } - else { + } else { labelsWithFrequencies.put(msg, 1L); } } - // select the most frequent label: if two or more labels have the same frequency, + // select the most frequent label: if two or more labels have the + // same frequency, // the node adopts the label with the highest value for (Entry entry : labelsWithFrequencies.entrySet()) { if (entry.getValue() == maxFrequency) { @@ -90,8 +91,7 @@ public void updateVertex(K vertexKey, Long vertexValue, MessageIterator in if (entry.getKey() > mostFrequentLabel) { mostFrequentLabel = entry.getKey(); } - } - else if (entry.getValue() > maxFrequency) { + } else if (entry.getValue() > maxFrequency) { maxFrequency = entry.getValue(); mostFrequentLabel = entry.getKey(); } @@ -100,16 +100,16 @@ else if (entry.getValue() > maxFrequency) { // set the new vertex value setNewVertexValue(mostFrequentLabel); } - } + } - /** - * Sends the vertex label to all out-neighbors - */ - public static final class SendNewLabelToNeighbors & Serializable> - extends MessagingFunction { + /** + * Sends the vertex label to all out-neighbors + */ + public static final class SendNewLabelToNeighbors & Serializable> + extends MessagingFunction { - public void sendMessages(K vertexKey, Long newLabel) { - sendMessageToAllNeighbors(newLabel); - } - } + public void sendMessages(K vertexKey, Long newLabel) { + sendMessageToAllNeighbors(newLabel); + } + } } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java index 2408d63fbf34c..be6b370e009d0 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/PageRank.java @@ -27,68 +27,69 @@ import org.apache.flink.gelly.spargel.MessagingFunction; import org.apache.flink.gelly.spargel.VertexUpdateFunction; -public class PageRank & Serializable> implements GraphAlgorithm { +public class PageRank & Serializable> implements + GraphAlgorithm { - private long numVertices; - private double beta; - private int maxIterations; + private long numVertices; + private double beta; + private int maxIterations; - public PageRank(long numVertices, double beta, int maxIterations) { - this.numVertices = numVertices; - this.beta = beta; - this.maxIterations = maxIterations; - } + public PageRank(long numVertices, double beta, int maxIterations) { + this.numVertices = numVertices; + this.beta = beta; + this.maxIterations = maxIterations; + } - @Override - public Graph run(Graph network) { - return network.runVertexCentricIteration( - new VertexRankUpdater(numVertices, beta), - new RankMessenger(), - maxIterations - ); - } + @Override + public Graph run(Graph network) { + return network.runVertexCentricIteration(new VertexRankUpdater( + numVertices, beta), new RankMessenger(), maxIterations); + } + /** + * Function that updates the rank of a vertex by summing up the partial + * ranks from all incoming messages and then applying the dampening formula. + */ + @SuppressWarnings("serial") + public static final class VertexRankUpdater & Serializable> + extends VertexUpdateFunction { - /** - * Function that updates the rank of a vertex by summing up the partial ranks from all incoming messages - * and then applying the dampening formula. - */ - @SuppressWarnings("serial") - public static final class VertexRankUpdater & Serializable> extends VertexUpdateFunction { + private final long numVertices; + private final double beta; - private final long numVertices; - private final double beta; + public VertexRankUpdater(long numVertices, double beta) { + this.numVertices = numVertices; + this.beta = beta; + } - public VertexRankUpdater(long numVertices, double beta) { - this.numVertices = numVertices; - this.beta = beta; - } + @Override + public void updateVertex(K vertexKey, Double vertexValue, + MessageIterator inMessages) { + double rankSum = 0.0; + for (double msg : inMessages) { + rankSum += msg; + } - @Override - public void updateVertex(K vertexKey, Double vertexValue, MessageIterator inMessages) { - double rankSum = 0.0; - for (double msg : inMessages) { - rankSum += msg; - } + // apply the dampening factor / random jump + double newRank = (beta * rankSum) + (1 - beta) / numVertices; + setNewVertexValue(newRank); + } + } - // apply the dampening factor / random jump - double newRank = (beta * rankSum) + (1-beta)/numVertices; - setNewVertexValue(newRank); - } - } + /** + * Distributes the rank of a vertex among all target vertices according to + * the transition probability, which is associated with an edge as the edge + * value. + */ + @SuppressWarnings("serial") + public static final class RankMessenger & Serializable> + extends MessagingFunction { - /** - * Distributes the rank of a vertex among all target vertices according to the transition probability, - * which is associated with an edge as the edge value. - */ - @SuppressWarnings("serial") - public static final class RankMessenger & Serializable> extends MessagingFunction { - - @Override - public void sendMessages(K vertexId, Double newRank) { - for (Edge edge : getOutgoingEdges()) { - sendMessageTo(edge.getTarget(), newRank * edge.getValue()); - } - } - } + @Override + public void sendMessages(K vertexId, Double newRank) { + for (Edge edge : getOutgoingEdges()) { + sendMessageTo(edge.getTarget(), newRank * edge.getValue()); + } + } + } } diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java index d4e41bb377d96..abd34f5c92d9c 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/library/SingleSourceShortestPaths.java @@ -19,7 +19,10 @@ package org.apache.flink.gelly.library; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.gelly.*; +import org.apache.flink.gelly.Edge; +import org.apache.flink.gelly.Graph; +import org.apache.flink.gelly.GraphAlgorithm; +import org.apache.flink.gelly.Vertex; import org.apache.flink.gelly.spargel.MessageIterator; import org.apache.flink.gelly.spargel.MessagingFunction; import org.apache.flink.gelly.spargel.VertexUpdateFunction; @@ -27,85 +30,85 @@ import java.io.Serializable; @SuppressWarnings("serial") -public class SingleSourceShortestPaths & Serializable> implements GraphAlgorithm { +public class SingleSourceShortestPaths & Serializable> + implements GraphAlgorithm { - private final K srcVertexId; - private final Integer maxIterations; + private final K srcVertexId; + private final Integer maxIterations; - public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { - this.srcVertexId = srcVertexId; - this.maxIterations = maxIterations; - } + public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) { + this.srcVertexId = srcVertexId; + this.maxIterations = maxIterations; + } - @Override - public Graph run(Graph input) { + @Override + public Graph run(Graph input) { - return input.mapVertices(new InitVerticesMapper(srcVertexId)) - .runVertexCentricIteration( - new VertexDistanceUpdater(), - new MinDistanceMessenger(), - maxIterations - ); - } + return input.mapVertices(new InitVerticesMapper(srcVertexId)) + .runVertexCentricIteration(new VertexDistanceUpdater(), + new MinDistanceMessenger(), maxIterations); + } - public static final class InitVerticesMapper & Serializable> - implements MapFunction, Double> { + public static final class InitVerticesMapper & Serializable> + implements MapFunction, Double> { - private K srcVertexId; + private K srcVertexId; + + public InitVerticesMapper(K srcId) { + this.srcVertexId = srcId; + } - public InitVerticesMapper(K srcId) { - this.srcVertexId = srcId; - } - public Double map(Vertex value) { if (value.f0.equals(srcVertexId)) { return 0.0; - } - else { + } else { return Double.MAX_VALUE; } } - } - - /** - * Function that updates the value of a vertex by picking the minimum distance from all incoming messages. - * - * @param - */ - public static final class VertexDistanceUpdater & Serializable> - extends VertexUpdateFunction { - - @Override - public void updateVertex(K vertexKey, Double vertexValue, MessageIterator inMessages) { - - Double minDistance = Double.MAX_VALUE; - - for (double msg : inMessages) { - if (msg < minDistance) { - minDistance = msg; - } - } - - if (vertexValue > minDistance) { - setNewVertexValue(minDistance); - } - } - } - - /** - * Distributes the minimum distance associated with a given vertex among all the target vertices - * summed up with the edge's value. - * - * @param - */ - public static final class MinDistanceMessenger & Serializable> - extends MessagingFunction { - - @Override - public void sendMessages(K vertexKey, Double newDistance) throws Exception { - for (Edge edge : getOutgoingEdges()) { - sendMessageTo(edge.getTarget(), newDistance + edge.getValue()); - } - } - } + } + + /** + * Function that updates the value of a vertex by picking the minimum + * distance from all incoming messages. + * + * @param + */ + public static final class VertexDistanceUpdater & Serializable> + extends VertexUpdateFunction { + + @Override + public void updateVertex(K vertexKey, Double vertexValue, + MessageIterator inMessages) { + + Double minDistance = Double.MAX_VALUE; + + for (double msg : inMessages) { + if (msg < minDistance) { + minDistance = msg; + } + } + + if (vertexValue > minDistance) { + setNewVertexValue(minDistance); + } + } + } + + /** + * Distributes the minimum distance associated with a given vertex among all + * the target vertices summed up with the edge's value. + * + * @param + */ + public static final class MinDistanceMessenger & Serializable> + extends MessagingFunction { + + @Override + public void sendMessages(K vertexKey, Double newDistance) + throws Exception { + for (Edge edge : getOutgoingEdges()) { + sendMessageTo(edge.getTarget(), newDistance + edge.getValue()); + } + } + } } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java index 563a2ce9f7287..f3681b2f77b14 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/utils/GraphUtils.java @@ -35,24 +35,23 @@ public static DataSet count(DataSet set, ExecutionEnvironment env) { List list = new ArrayList(); list.add(0); DataSet initialCount = env.fromCollection(list); - return set - .map(new OneMapper()) - .union(initialCount) - .reduce(new AddOnesReducer()) - .first(1); - } + return set.map(new OneMapper()).union(initialCount) + .reduce(new AddOnesReducer()).first(1); + } - private static final class OneMapper implements MapFunction { - @Override - public Integer map(T o) throws Exception { - return 1; - } - } - - private static final class AddOnesReducer implements ReduceFunction { - @Override - public Integer reduce(Integer one, Integer two) throws Exception { - return one + two; - } - } + private static final class OneMapper implements + MapFunction { + @Override + public Integer map(T o) throws Exception { + return 1; + } + } + + private static final class AddOnesReducer implements + ReduceFunction { + @Override + public Integer reduce(Integer one, Integer two) throws Exception { + return one + two; + } + } } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java index 08ff5ecf9867f..f5e4707b35d87 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/GraphValidator.java @@ -17,6 +17,7 @@ */ package org.apache.flink.gelly.validation; + import java.io.Serializable; import org.apache.flink.api.java.DataSet; @@ -24,15 +25,15 @@ /** * A validation method for different types of Graphs - * + * * @param * @param * @param */ @SuppressWarnings("serial") -public abstract class GraphValidator & Serializable, VV extends Serializable, - EV extends Serializable> implements Serializable{ +public abstract class GraphValidator & Serializable, VV extends Serializable, EV extends Serializable> + implements Serializable { - public abstract DataSet validate(Graph graph); + public abstract DataSet validate(Graph graph); } \ No newline at end of file diff --git a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java index bd7a6dc450684..d8ecc63d71a8e 100644 --- a/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java +++ b/flink-addons/flink-gelly/src/main/java/org/apache/flink/gelly/validation/InvalidVertexIdsValidator.java @@ -32,63 +32,56 @@ import java.io.Serializable; @SuppressWarnings("serial") -public class InvalidVertexIdsValidator & Serializable, VV extends Serializable, - EV extends Serializable> extends GraphValidator { +public class InvalidVertexIdsValidator & Serializable, VV extends Serializable, EV extends Serializable> + extends GraphValidator { - /** - * Checks that the edge set input contains valid vertex Ids, - * i.e. that they also exist in the vertex input set. - * @return a singleton DataSet stating whether a graph is valid - * with respect to its vertex ids. - */ - @Override - public DataSet validate(Graph graph) { - DataSet> edgeIds = graph.getEdges().flatMap(new MapEdgeIds()).distinct(); - DataSet invalidIds = graph.getVertices().coGroup(edgeIds).where(0).equalTo(0) - .with(new GroupInvalidIds()).first(1); + /** + * Checks that the edge set input contains valid vertex Ids, i.e. that they + * also exist in the vertex input set. + * + * @return a singleton DataSet stating whether a graph is valid + * with respect to its vertex ids. + */ + @Override + public DataSet validate(Graph graph) { + DataSet> edgeIds = graph.getEdges() + .flatMap(new MapEdgeIds()).distinct(); + DataSet invalidIds = graph.getVertices().coGroup(edgeIds).where(0) + .equalTo(0).with(new GroupInvalidIds()).first(1); - return GraphUtils.count(invalidIds.map(new KToTupleMap()), graph.getContext()) - .map(new InvalidIdsMap()); - } + return GraphUtils.count(invalidIds.map(new KToTupleMap()), + graph.getContext()).map(new InvalidIdsMap()); + } - private static final class MapEdgeIds & Serializable, - EV extends Serializable> implements FlatMapFunction, - Tuple1> { + private static final class MapEdgeIds & Serializable, EV extends Serializable> + implements FlatMapFunction, Tuple1> { + public void flatMap(Edge edge, Collector> out) { + out.collect(new Tuple1(edge.f0)); + out.collect(new Tuple1(edge.f1)); + } + } - @Override - public void flatMap(Edge edge, Collector> out) { - out.collect(new Tuple1(edge.f0)); - out.collect(new Tuple1(edge.f1)); - } - } + private static final class GroupInvalidIds & Serializable, VV extends Serializable> + implements CoGroupFunction, Tuple1, K> { + public void coGroup(Iterable> vertexId, + Iterable> edgeId, Collector out) { + if (!(vertexId.iterator().hasNext())) { + // found an id that doesn't exist in the vertex set + out.collect(edgeId.iterator().next().f0); + } + } + } - private static final class GroupInvalidIds & Serializable, - VV extends Serializable> implements CoGroupFunction, Tuple1, K> { + private static final class KToTupleMap implements MapFunction> { + public Tuple1 map(K key) throws Exception { + return new Tuple1(key); + } + } - @Override - public void coGroup(Iterable> vertexId, - Iterable> edgeId, Collector out) { - if (!(vertexId.iterator().hasNext())) { - // found an id that doesn't exist in the vertex set - out.collect(edgeId.iterator().next().f0); - } - } - } - - private static final class KToTupleMap implements MapFunction> { - - @Override - public Tuple1 map (K key)throws Exception { - return new Tuple1(key); - } - } - - private static final class InvalidIdsMap implements MapFunction { - - @Override - public Boolean map (Integer numberOfInvalidIds)throws Exception { - return numberOfInvalidIds == 0; - } - } + private static final class InvalidIdsMap implements MapFunction { + public Boolean map(Integer numberOfInvalidIds) throws Exception { + return numberOfInvalidIds == 0; + } + } } \ No newline at end of file