[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
GitHub user vasia opened a pull request: https://github.com/apache/flink/pull/1149 [FLINK-1520] [gelly] Create a Graph from CSV files This builds on @shghatge's work in #847. I addressed the remaining issues, rebased, and edited the docs. @andralungu, you've already reviewed this, but if you could give it one more look, that'd be great :) Thanks! You can merge this pull request into a Git repository by running: $ git pull https://github.com/vasia/flink csvInput Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/1149.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1149 commit 46f52ae64664be39f73af2505e5ded5e9736a867 Author: Shivani Date: 2015-06-17T13:37:36Z [FLINK-1520] [gelly] Read edges and vertices from CSV files commit ab114f39e9f1f21802ca63c8bb186f1015b8f460 Author: Shivani Date: 2015-07-06T13:41:59Z [FLINK-1520][gelly]Changed the methods for specifying types. Created a new file for tests. Made appropriate changes in gelly_guide.md commit 8a0b66489407de9aec84c3b715aded7225772ee4 Author: vasia Date: 2015-07-14T18:46:33Z [FLINK-1520] [gelly] types and formatting changes to the graph csv reader commit 8007acbf06649694429be189bab70aa451cee679 Author: vasia Date: 2015-07-27T13:43:59Z [FLINK-1520] [gelly] added named types methods for reading a Graph from CSV input, with and without vertex/edge values. Changes the examples and the tests accordingly. commit 9d02c2baba817948ff8710d2a2ae2dda752bff48 Author: vasia Date: 2015-09-19T19:18:53Z [FLINK-1520] [gelly] corrections in Javadocs; updated documentation --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user andralungu commented on a diff in the pull request: https://github.com/apache/flink/pull/1149#discussion_r39947900 --- Diff: flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java --- @@ -0,0 +1,486 @@ +/* + * 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.graph; +import com.google.common.base.Preconditions; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.io.CsvReader; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.NullValue; +import org.apache.flink.api.java.ExecutionEnvironment; + +/** + * A class to build a Graph using path(s) provided to CSV file(s) with optional vertex and edge data. + * The class also configures the CSV readers used to read edge and vertex data such as the field types, + * the delimiters (row and field), the fields that should be included or skipped, and other flags, + * such as whether to skip the initial line as the header. + * The configuration is done using the functions provided in the {@link org.apache.flink.api.java.io.CsvReader} class. + */ + +public class GraphCsvReader { + + @SuppressWarnings("unused") + private final Path vertexPath, edgePath; + private final ExecutionEnvironment executionContext; + protected CsvReader edgeReader; + protected CsvReader vertexReader; + protected MapFunction mapper; + protected Class vertexKey; + protected Class vertexValue; + protected Class edgeValue; + +// + public GraphCsvReader(Path vertexPath, Path edgePath, ExecutionEnvironment context) { + this.vertexPath = vertexPath; + this.edgePath = edgePath; + this.vertexReader = new CsvReader(vertexPath, context); + this.edgeReader = new CsvReader(edgePath, context); + this.mapper = null; + this.executionContext = context; + } + + public GraphCsvReader(Path edgePath, ExecutionEnvironment context) { + this.vertexPath = null; + this.edgePath = edgePath; + this.edgeReader = new CsvReader(edgePath, context); + this.vertexReader = null; + this.mapper = null; + this.executionContext = context; + } + + public GraphCsvReader(Path edgePath, final MapFunction mapper, ExecutionEnvironment context) { + this.vertexPath = null; + this.edgePath = edgePath; + this.edgeReader = new CsvReader(edgePath, context); + this.vertexReader = null; + this.mapper = mapper; + this.executionContext = context; + } + + public GraphCsvReader (String edgePath, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context); + + } + + public GraphCsvReader(String vertexPath, String edgePath, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(vertexPath, "The file path may not be null.")), + new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context); + } + + + public GraphCsvReader(String edgePath, final MapFunction mapper, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), mapper, context); + } + + /** +* Creates a Graph from CSV input with vertex values and edge values. +* The vertex values are specified through a vertices input file or a user-defined map function. +* +* @param vertex
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user andralungu commented on a diff in the pull request: https://github.com/apache/flink/pull/1149#discussion_r39948297 --- Diff: flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java --- @@ -110,24 +105,20 @@ public static void main(String [] args) throws Exception { // Emit results if(fileOutput) { resultedVertices.writeAsCsv(outputPath, "\n", ","); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute("Incremental SSSP Example"); } else { resultedVertices.print(); } + env.execute("Incremental SSSP Example"); --- End diff -- I'm not sure whether I am missing something... Why do you add `env.execute()` after `print()`. It's no longer needed. Have a look here: https://github.com/apache/flink/blob/master/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user andralungu commented on the pull request: https://github.com/apache/flink/pull/1149#issuecomment-141905104 Hi @vasia, As you said, I already reviewed this :P. I left a couple of comments inline. Please reverify the forwarded fields annotations. If you put them there for one mapper, add them for the others too. Appart from that, it's good to merge. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user vasia commented on a diff in the pull request: https://github.com/apache/flink/pull/1149#discussion_r39948730 --- Diff: flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java --- @@ -110,24 +105,20 @@ public static void main(String [] args) throws Exception { // Emit results if(fileOutput) { resultedVertices.writeAsCsv(outputPath, "\n", ","); - - // since file sinks are lazy, we trigger the execution explicitly - env.execute("Incremental SSSP Example"); } else { resultedVertices.print(); } + env.execute("Incremental SSSP Example"); --- End diff -- That's result of auto-merge I guess. Thanks for spotting it! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user vasia commented on a diff in the pull request: https://github.com/apache/flink/pull/1149#discussion_r39948692 --- Diff: flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java --- @@ -0,0 +1,486 @@ +/* + * 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.graph; +import com.google.common.base.Preconditions; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.io.CsvReader; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.NullValue; +import org.apache.flink.api.java.ExecutionEnvironment; + +/** + * A class to build a Graph using path(s) provided to CSV file(s) with optional vertex and edge data. + * The class also configures the CSV readers used to read edge and vertex data such as the field types, + * the delimiters (row and field), the fields that should be included or skipped, and other flags, + * such as whether to skip the initial line as the header. + * The configuration is done using the functions provided in the {@link org.apache.flink.api.java.io.CsvReader} class. + */ + +public class GraphCsvReader { + + @SuppressWarnings("unused") + private final Path vertexPath, edgePath; + private final ExecutionEnvironment executionContext; + protected CsvReader edgeReader; + protected CsvReader vertexReader; + protected MapFunction mapper; + protected Class vertexKey; + protected Class vertexValue; + protected Class edgeValue; + +// + public GraphCsvReader(Path vertexPath, Path edgePath, ExecutionEnvironment context) { + this.vertexPath = vertexPath; + this.edgePath = edgePath; + this.vertexReader = new CsvReader(vertexPath, context); + this.edgeReader = new CsvReader(edgePath, context); + this.mapper = null; + this.executionContext = context; + } + + public GraphCsvReader(Path edgePath, ExecutionEnvironment context) { + this.vertexPath = null; + this.edgePath = edgePath; + this.edgeReader = new CsvReader(edgePath, context); + this.vertexReader = null; + this.mapper = null; + this.executionContext = context; + } + + public GraphCsvReader(Path edgePath, final MapFunction mapper, ExecutionEnvironment context) { + this.vertexPath = null; + this.edgePath = edgePath; + this.edgeReader = new CsvReader(edgePath, context); + this.vertexReader = null; + this.mapper = mapper; + this.executionContext = context; + } + + public GraphCsvReader (String edgePath, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context); + + } + + public GraphCsvReader(String vertexPath, String edgePath, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(vertexPath, "The file path may not be null.")), + new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context); + } + + + public GraphCsvReader(String edgePath, final MapFunction mapper, ExecutionEnvironment context) { + this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), mapper, context); + } + + /** +* Creates a Graph from CSV input with vertex values and edge values. +* The vertex values are specified through a vertices input file or a user-defined map function. +* +* @param vertexKey t
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user vasia commented on the pull request: https://github.com/apache/flink/pull/1149#issuecomment-141905875 Thanks @andralungu! I'll address your comments and merge later. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-1520] [gelly] Create a Graph from CSV f...
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/1149 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---