You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by rnowling <gi...@git.apache.org> on 2014/12/11 19:37:30 UTC

[GitHub] spark pull request: [SPARK-4728] Add exponential, gamma, and log n...

GitHub user rnowling opened a pull request:

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

    [SPARK-4728] Add exponential, gamma, and log normal sampling to MLlib da...

    ...ta generators
    
    This patch adds:
    
    * Exponential, gamma, and log normal generators that wrap Apache Commons math3 to the private API
    * Functions for generating exponential, gamma, and log normal RDDs and vector RDDs
    * Tests for the above

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

    $ git pull https://github.com/rnowling/spark spark4728

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

    https://github.com/apache/spark/pull/3680.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 #3680
    
----
commit 9f96232a675ae0850275347c3cc9bd69676df5af
Author: RJ Nowling <rn...@gmail.com>
Date:   2014-12-11T18:31:38Z

    [SPARK-4728] Add exponential, gamma, and log normal sampling to MLlib data generators

----


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67651823
  
    Thanks @mengxr , you 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67535783
  
    add to whitelist


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062815
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    --- End diff --
    
    Could be a `val`


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67567074
  
    @rnowling The update looks good to me. We used Poisson sampler from colt before, which cannot reset seed. That's why we made new instances in `setSeed`. Now with commons-math3, we don't need to do this anymore.
    
    There are two things left:
    
    1. Could you add tests in `JavaRandomRDDsSuite`? Just make sure the Java methods work.
    2. Could you create a JIRA for the Python API? It should be in a separate PR. It is okay if you don't have time to work on it. We just need a reminder.


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67550998
  
      [Test build #24594 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24594/consoleFull) for   PR 3680 at commit [`58f5b97`](https://github.com/apache/spark/commit/58f5b97174ee7e493cb2a3e64c568a1f72d0fa3a).
     * This patch **passes all 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062812
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -17,7 +17,7 @@
     
     package org.apache.spark.mllib.random
     
    -import org.apache.commons.math3.distribution.PoissonDistribution
    +import org.apache.commons.math3.distribution.{ExponentialDistribution, GammaDistribution, LogNormalDistribution, PoissonDistribution}
    --- End diff --
    
    line too wide


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66680583
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24372/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062822
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new ExponentialDistribution(mean)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): ExponentialGenerator = new ExponentialGenerator(mean)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the gamma distribution with the given shape and scale.
    + *
    + * @param shape shape for the gamma distribution.
    + * @param scale scale for the gamma distribution
    + */
    +@DeveloperApi
    +class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new GammaDistribution(shape, scale)
    --- End diff --
    
    Could be a `val`.


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66821640
  
    This last failure occurred in Spark Streaming.  Can we re-run the tests? 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67583687
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24608/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67576447
  
      [Test build #24608 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24608/consoleFull) for   PR 3680 at commit [`455f50a`](https://github.com/apache/spark/commit/455f50acd550322f0551027d2fc061ed7f1f0dd5).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

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


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67583682
  
      [Test build #24608 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24608/consoleFull) for   PR 3680 at commit [`455f50a`](https://github.com/apache/spark/commit/455f50acd550322f0551027d2fc061ed7f1f0dd5).
     * This patch **passes all 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66797701
  
      [Test build #24410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24410/consoleFull) for   PR 3680 at commit [`58f5b97`](https://github.com/apache/spark/commit/58f5b97174ee7e493cb2a3e64c568a1f72d0fa3a).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66667105
  
      [Test build #24372 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24372/consoleFull) for   PR 3680 at commit [`9f96232`](https://github.com/apache/spark/commit/9f96232a675ae0850275347c3cc9bd69676df5af).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67551007
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24594/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66807063
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24410/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67576155
  
    Hi @mengxr , 
    
    I added the tests to `JavaRandomRDDsSuite` and created a JIRA for the PySpark API:
    https://issues.apache.org/jira/browse/SPARK-4891
    
    I'd be happy to tackle the changes to the Python API as well but it may happen after the holidays.  Would you still be willing to assign the JIRA to me?  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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67536924
  
      [Test build #24594 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24594/consoleFull) for   PR 3680 at commit [`58f5b97`](https://github.com/apache/spark/commit/58f5b97174ee7e493cb2a3e64c568a1f72d0fa3a).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66686329
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24374/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062823
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new ExponentialDistribution(mean)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): ExponentialGenerator = new ExponentialGenerator(mean)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the gamma distribution with the given shape and scale.
    + *
    + * @param shape shape for the gamma distribution.
    + * @param scale scale for the gamma distribution
    + */
    +@DeveloperApi
    +class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new GammaDistribution(shape, scale)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new GammaDistribution(shape, scale)
    --- End diff --
    
    ditto


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67535760
  
    test 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67557181
  
    @mengxr Thanks for the quick feedback! In this most recent commit, I did the following:
    
    * Changed the vars to vals
    * Removed unnecessary creation of distribution objects
    * Wrapped import statement across lines -- is this the preferred solution?
    
    I made these changes for the Poisson generator as well as my new additions.


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062835
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new ExponentialDistribution(mean)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): ExponentialGenerator = new ExponentialGenerator(mean)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the gamma distribution with the given shape and scale.
    + *
    + * @param shape shape for the gamma distribution.
    + * @param scale scale for the gamma distribution
    + */
    +@DeveloperApi
    +class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new GammaDistribution(shape, scale)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new GammaDistribution(shape, scale)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): GammaGenerator = new GammaGenerator(shape, scale)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the log normal distribution with the
    + * given mean and standard deviation.
    + *
    + * @param mean mean for the log normal distribution.
    + * @param std standard deviation for the log normal distribution
    + */
    +@DeveloperApi
    +class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new LogNormalDistribution(mean, std)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new LogNormalDistribution(mean, std)
    --- End diff --
    
    not necessary


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67531552
  
    @mengxr @JoshRosen could you rerun the tests please?  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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66676689
  
      [Test build #24374 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24374/consoleFull) for   PR 3680 at commit [`84fd98d`](https://github.com/apache/spark/commit/84fd98d6b1e625e1c143bf16fccbf91ff2040d08).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67557007
  
      [Test build #24601 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24601/consoleFull) for   PR 3680 at commit [`3e1134a`](https://github.com/apache/spark/commit/3e1134a67efe53a58b36ad8d8ed666f70ce37cef).
     * 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67598867
  
    LGTM. Merged into master. I've assigned the Python API JIRA to you and set the target version to 1.3.0. So it is not in a hurry. Happy holidays!


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66807046
  
      [Test build #24410 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24410/consoleFull) for   PR 3680 at commit [`58f5b97`](https://github.com/apache/spark/commit/58f5b97174ee7e493cb2a3e64c568a1f72d0fa3a).
     * This patch **fails Spark unit 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062817
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new ExponentialDistribution(mean)
    --- End diff --
    
    Is this line necessary?


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66680567
  
      [Test build #24372 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24372/consoleFull) for   PR 3680 at commit [`9f96232`](https://github.com/apache/spark/commit/9f96232a675ae0850275347c3cc9bd69676df5af).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] `
      * `class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] `
      * `class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] `



---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-66686321
  
      [Test build #24374 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24374/consoleFull) for   PR 3680 at commit [`84fd98d`](https://github.com/apache/spark/commit/84fd98d6b1e625e1c143bf16fccbf91ff2040d08).
     * This patch **fails Spark unit 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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67568349
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24601/
    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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#discussion_r22062832
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala ---
    @@ -99,3 +99,69 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
     
       override def copy(): PoissonGenerator = new PoissonGenerator(mean)
     }
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the exponential distribution with the given mean.
    + *
    + * @param mean mean for the exponential distribution.
    + */
    +@DeveloperApi
    +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new ExponentialDistribution(mean)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new ExponentialDistribution(mean)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): ExponentialGenerator = new ExponentialGenerator(mean)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the gamma distribution with the given shape and scale.
    + *
    + * @param shape shape for the gamma distribution.
    + * @param scale scale for the gamma distribution
    + */
    +@DeveloperApi
    +class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new GammaDistribution(shape, scale)
    +
    +  override def nextValue(): Double = rng.sample()
    +
    +  override def setSeed(seed: Long) {
    +    rng = new GammaDistribution(shape, scale)
    +    rng.reseedRandomGenerator(seed)
    +  }
    +
    +  override def copy(): GammaGenerator = new GammaGenerator(shape, scale)
    +}
    +
    +/**
    + * :: DeveloperApi ::
    + * Generates i.i.d. samples from the log normal distribution with the
    + * given mean and standard deviation.
    + *
    + * @param mean mean for the log normal distribution.
    + * @param std standard deviation for the log normal distribution
    + */
    +@DeveloperApi
    +class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] {
    +
    +  private var rng = new LogNormalDistribution(mean, std)
    --- End diff --
    
    Could be a `val`.


---
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-4728][MLLib] Add exponential, gamma, an...

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

    https://github.com/apache/spark/pull/3680#issuecomment-67568335
  
      [Test build #24601 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24601/consoleFull) for   PR 3680 at commit [`3e1134a`](https://github.com/apache/spark/commit/3e1134a67efe53a58b36ad8d8ed666f70ce37cef).
     * This patch **passes all 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