You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Guillermo Ortiz <ko...@gmail.com> on 2016/02/26 12:59:34 UTC

Get all vertexes with outDegree equals to 0 with GraphX

I'm new with graphX. I need to get the vertex without out edges..
I guess that it's pretty easy but I did it pretty complicated.. and
inefficienct

val vertices: RDD[(VertexId, (List[String], List[String]))] =
  sc.parallelize(Array((1L, (List("a"), List[String]())),
    (2L, (List("b"), List[String]())),
    (3L, (List("c"), List[String]())),
    (4L, (List("d"), List[String]())),
    (5L, (List("e"), List[String]())),
    (6L, (List("f"), List[String]()))))

// Create an RDD for edges
val relationships: RDD[Edge[Boolean]] =
  sc.parallelize(Array(Edge(1L, 2L, true), Edge(2L, 3L, true),
Edge(3L, 4L, true), Edge(5L, 2L, true)))

val out = minGraph.outDegrees.map(vertex => vertex._1)

val finalVertexes = minGraph.vertices.keys.subtract(out)

//It must be something better than this way..
val nodes = finalVertexes.collect()
val result = minGraph.vertices.filter(v => nodes.contains(v._1))


What's the good way to do this operation? It seems that it should be
pretty easy.

RE: Get all vertexes with outDegree equals to 0 with GraphX

Posted by Mohammed Guller <mo...@glassbeam.com>.
Perhaps, the documentation of the filter method would help. Here is the method signature (copied from the API doc)

def  filter[VD2, ED2](preprocess: (Graph[VD, ED]) => Graph[VD2, ED2], epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, vpred: (VertexId, VD2) => Boolean = (v: VertexId, d: VD2) => true)
This method returns a subgraph of the original graph. The  data in the original graph remains unchanged. Brief description of the arguments:

VD2:    vertex type the vpred operates on
ED2:    edge type the epred operates on
preprocess:   a function to compute new vertex and edge data before filtering
epred:   edge predicate to filter on after preprocess
vpred:   vertex predicate to filter on after prerocess

In the solution below, the first function literal is the preprocess argument. The vpred argument is passed as named argument since we are using the default value for epred.

HTH.

Mohammed
Author: Big Data Analytics with Spark<http://www.amazon.com/Big-Data-Analytics-Spark-Practitioners/dp/1484209656/>

From: Guillermo Ortiz [mailto:konstt2000@gmail.com]
Sent: Saturday, February 27, 2016 6:17 AM
To: Mohammed Guller
Cc: Robin East; user
Subject: Re: Get all vertexes with outDegree equals to 0 with GraphX

Thank you, I have to think what the code does,, because I am a little noob in scala and it's hard to understand it to me.

2016-02-27 3:53 GMT+01:00 Mohammed Guller <mo...@glassbeam.com>>:
Here is another solution (minGraph is the graph from your code. I assume that is your original graph):

val graphWithNoOutEdges = minGraph.filter(
  graph => graph.outerJoinVertices(graph.outDegrees) {(vId, vData, outDegreesOpt) => outDegreesOpt.getOrElse(0)},
  vpred = (vId: VertexId, vOutDegrees: Int) => vOutDegrees == 0
)

val verticesWithNoOutEdges = graphWithNoOutEdges.vertices

Mohammed
Author: Big Data Analytics with Spark<http://www.amazon.com/Big-Data-Analytics-Spark-Practitioners/dp/1484209656/>

From: Guillermo Ortiz [mailto:konstt2000@gmail.com<ma...@gmail.com>]
Sent: Friday, February 26, 2016 5:46 AM
To: Robin East
Cc: user
Subject: Re: Get all vertexes with outDegree equals to 0 with GraphX

Yes, I am not really happy with that "collect".
I was taking a look to use subgraph method and others options and didn't figure out anything easy or direct..

I'm going to try your idea.

2016-02-26 14:16 GMT+01:00 Robin East <ro...@xense.co.uk>>:
Whilst I can think of other ways to do it I don’t think they would be conceptually or syntactically any simpler. GraphX doesn’t have the concept of built-in vertex properties which would make this simpler - a vertex in GraphX is a Vertex ID (Long) and a bunch of custom attributes that you assign. This means you have to find a way of ‘pushing’ the vertex degree into the graph so you can do comparisons (cf a join in relational databases) or as you have done create a list and filter against that (cf filtering against a sub-query in relational database).

One thing I would point out is that you probably want to avoid finalVerexes.collect() for a large-scale system - this will pull all the vertices into the driver and then push them out to the executors again as part of the filter operation. A better strategy for large graphs would be:

1. build a graph based on the existing graph where the vertex attribute is the vertex degree - the GraphX documentation shows how to do this
2. filter this “degrees” graph to just give you 0 degree vertices
3 use graph.mask passing in the 0-degree graph to get the original graph with just 0 degree vertices

Just one variation on several possibilities, the key point is that everything is just a graph transformation until you call an action on the resulting graph
-------------------------------------------------------------------------------
Robin East
Spark GraphX in Action Michael Malak and Robin East
Manning Publications Co.
http://www.manning.com/books/spark-graphx-in-action




On 26 Feb 2016, at 11:59, Guillermo Ortiz <ko...@gmail.com>> wrote:

I'm new with graphX. I need to get the vertex without out edges..
I guess that it's pretty easy but I did it pretty complicated.. and inefficienct


val vertices: RDD[(VertexId, (List[String], List[String]))] =
  sc.parallelize(Array((1L, (List("a"), List[String]())),
    (2L, (List("b"), List[String]())),
    (3L, (List("c"), List[String]())),
    (4L, (List("d"), List[String]())),
    (5L, (List("e"), List[String]())),
    (6L, (List("f"), List[String]()))))

// Create an RDD for edges
val relationships: RDD[Edge[Boolean]] =
  sc.parallelize(Array(Edge(1L, 2L, true), Edge(2L, 3L, true), Edge(3L, 4L, true), Edge(5L, 2L, true)))

val out = minGraph.outDegrees.map(vertex => vertex._1)

val finalVertexes = minGraph.vertices.keys.subtract(out)

//It must be something better than this way..
val nodes = finalVertexes.collect()
val result = minGraph.vertices.filter(v => nodes.contains(v._1))



What's the good way to do this operation? It seems that it should be pretty easy.




Re: Get all vertexes with outDegree equals to 0 with GraphX

Posted by Guillermo Ortiz <ko...@gmail.com>.
Thank you, I have to think what the code does,, because I am a little noob
in scala and it's hard to understand it to me.

2016-02-27 3:53 GMT+01:00 Mohammed Guller <mo...@glassbeam.com>:

> Here is another solution (minGraph is the graph from your code. I assume
> that is your original graph):
>
>
>
> val graphWithNoOutEdges = minGraph.filter(
>
>   graph => graph.outerJoinVertices(graph.outDegrees) {(vId, vData,
> outDegreesOpt) => outDegreesOpt.getOrElse(0)},
>
>   vpred = (vId: VertexId, vOutDegrees: Int) => vOutDegrees == 0
>
> )
>
>
>
> val verticesWithNoOutEdges = graphWithNoOutEdges.vertices
>
>
>
> Mohammed
>
> Author: Big Data Analytics with Spark
> <http://www.amazon.com/Big-Data-Analytics-Spark-Practitioners/dp/1484209656/>
>
>
>
> *From:* Guillermo Ortiz [mailto:konstt2000@gmail.com]
> *Sent:* Friday, February 26, 2016 5:46 AM
> *To:* Robin East
> *Cc:* user
> *Subject:* Re: Get all vertexes with outDegree equals to 0 with GraphX
>
>
>
> Yes, I am not really happy with that "collect".
>
> I was taking a look to use subgraph method and others options and didn't
> figure out anything easy or direct..
>
>
>
> I'm going to try your idea.
>
>
>
> 2016-02-26 14:16 GMT+01:00 Robin East <ro...@xense.co.uk>:
>
> Whilst I can think of other ways to do it I don’t think they would be
> conceptually or syntactically any simpler. GraphX doesn’t have the concept
> of built-in vertex properties which would make this simpler - a vertex in
> GraphX is a Vertex ID (Long) and a bunch of custom attributes that you
> assign. This means you have to find a way of ‘pushing’ the vertex degree
> into the graph so you can do comparisons (cf a join in relational
> databases) or as you have done create a list and filter against that (cf
> filtering against a sub-query in relational database).
>
>
>
> One thing I would point out is that you probably want to avoid
> finalVerexes.collect() for a large-scale system - this will pull all the
> vertices into the driver and then push them out to the executors again as
> part of the filter operation. A better strategy for large graphs would be:
>
>
>
> 1. build a graph based on the existing graph where the vertex attribute is
> the vertex degree - the GraphX documentation shows how to do this
>
> 2. filter this “degrees” graph to just give you 0 degree vertices
>
> 3 use graph.mask passing in the 0-degree graph to get the original graph
> with just 0 degree vertices
>
>
>
> Just one variation on several possibilities, the key point is that
> everything is just a graph transformation until you call an action on the
> resulting graph
>
>
> -------------------------------------------------------------------------------
>
> Robin East
>
> *Spark GraphX in Action *Michael Malak and Robin East
>
> Manning Publications Co.
>
> http://www.manning.com/books/spark-graphx-in-action
>
>
>
>
>
>
>
>
>
> On 26 Feb 2016, at 11:59, Guillermo Ortiz <ko...@gmail.com> wrote:
>
>
>
> I'm new with graphX. I need to get the vertex without out edges..
>
> I guess that it's pretty easy but I did it pretty complicated.. and
> inefficienct
>
>
>
> *val *vertices: RDD[(VertexId, (List[String], List[String]))] =
>   sc.parallelize(*Array*((1L, (*List*(*"a"*), *List*[String]())),
>     (2L, (*List*(*"b"*), *List*[String]())),
>     (3L, (*List*(*"c"*), *List*[String]())),
>     (4L, (*List*(*"d"*), *List*[String]())),
>     (5L, (*List*(*"e"*), *List*[String]())),
>     (6L, (*List*(*"f"*), *List*[String]()))))
>
>
> *// Create an RDD for edges**val *relationships: RDD[Edge[Boolean]] =
>   sc.parallelize(*Array*(*Edge*(1L, 2L, *true*), *Edge*(2L, 3L, *true*), *Edge*(3L, 4L, *true*), *Edge*(5L, 2L, *true*)))
>
> *val *out = minGraph.*outDegrees*.map(vertex => vertex._1)
>
> *val *finalVertexes = minGraph.vertices.keys.subtract(out)
>
> //It must be something better than this way..
> *val *nodes = finalVertexes.collect()
> *val *result = minGraph.vertices.filter(v => nodes.contains(v._1))
>
>
>
> What's the good way to do this operation? It seems that it should be pretty easy.
>
>
>
>
>

RE: Get all vertexes with outDegree equals to 0 with GraphX

Posted by Mohammed Guller <mo...@glassbeam.com>.
Here is another solution (minGraph is the graph from your code. I assume that is your original graph):

val graphWithNoOutEdges = minGraph.filter(
  graph => graph.outerJoinVertices(graph.outDegrees) {(vId, vData, outDegreesOpt) => outDegreesOpt.getOrElse(0)},
  vpred = (vId: VertexId, vOutDegrees: Int) => vOutDegrees == 0
)

val verticesWithNoOutEdges = graphWithNoOutEdges.vertices

Mohammed
Author: Big Data Analytics with Spark<http://www.amazon.com/Big-Data-Analytics-Spark-Practitioners/dp/1484209656/>

From: Guillermo Ortiz [mailto:konstt2000@gmail.com]
Sent: Friday, February 26, 2016 5:46 AM
To: Robin East
Cc: user
Subject: Re: Get all vertexes with outDegree equals to 0 with GraphX

Yes, I am not really happy with that "collect".
I was taking a look to use subgraph method and others options and didn't figure out anything easy or direct..

I'm going to try your idea.

2016-02-26 14:16 GMT+01:00 Robin East <ro...@xense.co.uk>>:
Whilst I can think of other ways to do it I don’t think they would be conceptually or syntactically any simpler. GraphX doesn’t have the concept of built-in vertex properties which would make this simpler - a vertex in GraphX is a Vertex ID (Long) and a bunch of custom attributes that you assign. This means you have to find a way of ‘pushing’ the vertex degree into the graph so you can do comparisons (cf a join in relational databases) or as you have done create a list and filter against that (cf filtering against a sub-query in relational database).

One thing I would point out is that you probably want to avoid finalVerexes.collect() for a large-scale system - this will pull all the vertices into the driver and then push them out to the executors again as part of the filter operation. A better strategy for large graphs would be:

1. build a graph based on the existing graph where the vertex attribute is the vertex degree - the GraphX documentation shows how to do this
2. filter this “degrees” graph to just give you 0 degree vertices
3 use graph.mask passing in the 0-degree graph to get the original graph with just 0 degree vertices

Just one variation on several possibilities, the key point is that everything is just a graph transformation until you call an action on the resulting graph
-------------------------------------------------------------------------------
Robin East
Spark GraphX in Action Michael Malak and Robin East
Manning Publications Co.
http://www.manning.com/books/spark-graphx-in-action




On 26 Feb 2016, at 11:59, Guillermo Ortiz <ko...@gmail.com>> wrote:

I'm new with graphX. I need to get the vertex without out edges..
I guess that it's pretty easy but I did it pretty complicated.. and inefficienct


val vertices: RDD[(VertexId, (List[String], List[String]))] =
  sc.parallelize(Array((1L, (List("a"), List[String]())),
    (2L, (List("b"), List[String]())),
    (3L, (List("c"), List[String]())),
    (4L, (List("d"), List[String]())),
    (5L, (List("e"), List[String]())),
    (6L, (List("f"), List[String]()))))

// Create an RDD for edges
val relationships: RDD[Edge[Boolean]] =
  sc.parallelize(Array(Edge(1L, 2L, true), Edge(2L, 3L, true), Edge(3L, 4L, true), Edge(5L, 2L, true)))

val out = minGraph.outDegrees.map(vertex => vertex._1)

val finalVertexes = minGraph.vertices.keys.subtract(out)

//It must be something better than this way..
val nodes = finalVertexes.collect()
val result = minGraph.vertices.filter(v => nodes.contains(v._1))



What's the good way to do this operation? It seems that it should be pretty easy.



Re: Get all vertexes with outDegree equals to 0 with GraphX

Posted by Guillermo Ortiz <ko...@gmail.com>.
Yes, I am not really happy with that "collect".
I was taking a look to use subgraph method and others options and didn't
figure out anything easy or direct..

I'm going to try your idea.

2016-02-26 14:16 GMT+01:00 Robin East <ro...@xense.co.uk>:

> Whilst I can think of other ways to do it I don’t think they would be
> conceptually or syntactically any simpler. GraphX doesn’t have the concept
> of built-in vertex properties which would make this simpler - a vertex in
> GraphX is a Vertex ID (Long) and a bunch of custom attributes that you
> assign. This means you have to find a way of ‘pushing’ the vertex degree
> into the graph so you can do comparisons (cf a join in relational
> databases) or as you have done create a list and filter against that (cf
> filtering against a sub-query in relational database).
>
> One thing I would point out is that you probably want to avoid
> finalVerexes.collect() for a large-scale system - this will pull all the
> vertices into the driver and then push them out to the executors again as
> part of the filter operation. A better strategy for large graphs would be:
>
> 1. build a graph based on the existing graph where the vertex attribute is
> the vertex degree - the GraphX documentation shows how to do this
> 2. filter this “degrees” graph to just give you 0 degree vertices
> 3 use graph.mask passing in the 0-degree graph to get the original graph
> with just 0 degree vertices
>
> Just one variation on several possibilities, the key point is that
> everything is just a graph transformation until you call an action on the
> resulting graph
>
> -------------------------------------------------------------------------------
> Robin East
> *Spark GraphX in Action* Michael Malak and Robin East
> Manning Publications Co.
> http://www.manning.com/books/spark-graphx-in-action
>
>
>
>
>
> On 26 Feb 2016, at 11:59, Guillermo Ortiz <ko...@gmail.com> wrote:
>
> I'm new with graphX. I need to get the vertex without out edges..
> I guess that it's pretty easy but I did it pretty complicated.. and
> inefficienct
>
> val vertices: RDD[(VertexId, (List[String], List[String]))] =
>   sc.parallelize(Array((1L, (List("a"), List[String]())),
>     (2L, (List("b"), List[String]())),
>     (3L, (List("c"), List[String]())),
>     (4L, (List("d"), List[String]())),
>     (5L, (List("e"), List[String]())),
>     (6L, (List("f"), List[String]()))))
>
> // Create an RDD for edges
> val relationships: RDD[Edge[Boolean]] =
>   sc.parallelize(Array(Edge(1L, 2L, true), Edge(2L, 3L, true), Edge(3L, 4L, true), Edge(5L, 2L, true)))
>
> val out = minGraph.outDegrees.map(vertex => vertex._1)
>
> val finalVertexes = minGraph.vertices.keys.subtract(out)
>
> //It must be something better than this way..
> val nodes = finalVertexes.collect()
> val result = minGraph.vertices.filter(v => nodes.contains(v._1))
>
>
> What's the good way to do this operation? It seems that it should be pretty easy.
>
>
>

Re: Get all vertexes with outDegree equals to 0 with GraphX

Posted by Robin East <ro...@xense.co.uk>.
Whilst I can think of other ways to do it I don’t think they would be conceptually or syntactically any simpler. GraphX doesn’t have the concept of built-in vertex properties which would make this simpler - a vertex in GraphX is a Vertex ID (Long) and a bunch of custom attributes that you assign. This means you have to find a way of ‘pushing’ the vertex degree into the graph so you can do comparisons (cf a join in relational databases) or as you have done create a list and filter against that (cf filtering against a sub-query in relational database). 

One thing I would point out is that you probably want to avoid finalVerexes.collect() for a large-scale system - this will pull all the vertices into the driver and then push them out to the executors again as part of the filter operation. A better strategy for large graphs would be:

1. build a graph based on the existing graph where the vertex attribute is the vertex degree - the GraphX documentation shows how to do this
2. filter this “degrees” graph to just give you 0 degree vertices
3 use graph.mask passing in the 0-degree graph to get the original graph with just 0 degree vertices

Just one variation on several possibilities, the key point is that everything is just a graph transformation until you call an action on the resulting graph
-------------------------------------------------------------------------------
Robin East
Spark GraphX in Action Michael Malak and Robin East
Manning Publications Co.
http://www.manning.com/books/spark-graphx-in-action <http://www.manning.com/books/spark-graphx-in-action>





> On 26 Feb 2016, at 11:59, Guillermo Ortiz <ko...@gmail.com> wrote:
> 
> I'm new with graphX. I need to get the vertex without out edges..
> I guess that it's pretty easy but I did it pretty complicated.. and inefficienct 
> 
> val vertices: RDD[(VertexId, (List[String], List[String]))] =
>   sc.parallelize(Array((1L, (List("a"), List[String]())),
>     (2L, (List("b"), List[String]())),
>     (3L, (List("c"), List[String]())),
>     (4L, (List("d"), List[String]())),
>     (5L, (List("e"), List[String]())),
>     (6L, (List("f"), List[String]()))))
> 
> // Create an RDD for edges
> val relationships: RDD[Edge[Boolean]] =
>   sc.parallelize(Array(Edge(1L, 2L, true), Edge(2L, 3L, true), Edge(3L, 4L, true), Edge(5L, 2L, true)))
> val out = minGraph.outDegrees.map(vertex => vertex._1)
> val finalVertexes = minGraph.vertices.keys.subtract(out)
> //It must be something better than this way..
> val nodes = finalVertexes.collect()
> val result = minGraph.vertices.filter(v => nodes.contains(v._1))
> 
> What's the good way to do this operation? It seems that it should be pretty easy.