/** * Creates a graph from a Collection of edges. * Vertices are created automatically and their values are set to * NullValue. * * @param edges a Collection of edges. * @param context the flink execution environment. * @return the newly created graph. */ public static <K, EV> Graph<K, NullValue, EV> fromCollection(Collection<Edge<K, EV>> edges, ExecutionEnvironment context) { return fromDataSet(context.fromCollection(edges), context); }
/** * Creates a graph from a Collection of edges. * 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 vertexValueInitializer a map function that initializes the vertex values. * It allows to apply a map transformation on the vertex ID to produce an initial vertex value. * @param context the flink execution environment. * @return the newly created graph. */ public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Edge<K, EV>> edges, final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) { return fromDataSet(context.fromCollection(edges), vertexValueInitializer, 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 edges. * @param context the flink execution environment. * @return the newly created graph. */ public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Vertex<K, VV>> vertices, Collection<Edge<K, EV>> edges, ExecutionEnvironment context) { return fromDataSet(context.fromCollection(vertices), context.fromCollection(edges), context); }
/** * Creates a Graph from CSV input without vertex values or edge values. * @param vertexKey the type of the vertex IDs * @return a Graph where the vertex IDs are read from the edges input file. */ public <K> Graph<K, NullValue, NullValue> keyType(Class<K> vertexKey) { if (edgeReader == null) { throw new RuntimeException("The edge input file cannot be null!"); } DataSet<Edge<K, NullValue>> edges = edgeReader .types(vertexKey, vertexKey) .name(GraphCsvReader.class.getName()) .map(new Tuple2ToEdgeMap<>()) .name("Type conversion"); return Graph.fromDataSet(edges, executionContext); }
/** * Creates a graph from a DataSet of Tuple2 objects for edges. * Each Tuple2 will become one Edge, where the source ID will be the first field of the Tuple2 * and the target ID will be the second field of the Tuple2. * * <p>Edge value types and Vertex values types will be set to NullValue. * * @param edges a DataSet of Tuple2. * @param context the flink execution environment. * @return the newly created graph. */ public static <K> Graph<K, NullValue, NullValue> fromTuple2DataSet(DataSet<Tuple2<K, K>> edges, ExecutionEnvironment context) { DataSet<Edge<K, NullValue>> edgeDataSet = edges .map(new Tuple2ToEdgeMap<>()) .name("To Edge"); return fromDataSet(edgeDataSet, context); }
@Override public Graph<K, VV, NEW> runInternal(Graph<K, VV, OLD> input) throws Exception { DataSet<Edge<K, NEW>> translatedEdges = translateEdgeValues(input.getEdges(), translator, parallelism); return Graph.fromDataSet(input.getVertices(), translatedEdges, input.getContext()); } }
@Override public Graph<K, NEW, EV> runInternal(Graph<K, OLD, EV> input) throws Exception { DataSet<Vertex<K, NEW>> translatedVertices = translateVertexValues(input.getVertices(), translator, parallelism); return Graph.fromDataSet(translatedVertices, input.getEdges(), input.getContext()); } }
@Override public Graph<LongValue, NullValue, NullValue> generate() { Preconditions.checkState(vertexCount >= 0); // Vertices DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount); // Edges DataSource<Edge<LongValue, NullValue>> edges = env .fromCollection(Collections.<Edge<LongValue, NullValue>>emptyList(), TypeInformation.of(new TypeHint<Edge<LongValue, NullValue>>(){})) .setParallelism(parallelism) .name("Empty edge set"); // Graph return Graph.fromDataSet(vertices, edges, env); } }
private static Graph<Long, Double, Double> getGraph(ExecutionEnvironment env) { if (fileOutput) { return Graph.fromCsvReader(verticesInputPath, edgesInputPath, env).lineDelimiterEdges("\n") .types(Long.class, Double.class, Double.class); } else { return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgeDataSet(env), env); } }
@Override public Graph<NEW, VV, EV> runInternal(Graph<OLD, VV, EV> input) throws Exception { // Vertices DataSet<Vertex<NEW, VV>> translatedVertices = translateVertexIds(input.getVertices(), translator, parallelism); // Edges DataSet<Edge<NEW, EV>> translatedEdges = translateEdgeIds(input.getEdges(), translator, parallelism); // Graph return Graph.fromDataSet(translatedVertices, translatedEdges, input.getContext()); } }
private static Graph<Long, Double, Double> getSSSPGraph(ExecutionEnvironment env) { if (fileOutput) { return Graph.fromCsvReader(verticesInputPath, edgesInSSSPInputPath, env).lineDelimiterEdges("\n") .types(Long.class, Double.class, Double.class); } else { return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgesInSSSP(env), env); } }
/** * Convert a bipartite graph into an undirected graph that contains only bottom vertices. An edge between two * vertices in the new graph will exist only if the original bipartite graph contains a top vertex they are both * connected to. * * <p>The simple projection performs a single join and returns edges containing the bipartite edge values. * * <p>Note: KB must override .equals(). This requirement may be removed in a future release. * * @return simple bottom projection of the bipartite graph */ public Graph<KB, VVB, Tuple2<EV, EV>> projectionBottomSimple() { DataSet<Edge<KB, Tuple2<EV, EV>>> newEdges = edges.join(edges) .where(0) .equalTo(0) .with(new ProjectionBottomSimple<>()) .name("Simple bottom projection"); return Graph.fromDataSet(bottomVertices, newEdges, context); }
@Override public Graph<LongValue, NullValue, NullValue> generate() { Preconditions.checkState(!dimensions.isEmpty(), "No dimensions added to GridGraph"); // Vertices DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount); // Edges LongValueSequenceIterator iterator = new LongValueSequenceIterator(0, this.vertexCount - 1); DataSet<Edge<LongValue, NullValue>> edges = env .fromParallelCollection(iterator, LongValue.class) .setParallelism(parallelism) .name("Edge iterators") .flatMap(new LinkVertexToNeighbors(vertexCount, dimensions)) .setParallelism(parallelism) .name("Grid graph edges"); // Graph return Graph.fromDataSet(vertices, edges, env); }
/** * Convert a bipartite graph into a graph that contains only bottom vertices. An edge between two vertices in the * new graph will exist only if the original bipartite graph contains at least one top vertex they both connect to. * * <p>The full projection performs three joins and returns edges containing the connecting vertex ID and value, * both bottom vertex values, and both bipartite edge values. * * <p>Note: KB must override .equals(). This requirement may be removed in a future release. * * @return full bottom projection of the bipartite graph */ public Graph<KB, VVB, Projection<KT, VVT, VVB, EV>> projectionBottomFull() { DataSet<Tuple5<KT, KB, EV, VVT, VVB>> edgesWithVertices = joinEdgeWithVertices(); DataSet<Edge<KB, Projection<KT, VVT, VVB, EV>>> newEdges = edgesWithVertices.join(edgesWithVertices) .where(0) .equalTo(0) .with(new ProjectionBottomFull<>()) .name("Full bottom projection"); return Graph.fromDataSet(bottomVertices, newEdges, context); }
@Override public Graph<K, Long, Double> run(Graph<K, Long, Double> graph) { DataSet<Vertex<K, Tuple2<Long, Double>>> initializedVertices = graph.getVertices() .map(new AddScoreToVertexValuesMapper<>()); Graph<K, Tuple2<Long, Double>, Double> graphWithScoredVertices = Graph.fromDataSet(initializedVertices, graph.getEdges(), graph.getContext()).getUndirected(); return graphWithScoredVertices.runScatterGatherIteration(new LabelMessenger<>(), new VertexLabelUpdater<>(delta), maxIterations) .mapVertices(new RemoveScoreFromVertexValuesMapper<>()); }
@Override public Graph<LongValue, NullValue, NullValue> generate() { Preconditions.checkState(vertexPairCount > 0); // Vertices long vertexCount = 2 * vertexPairCount; DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount); // Edges LongValueSequenceIterator iterator = new LongValueSequenceIterator(0, vertexCount - 1); DataSet<Edge<LongValue, NullValue>> edges = env .fromParallelCollection(iterator, LongValue.class) .setParallelism(parallelism) .name("Edge iterators") .map(new LinkVertexToSingletonNeighbor()) .setParallelism(parallelism) .name("Complete graph edges"); // Graph return Graph.fromDataSet(vertices, edges, env); }
@Override public Graph<LongValue, NullValue, NullValue> generate() { Preconditions.checkState(vertexCount >= 2); // Vertices DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount); // Edges LongValueSequenceIterator iterator = new LongValueSequenceIterator(1, this.vertexCount - 1); DataSet<Edge<LongValue, NullValue>> edges = env .fromParallelCollection(iterator, LongValue.class) .setParallelism(parallelism) .name("Edge iterators") .flatMap(new LinkVertexToCenter()) .setParallelism(parallelism) .name("Star graph edges"); // Graph return Graph.fromDataSet(vertices, edges, env); }
@Override public Graph<K, VV, EV> runInternal(Graph<K, VV, EV> input) throws Exception { // Edges DataSet<Edge<K, EV>> edges = input .getEdges() .flatMap(new SymmetrizeAndRemoveSelfLoops<>(clipAndFlip)) .setParallelism(parallelism) .name("Remove self-loops") .distinct(0, 1) .setCombineHint(CombineHint.NONE) .setParallelism(parallelism) .name("Remove duplicate edges"); // Graph return Graph.fromDataSet(input.getVertices(), edges, input.getContext()); }
@Override public Graph<K, VV, EV> runInternal(Graph<K, VV, EV> input) throws Exception { // Edges DataSet<Edge<K, EV>> edges = input .getEdges() .filter(new RemoveSelfLoops<>()) .setParallelism(parallelism) .name("Remove self-loops") .distinct(0, 1) .setCombineHint(CombineHint.NONE) .setParallelism(parallelism) .name("Remove duplicate edges"); // Graph return Graph.fromDataSet(input.getVertices(), edges, input.getContext()); }
/** * Adds the given list edges to the graph. * * <p>When adding an edge for a non-existing set of vertices, the edge is considered invalid and ignored. * * @param newEdges the data set of edges to be added * @return a new graph containing the existing edges plus the newly added edges. */ public Graph<K, VV, EV> addEdges(List<Edge<K, EV>> newEdges) { DataSet<Edge<K, EV>> newEdgesDataSet = this.context.fromCollection(newEdges); DataSet<Edge<K, EV>> validNewEdges = this.getVertices().join(newEdgesDataSet) .where(0).equalTo(0) .with(new JoinVerticesWithEdgesOnSrc<>()).name("Join with source") .join(this.getVertices()).where(1).equalTo(0) .with(new JoinWithVerticesOnTrg<>()).name("Join with target"); return Graph.fromDataSet(this.vertices, this.edges.union(validNewEdges), this.context); }