You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jbencook <gi...@git.apache.org> on 2014/12/22 17:50:34 UTC

[GitHub] spark pull request: [SPARK-4860][pyspark][sql] speeding up `sample...

GitHub user jbencook opened a pull request:

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

    [SPARK-4860][pyspark][sql] speeding up `sample()` and `takeSample()`

    This PR modifies the python `SchemaRDD` to use `sample()` and `takeSample()` from Scala instead of the slower python implementations from `rdd.py`. This is worthwhile because the `Row`'s are already serialized as Java objects.
    
    In order to use the faster `takeSample()`, a `takeSampleToPython()` method was implemented in `SchemaRDD.scala` following the pattern of `collectToPython()`.

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

    $ git pull https://github.com/jbencook/spark master

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

    https://github.com/apache/spark/pull/3764.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 #3764
    
----
commit 020cbdf8814d25121a909b634f908dc8c41bd739
Author: jbencook <jb...@gmail.com>
Date:   2014-12-22T16:40:49Z

    [SPARK-4860][pyspark][sql] using Scala implementations of `sample()` and `takeSample()`

----


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67908389
  
    Thanks for the comment @davies. Do you want me to add the sample() method to the JavaSchemaRDD in this PR? Or make a ticket for both sample() and randomSplit()?


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67942534
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24737/
    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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67942526
  
      [Test build #24737 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24737/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67920279
  
      [Test build #24720 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24720/consoleFull) for   PR 3764 at commit [`de22f70`](https://github.com/apache/spark/commit/de22f706d8bbe6d80a6ea2e9a5343b77e0695471).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67942895
  
    Oops- shouldn't have tried to fix this before I had my coffee. Will fix in a bit. 


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-68013963
  
      [Test build #554 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/554/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67860230
  
    Can one of the admins verify this patch?


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-68005823
  
    @davies I can't seem to replicate this failure on my machine. Is there any chance these tests are timing out non-deterministically? Can you think of any reason why the indentation would cause this?
    
    Looks like everything was pretty quick in the successful builds, e.g. [#24720](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24720/consoleFull):
    
        [info] CliSuite:
        [info] - Simple commands (29 seconds, 879 milliseconds)
        [info] - Single command with -e (22 seconds, 480 milliseconds)
        [info] HiveThriftServer2Suite:
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - Test JDBC query execution (31 seconds, 4 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - Test JDBC query execution in Http Mode (29 seconds, 606 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - SPARK-3004 regression: result set containing NULL (28 seconds, 833 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - GetInfo Thrift API (26 seconds, 231 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - Checks Hive version (26 seconds, 886 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - Checks Hive version in Http Mode (27 seconds, 400 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - SPARK-4292 regression: result set iterator issue (31 seconds, 443 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - SPARK-4309 regression: Date type support (26 seconds, 997 milliseconds)
        stopping org.apache.spark.sql.hive.thriftserver.HiveThriftServer2
        [info] - SPARK-4407 regression: Complex type support (27 seconds, 714 milliseconds)



---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#discussion_r22196879
  
    --- Diff: python/pyspark/sql.py ---
    @@ -2085,6 +2085,35 @@ def subtract(self, other, numPartitions=None):
             else:
                 raise ValueError("Can only subtract another SchemaRDD")
     
    +    def sample(self, withReplacement, fraction, seed=None):
    +        """
    +        Return a sampled subset of this SchemaRDD.
    +
    +        >>> srdd = sqlCtx.inferSchema(rdd)
    +        >>> srdd.sample(False, 0.5, 97).count()
    +        2L
    +        """
    +        assert fraction >= 0.0, "Negative fraction value: %s" % fraction
    +        seed = seed if seed is not None else random.randint(0, sys.maxint)
    +        rdd = self._jschema_rdd.baseSchemaRDD().sample(
    --- End diff --
    
    Could you add sample() for JavaSchemaRDD()? Then this line could be changed to use self._jschema_rdd.sample()


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67890740
  
    @davies can you take a look?


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-68017882
  
    This looks good to me, too, so I'm going to merge it into `master` (1.3.0).  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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#discussion_r22204525
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala ---
    @@ -438,6 +438,20 @@ class SchemaRDD(
       }
     
       /**
    +   * Serializes the Array[Row] returned by SchemaRDD's takeSample(), using the same
    +   * format as javaToPython and collectToPython. It is used by pyspark.
    +   */
    +  private[sql] def takeSampleToPython(withReplacement: Boolean,
    +                                       num: Int,
    --- End diff --
    
    nit: these arguments should be indented by 4 spaces, see others as example.


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67973120
  
    Jenkins, 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67924667
  
    LGTM, just two minor comments. After fixing them, I think it's ready to merge.


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#discussion_r22204521
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala ---
    @@ -218,4 +218,10 @@ class JavaSchemaRDD(
        */
       def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD =
         this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD
    +
    +  /**
    +   * Return an RDD with a sampled version of the underlying dataset.
    --- End diff --
    
    Return a SchemaRDD


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67888116
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24706/
    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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67984383
  
      [Test build #24740 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24740/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67909835
  
    @jbencook Maybe we could just add sample() in this PR, leave others later.


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67875459
  
    ok to test


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67920282
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24720/
    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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-68006697
  
      [Test build #554 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/554/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67888111
  
      [Test build #24706 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24706/consoleFull) for   PR 3764 at commit [`020cbdf`](https://github.com/apache/spark/commit/020cbdf8814d25121a909b634f908dc8c41bd739).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67905974
  
    @jbencook LGTM, just one minor comment.
    
    @marmbrus There are few APIs missing in JavaSchemaRDD (such as sample(), randomSplit()).


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67984392
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24740/
    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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67973506
  
      [Test build #24740 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24740/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67935475
  
      [Test build #24737 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24737/consoleFull) for   PR 3764 at commit [`6fbc769`](https://github.com/apache/spark/commit/6fbc76993a3ab8c00afde49234d00cf0128f5db5).
     * 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67875679
  
      [Test build #24706 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24706/consoleFull) for   PR 3764 at commit [`020cbdf`](https://github.com/apache/spark/commit/020cbdf8814d25121a909b634f908dc8c41bd739).
     * 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-4860][pyspark][sql] speeding up `sample...

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

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


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-68006720
  
    @jbencook Maybe this case is flaky, let's test it again.


---
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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67915532
  
    OK @davies, should be good to go 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-4860][pyspark][sql] speeding up `sample...

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

    https://github.com/apache/spark/pull/3764#issuecomment-67915447
  
      [Test build #24720 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24720/consoleFull) for   PR 3764 at commit [`de22f70`](https://github.com/apache/spark/commit/de22f706d8bbe6d80a6ea2e9a5343b77e0695471).
     * 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