You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ankurdave <gi...@git.apache.org> on 2014/09/25 04:35:58 UTC

[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

GitHub user ankurdave opened a pull request:

    https://github.com/apache/spark/pull/2530

    [SPARK-3666] Extract interfaces for EdgeRDD and VertexRDD

    This discourages users from calling the VertexRDD and EdgeRDD constructor and makes it easier for future changes to ensure backward compatibility.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ankurdave/spark SPARK-3666

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2530.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2530
    
----
commit 55b6398d1308eae95e5926bc254a088e658d618c
Author: Ankur Dave <an...@gmail.com>
Date:   2014-09-22T05:37:47Z

    Extract EdgeRDD interface and move implementation to EdgeRDDImpl

commit 620e6035fd6c60e248906bf874cae3fae9ea761a
Author: Ankur Dave <an...@gmail.com>
Date:   2014-09-22T19:47:12Z

    Extract VertexRDD interface and move implementation to VertexRDDImpl
    
    This breaks binary compatibility with the error
    
        java.lang.IncompatibleClassChangeError: Found interface
        org.apache.spark.graphx.VertexRDD, but class was expected
    
    The solution is to do a clean rebuild (sbt/sbt clean assembly).

----


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62504892
  
      [Test build #23197 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23197/consoleFull) for   PR 2530 at commit [`1472390`](https://github.com/apache/spark/commit/1472390b375435c04d3b07abb8bdc39508daa226).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62497013
  
    I filed a JIRA for Mima misreporting: https://issues.apache.org/jira/browse/SPARK-4335
    
    cc @pwendell @ScrapCodes 


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by ankurdave <gi...@git.apache.org>.
Github user ankurdave commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62690967
  
    @rxin MIMA passed! PTAL.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62503914
  
    Also @ankurdave this needs to be updated now...


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61388649
  
      [Test build #22716 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22716/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-56770727
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20777/consoleFull) for   PR 2530 at commit [`620e603`](https://github.com/apache/spark/commit/620e6035fd6c60e248906bf874cae3fae9ea761a).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait EdgeRDD[@specialized ED, VD] extends RDD[Edge[ED]] `
      * `trait VertexRDD[@specialized VD] extends RDD[(VertexId, VD)] `
      * `class EdgeRDDImpl[@specialized ED: ClassTag, VD: ClassTag](`
      * `class VertexRDDImpl[@specialized VD](`



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-58594876
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21554/consoleFull) for   PR 2530 at commit [`9ba4ec4`](https://github.com/apache/spark/commit/9ba4ec45a019573bf395d660508ef30dd1597de7).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait EdgeRDD[@specialized ED, VD] extends RDD[Edge[ED]] `
      * `trait VertexRDD[@specialized VD] extends RDD[(VertexId, VD)] `



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61390141
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22714/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61390445
  
      [Test build #22716 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22716/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62800777
  
    Merging in master & branch-1.2. Thanks!



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-56766994
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20777/consoleFull) for   PR 2530 at commit [`620e603`](https://github.com/apache/spark/commit/620e6035fd6c60e248906bf874cae3fae9ea761a).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61398050
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22754/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62510292
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23197/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62481151
  
      [Test build #516 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/516/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61390136
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22714/consoleFull) for   PR 2530 at commit [`cbe15f2`](https://github.com/apache/spark/commit/cbe15f25e46ff5d0ab46bbc4bf802a06f21c5b2d).
     * This patch **fails** unit tests.
     * This patch **does not** merge cleanly!



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-56770729
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20777/


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by ankurdave <gi...@git.apache.org>.
Github user ankurdave commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-58598193
  
    @rxin Right, this just contains refactoring and doc changes. I marked the Impl constructors package-private since they are accessed by the corresponding interface object (EdgeRDD.fromEdgePartitions and VertexRDD.apply). I'll evaluate specialization separately. PTAL


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62680914
  
    I took a look and I think MIMA might be reporting a semi-legitimate issue - though this is a tricky situation. I think the issue is - if someone extended the earlier version of EdgeRDD, their class would have inherited the concrete `getPartitions` call that was present in the earlier version. Now `EdgeRDD` doesn't have this  `getPartitions` function (it has been moved to the concrete version). I think in general it's fine because this is defined in the parent RDD class, but maybe MIMA doesn't check whether the function is defined further up in the inheritance chain.
    
    In terms of compatiblity for callers, this is a protected method so it really doesn't matter.
    
    I'd go ahead and just add ignores for this and be done with it. Fixing this in MIMA might be diffiuclt


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62688542
  
      [Test build #23254 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23254/consoleFull) for   PR 2530 at commit [`d681f45`](https://github.com/apache/spark/commit/d681f45721c0134566877ab911d415ec02dd9998).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `abstract class EdgeRDD[ED, VD](`
      * `abstract class VertexRDD[VD](`



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61390446
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22716/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62489075
  
      [Test build #516 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/516/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `abstract class EdgeRDD[ED, VD](`
      * `abstract class VertexRDD[VD](`



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-58589180
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21554/consoleFull) for   PR 2530 at commit [`9ba4ec4`](https://github.com/apache/spark/commit/9ba4ec45a019573bf395d660508ef30dd1597de7).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-60716747
  
    This LGTM.
    
    One question related to this - should we use abstract class instead of trait? It would be better for maintaining binary compatibility across versions


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61396599
  
      [Test build #22754 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22754/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2530#discussion_r18122160
  
    --- Diff: graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala ---
    @@ -32,74 +32,7 @@ import org.apache.spark.graphx.impl.EdgePartitionBuilder
      * edge to provide the triplet view. Shipping of the vertex attributes is managed by
      * `impl.ReplicatedVertexView`.
      */
    -class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag](
    -    val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])],
    -    val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY)
    -  extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
    -
    -  override def setName(_name: String): this.type = {
    -    if (partitionsRDD.name != null) {
    -      partitionsRDD.setName(partitionsRDD.name + ", " + _name)
    -    } else {
    -      partitionsRDD.setName(_name)
    -    }
    -    this
    -  }
    -  setName("EdgeRDD")
    -
    -  override protected def getPartitions: Array[Partition] = partitionsRDD.partitions
    -
    -  /**
    -   * If `partitionsRDD` already has a partitioner, use it. Otherwise assume that the
    -   * [[PartitionID]]s in `partitionsRDD` correspond to the actual partitions and create a new
    -   * partitioner that allows co-partitioning with `partitionsRDD`.
    -   */
    -  override val partitioner =
    -    partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD)))
    -
    -  override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = {
    -    val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context)
    -    if (p.hasNext) {
    -      p.next._2.iterator.map(_.copy())
    -    } else {
    -      Iterator.empty
    -    }
    -  }
    -
    -  override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect()
    -
    -  /**
    -   * Persists the edge partitions at the specified storage level, ignoring any existing target
    -   * storage level.
    -   */
    -  override def persist(newLevel: StorageLevel): this.type = {
    -    partitionsRDD.persist(newLevel)
    -    this
    -  }
    -
    -  override def unpersist(blocking: Boolean = true): this.type = {
    -    partitionsRDD.unpersist(blocking)
    -    this
    -  }
    -
    -  /** Persists the vertex partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. */
    -  override def cache(): this.type = {
    -    partitionsRDD.persist(targetStorageLevel)
    -    this
    -  }
    -
    -  private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag](
    -      f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = {
    -    this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter =>
    -      if (iter.hasNext) {
    -        val (pid, ep) = iter.next()
    -        Iterator(Tuple2(pid, f(pid, ep)))
    -      } else {
    -        Iterator.empty
    -      }
    -    }, preservesPartitioning = true))
    -  }
    -
    +trait EdgeRDD[@specialized ED, VD] extends RDD[Edge[ED]] {
    --- End diff --
    
    i think we should probably remove the specialized and test performance. maybe we can do that in a a separate pr


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61394189
  
      [Test build #22741 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22741/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2530#discussion_r18122169
  
    --- Diff: graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala ---
    @@ -0,0 +1,217 @@
    +/*
    + * 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.spark.graphx.impl
    +
    +import scala.reflect.ClassTag
    +
    +import org.apache.spark._
    +import org.apache.spark.SparkContext._
    +import org.apache.spark.rdd._
    +import org.apache.spark.storage.StorageLevel
    +
    +import org.apache.spark.graphx._
    +import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._
    +import org.apache.spark.graphx.impl.VertexRDDFunctions._
    +
    +class VertexRDDImpl[@specialized VD](
    --- End diff --
    
    ctor private too


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61398049
  
      [Test build #22754 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22754/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `abstract class EdgeRDD[ED, VD](`
      * `abstract class VertexRDD[VD](`



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61388473
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22704/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61394102
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2530


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2530#discussion_r18122168
  
    --- Diff: graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.spark.graphx.impl
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +
    +import org.apache.spark.graphx._
    +
    +class EdgeRDDImpl[@specialized ED: ClassTag, VD: ClassTag](
    --- End diff --
    
    can u mark the constructor as private?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61388471
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22704/consoleFull) for   PR 2530 at commit [`931b587`](https://github.com/apache/spark/commit/931b587267045c9af731c84076a37777517372bd).
     * This patch **fails** unit tests.
     * This patch **does not** merge cleanly!



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61386195
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22704/consoleFull) for   PR 2530 at commit [`931b587`](https://github.com/apache/spark/commit/931b587267045c9af731c84076a37777517372bd).
     * This patch **does not** merge cleanly!


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61395476
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22741/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62510289
  
      [Test build #23197 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23197/consoleFull) for   PR 2530 at commit [`1472390`](https://github.com/apache/spark/commit/1472390b375435c04d3b07abb8bdc39508daa226).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-57043598
  
    Looks good. I didn't look closely but I'm assuming you didn't change any impl logic. Let me know if that is not true.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61388401
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22714/consoleFull) for   PR 2530 at commit [`cbe15f2`](https://github.com/apache/spark/commit/cbe15f25e46ff5d0ab46bbc4bf802a06f21c5b2d).
     * This patch **does not** merge cleanly!


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61395474
  
      [Test build #22741 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22741/consoleFull) for   PR 2530 at commit [`24201d4`](https://github.com/apache/spark/commit/24201d453c3d490f39a5eb43f70a1ba69012222b).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `abstract class EdgeRDD[ED, VD](`
      * `abstract class VertexRDD[VD](`



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62688547
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23254/
    Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-62681405
  
      [Test build #23254 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23254/consoleFull) for   PR 2530 at commit [`d681f45`](https://github.com/apache/spark/commit/d681f45721c0134566877ab911d415ec02dd9998).
     * This patch merges cleanly.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-61396475
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3666] Extract interfaces for EdgeRDD an...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2530#issuecomment-58594881
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21554/Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org