@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()); } }
/** * 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<K, VV, EV> removeEdge(Edge<K, EV> edge) { DataSet<Edge<K, EV>> newEdges = getEdges().filter(new EdgeRemovalEdgeFilter<>(edge)).name("Remove edge"); return new Graph<>(this.vertices, newEdges, this.context); }
@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 DataSet plan(Graph<K, VV, EV> graph) throws Exception { DataSet<Edge<K, EV>> edges = graph.getEdges(); if (hasNullValueEdges(edges)) { return edges .map(new EdgeToTuple2Map<>()) .name("Edge to Tuple2") .setParallelism(parallelism.getValue().intValue()); } else { return edges; } }
@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()); } }
/** * 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<K, VV, EV> union(Graph<K, VV, EV> graph) { DataSet<Vertex<K, VV>> unionedVertices = graph .getVertices() .union(this.getVertices()) .name("Vertices") .distinct() .name("Vertices"); DataSet<Edge<K, EV>> unionedEdges = graph .getEdges() .union(this.getEdges()) .name("Edges"); return new Graph<>(unionedVertices, unionedEdges, this.context); }
@Override public ChecksumHashCode<K, VV, EV> run(Graph<K, VV, EV> input) throws Exception { super.run(input); vertexChecksum = new org.apache.flink.graph.asm.dataset.ChecksumHashCode<>(); vertexChecksum.run(input.getVertices()); edgeChecksum = new org.apache.flink.graph.asm.dataset.ChecksumHashCode<>(); edgeChecksum.run(input.getEdges()); return this; }
/** * Computes the intersection between the edge set and the given edge set. For all matching pairs, both edges will be * in the resulting data set. * * @param edges edges to compute intersection with * @return edge set containing both edges from all matching pairs of the same edge */ private DataSet<Edge<K, EV>> getPairwiseEdgeIntersection(DataSet<Edge<K, EV>> edges) { return this.getEdges() .coGroup(edges) .where(0, 1, 2) .equalTo(0, 1, 2) .with(new MatchingEdgeReducer<>()) .name("Intersect edges"); }
/** * Removes all the edges that match the edges in the given data set from the graph. * * @param edgesToBeRemoved the list of edges to be removed * @return a new graph where the edges have been removed and in which the vertices remained intact */ public Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved) { DataSet<Edge<K, EV>> newEdges = getEdges().coGroup(this.context.fromCollection(edgesToBeRemoved)) .where(0, 1).equalTo(0, 1).with(new EdgeRemovalCoGroup<>()).name("Remove edges"); return new Graph<>(this.vertices, 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 DataSet<Edge<K, Tuple2<EV, Degrees>>> runInternal(Graph<K, VV, EV> input) throws Exception { // t, d(t) DataSet<Vertex<K, Degrees>> vertexDegrees = input .run(new VertexDegrees<K, VV, EV>() .setParallelism(parallelism)); // s, t, d(t) return input.getEdges() .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) .where(1) .equalTo(0) .with(new JoinEdgeWithVertexDegree<>()) .setParallelism(parallelism) .name("Edge target degrees"); } }
@Override public DataSet<Edge<K, Tuple2<EV, Degrees>>> runInternal(Graph<K, VV, EV> input) throws Exception { // s, d(s) DataSet<Vertex<K, Degrees>> vertexDegrees = input .run(new VertexDegrees<K, VV, EV>() .setParallelism(parallelism)); // s, t, d(s) return input.getEdges() .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) .where(0) .equalTo(0) .with(new JoinEdgeWithVertexDegree<>()) .setParallelism(parallelism) .name("Edge source degrees"); } }
@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()); }
/** * Checks that the edge set input contains valid vertex Ids, i.e. that they * also exist in the vertex input set. * * @return a boolean stating whether a graph is valid * with respect to its vertex ids. */ @Override public boolean validate(Graph<K, VV, EV> graph) throws Exception { DataSet<Tuple1<K>> edgeIds = graph.getEdges() .flatMap(new MapEdgeIds<>()).distinct(); DataSet<K> invalidIds = graph.getVertices().coGroup(edgeIds).where(0) .equalTo(0).with(new GroupInvalidIds<>()).first(1); return invalidIds.map(new KToTupleMap<>()).count() == 0; }
@Override public DataSet<Edge<K, Tuple2<EV, LongValue>>> runInternal(Graph<K, VV, EV> input) throws Exception { // t, d(t) DataSet<Vertex<K, LongValue>> vertexDegrees = input .run(new VertexDegree<K, VV, EV>() .setReduceOnTargetId(!reduceOnSourceId.get()) .setParallelism(parallelism)); // s, t, d(t) return input.getEdges() .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) .where(1) .equalTo(0) .with(new JoinEdgeWithVertexDegree<>()) .setParallelism(parallelism) .name("Edge target degree"); } }
@Override public DataSet<Edge<K, Tuple2<EV, LongValue>>> runInternal(Graph<K, VV, EV> input) throws Exception { // s, d(s) DataSet<Vertex<K, LongValue>> vertexDegrees = input .run(new VertexDegree<K, VV, EV>() .setReduceOnTargetId(reduceOnTargetId.get()) .setParallelism(parallelism)); // s, t, d(s) return input.getEdges() .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) .where(0) .equalTo(0) .with(new JoinEdgeWithVertexDegree<>()) .setParallelism(parallelism) .name("Edge source degree"); } }
/** * This method allows access to the graph's edge values along with its source and target vertex values. * * @return a triplet DataSet consisting of (srcVertexId, trgVertexId, srcVertexValue, trgVertexValue, edgeValue) */ public DataSet<Triplet<K, VV, EV>> getTriplets() { return this.getVertices() .join(this.getEdges()).where(0).equalTo(0) .with(new ProjectEdgeWithSrcValue<>()) .name("Project edge with source value") .join(this.getVertices()).where(1).equalTo(0) .with(new ProjectEdgeWithVertexValues<>()) .name("Project edge with vertex values"); }
/** * Removes the given list of vertices and its edges from the graph. * * @param verticesToBeRemoved the DataSet of vertices to be removed * @return the resulted graph containing the initial vertices and edges minus the vertices * and edges removed. */ private Graph<K, VV, EV> removeVertices(DataSet<Vertex<K, VV>> verticesToBeRemoved) { DataSet<Vertex<K, VV>> newVertices = getVertices().coGroup(verticesToBeRemoved).where(0).equalTo(0) .with(new VerticesRemovalCoGroup<>()).name("Remove vertices"); DataSet <Edge< K, EV>> newEdges = newVertices.join(getEdges()).where(0).equalTo(0) // if the edge source was removed, the edge will also be removed .with(new ProjectEdgeToBeRemoved<>()).name("Edges to be removed") // if the edge target was removed, the edge will also be removed .join(newVertices).where(1).equalTo(0) .with(new ProjectEdge<>()).name("Remove edges"); return new Graph<>(newVertices, newEdges, context); }
@Override public DataSet<Tuple3<K, K, K>> run(Graph<K, VV, EV> input) throws Exception { DataSet<Edge<K, EV>> edges = input.getEdges(); // annotate edges with degrees DataSet<EdgeWithDegrees<K>> edgesWithDegrees = edges.flatMap(new EdgeDuplicator<>()) .groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new DegreeCounter<>()) .groupBy(EdgeWithDegrees.V1, EdgeWithDegrees.V2).reduce(new DegreeJoiner<>()); // project edges by degrees DataSet<Edge<K, NullValue>> edgesByDegree = edgesWithDegrees.map(new EdgeByDegreeProjector<>()); // project edges by vertex id DataSet<Edge<K, NullValue>> edgesById = edgesByDegree.map(new EdgeByIdProjector<>()); DataSet<Tuple3<K, K, K>> triangles = edgesByDegree // build triads .groupBy(EdgeWithDegrees.V1).sortGroup(EdgeWithDegrees.V2, Order.ASCENDING) .reduceGroup(new TriadBuilder<>()) // filter triads .join(edgesById, JoinHint.REPARTITION_HASH_SECOND).where(Triad.V2, Triad.V3).equalTo(0, 1).with(new TriadFilter<>()); return triangles; }