Re: Two joins in GraphX Pregel implementation

2015-07-28 Thread Ankur Dave
On 27 Jul 2015, at 16:42, Ulanov, Alexander alexander.ula...@hp.com wrote: It seems that the mentioned two joins can be rewritten as one outer join You're right. In fact, the outer join can be streamlined further using a method from GraphOps: g = g.joinVertices(messages)(vprog).cache() Then,

Re: GraphX: New graph operator

2015-06-02 Thread Ankur Dave
I think it would be good to have more basic operators like union or difference, as long as they have an efficient distributed implementation and are plausibly useful. If they can be written in terms of the existing GraphX API, it would be best to put them into GraphOps to keep the core GraphX

Re: GraphX implementation of ALS?

2015-05-26 Thread Ankur Dave
This is the latest GraphX-based ALS implementation that I'm aware of: https://github.com/ankurdave/spark/blob/GraphXALS/graphx/src/main/scala/org/apache/spark/graphx/lib/ALS.scala When I benchmarked it last year, it was about twice as slow as MLlib's ALS, and I think the latter has gotten faster

Re: GraphX vertex partition/location strategy

2015-01-19 Thread Ankur Dave
No - the vertices are hash-partitioned onto workers independently of the edges. It would be nice for each vertex to be on the worker with the most adjacent edges, but we haven't done this yet since it would add a lot of complexity to avoid load imbalance while reducing the overall communication by

Re: [VOTE] Designating maintainers for some Spark components

2014-11-06 Thread Ankur Dave
+1 (binding) Ankur http://www.ankurdave.com/ On Wed, Nov 5, 2014 at 5:31 PM, Matei Zaharia matei.zaha...@gmail.com wrote: I'd like to formally call a [VOTE] on this model, to last 72 hours. The [VOTE] will end on Nov 8, 2014 at 6 PM PST.

Re: PARSING_ERROR from kryo

2014-09-15 Thread Ankur Dave
At 2014-09-15 08:59:48 -0700, Andrew Ash and...@andrewash.com wrote: I'm seeing the same exception now on the Spark 1.1.0 release. Did you ever get this figured out? [...] On Thu, Aug 21, 2014 at 2:14 PM, npanj nitinp...@gmail.com wrote: I am getting PARSING_ERROR while running my job on

Re: Graphx seems to be broken while Creating a large graph(6B nodes in my case)

2014-08-25 Thread Ankur Dave
I posted the fix on the JIRA ticket (https://issues.apache.org/jira/browse/SPARK-3190). To update the user list, this is indeed an integer overflow problem when summing up the partition sizes. The fix is to use Longs for the sum: https://github.com/apache/spark/pull/2106. Ankur

Re: VertexPartition and ShippableVertexPartition

2014-07-28 Thread Ankur Dave
On Mon, Jul 28, 2014 at 4:29 AM, Larry Xiao xia...@sjtu.edu.cn wrote: On 7/28/14, 3:41 PM, shijiaxin wrote: There is a VertexPartition in the EdgePartition,which is created by EdgePartitionBuilder.toEdgePartition. and There is also a ShippableVertexPartition in the VertexRDD. These two

Re: GraphX graph partitioning strategy

2014-07-25 Thread Ankur Dave
Hi Larry, GraphX's graph constructor leaves the edges in their original partitions by default. To support arbitrary multipass graph partitioning, one idea is to take advantage of that by partitioning the graph externally to GraphX (though possibly using information from GraphX such as the

Re: GraphX graph partitioning strategy

2014-07-25 Thread Ankur Dave
Oops, the code should be: val unpartitionedGraph: Graph[Int, Int] = ...val numPartitions: Int = 128 def getTripletPartition(e: EdgeTriplet[Int, Int]): PartitionID = ... // Get the triplets using GraphX, then use Spark to repartition themval partitionedEdges = unpartitionedGraph.triplets .map(e

Re: GraphX can not unpersist edges of old graph?

2014-06-12 Thread Ankur Dave
We didn't provide an unpersist API for Graph because the internal dependency structure of a graph can make it hard to unpersist correctly in a way that avoids recomputation. However, you can directly unpersist a graph's vertices and edges RDDs using graph.vertices.unpersist() and

Re: Removing spark-debugger.md file from master?

2014-06-03 Thread Ankur Dave
I agree, let's go ahead and remove it. Ankur http://www.ankurdave.com/

Re: [VOTE] Release Apache Spark 1.0.0 (RC11)

2014-05-27 Thread Ankur Dave
0 OK, I withdraw my downvote. Ankur http://www.ankurdave.com/

Re: Spark 1.0: outerJoinVertices seems to return null for vertex attributes when input was partitioned and vertex attribute type is changed

2014-05-26 Thread Ankur Dave
This is probably due to SPARK-1931https://issues.apache.org/jira/browse/SPARK-1931, which I just fixed in PR #885 https://github.com/apache/spark/pull/885. Is the problem resolved if you use the current Spark master? Ankur http://www.ankurdave.com/