Return-Path: X-Original-To: apmail-flink-issues-archive@minotaur.apache.org Delivered-To: apmail-flink-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 6295A18ED4 for ; Wed, 17 Jun 2015 21:06:04 +0000 (UTC) Received: (qmail 67595 invoked by uid 500); 17 Jun 2015 21:06:04 -0000 Delivered-To: apmail-flink-issues-archive@flink.apache.org Received: (qmail 67550 invoked by uid 500); 17 Jun 2015 21:06:04 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 67541 invoked by uid 99); 17 Jun 2015 21:06:04 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Jun 2015 21:06:04 +0000 X-ASF-Spam-Status: No, hits=-2000.4 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 17 Jun 2015 21:03:52 +0000 Received: (qmail 67215 invoked by uid 99); 17 Jun 2015 21:05:39 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Jun 2015 21:05:39 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5E7B7DFFC0; Wed, 17 Jun 2015 21:05:39 +0000 (UTC) From: vasia To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request: [FLINK-1520]Read edges and vertices from CSV f... Content-Type: text/plain Message-Id: <20150617210539.5E7B7DFFC0@git1-us-west.apache.org> Date: Wed, 17 Jun 2015 21:05:39 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Github user vasia commented on a diff in the pull request: https://github.com/apache/flink/pull/847#discussion_r32674923 --- Diff: flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java --- @@ -0,0 +1,388 @@ +/* + * 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.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Graph; +import org.apache.flink.types.NullValue; +import org.apache.flink.core.fs.Path; + + +/** + * A class to build a Graph using path(s) provided to CSV file(s) with edge (vertices) data + * The class also configures the CSV readers used to read edges(vertices) 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{ + + private final Path path1,path2; + private final ExecutionEnvironment executionContext; + + private Path edgePath; + private Path vertexPath; + protected CsvReader EdgeReader; + protected CsvReader VertexReader; + protected MapFunction mapper; + +//-------------------------------------------------------------------------------------------------------------------- + + public GraphCsvReader(Path path1,Path path2, ExecutionEnvironment context) + { + this.path1 = path1; + this.path2 = path2; + this.VertexReader = new CsvReader(path1,context); + this.EdgeReader = new CsvReader(path2,context); + this.mapper=null; + this.executionContext=context; + } + + public GraphCsvReader(Path path2, ExecutionEnvironment context) + { + this.path1=null; + this.path2 = path2; + this.EdgeReader = new CsvReader(path2,context); + this.VertexReader = null; + this.mapper = null; + this.executionContext=context; + } + + public GraphCsvReader(Path path2,final MapFunction mapper, ExecutionEnvironment context) + { + this.path1=null; + this.path2 = path2; + this.EdgeReader = new CsvReader(path2,context); + this.VertexReader = null; + this.mapper = mapper; + this.executionContext=context; + } + + public GraphCsvReader (String path2,ExecutionEnvironment context) + { + this(new Path(Preconditions.checkNotNull(path2, "The file path may not be null.")), context); + + } + + public GraphCsvReader(String path1, String path2, ExecutionEnvironment context) + { + this(new Path(Preconditions.checkNotNull(path1, "The file path may not be null.")),new Path(Preconditions.checkNotNull(path2, "The file path may not be null.")), context); + } + + + public GraphCsvReader (String path2, final MapFunction mapper, ExecutionEnvironment context) + { + + this(new Path(Preconditions.checkNotNull(path2, "The file path may not be null.")),mapper, context); + + + } + + public CsvReader getEdgeReader() + { + return this.EdgeReader; + } + + public CsvReader getVertexReader() + { + return this.VertexReader; + } + //-------------------------------------------------------------------------------------------------------------------- + + /** + * Specifies the types for the Graph fields and returns a Graph with those field types + * + * This method is overloaded for the case in which Vertices don't have a value + * + * @param type0 The type of CSV field 0 and 1 for the CSV reader used for reading Edge data, the type of CSV field 0 for the CSV reader used for reading Vertex data and the type of Vetex ID in the returned Graph. + * @param type1 The type of CSV field 1 for the CSV reader used for reading Vertex data and the type of Vertex Value in the returned Graph. + * @param type2 The type of CSV field 2 for the CSV reader used for reading Edge data and the type of Edge Value in the returned Graph. + * @return The {@link org.apache.flink.graph.Graph} with Edges and Vertices extracted from the parsed CSV data. + */ + public Graph types(Class type0, Class type1, Class type2) + { + DataSet> edges = this.EdgeReader.types(type0,type0,type2); + if(path1!=null) + { + DataSet> vertices = this.VertexReader.types(type0,type1); + return Graph.fromTupleDataSet(vertices,edges,executionContext); + } + else + { + return Graph.fromTupleDataSet(edges,mapper,executionContext); + } + + + } + /** + * Specifies the types for the Graph fields and returns a Graph with those field types + * + * This method is overloaded for the case in which Vertices don't have a value + * + * @param type0 The type of CSV field 0 and 1 for the CSV reader used for reading Edge data and the type of Vetex ID in the returned Graph. + * @param type1 The type of CSV field 2 for the CSV reader used for reading Edge data and the type of Edge Value in the returned Graph. + * @return The {@link org.apache.flink.graph.Graph} with Edge extracted from the parsed CSV data and Vertices mapped from Edges with null Value. + */ + public Graph types(Class type0, Class type1) + { + DataSet> edges = this.EdgeReader.types(type0,type0,type1); + return Graph.fromTupleDataSet(edges,executionContext); + } + + /** + *Configures the Delimiter that separates rows for the CSV readers used to read the edges and vertices + * ({@code '\n'}) is used by default. + * + *@param delimiter The delimiter that separates the rows. + * @return The GraphCsv reader instance itself, to allow for fluent function chaining. + */ + public GraphCsvReader lineDelimiter(String delimiter) + { + this.EdgeReader.lineDelimiter(delimiter); + this.VertexReader.lineDelimiter(delimiter); + return this; + } + + /** + *Configures the Delimiter that separates fields in a row for the CSV readers used to read the edges and vertices + * ({@code ','}) is used by default. + * + * @param delimiter The delimiter that separates the fields in a row. + * @return The GraphCsv reader instance itself, to allow for fluent function chaining. + */ + @Deprecated + public GraphCsvReader fieldDelimiter(char delimiter) + { + this.EdgeReader.fieldDelimiter(delimiter); + this.VertexReader.fieldDelimiter(delimiter); --- End diff -- same --- 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 infrastructure@apache.org or file a JIRA ticket with INFRA. ---