You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by davies <gi...@git.apache.org> on 2014/08/29 21:02:14 UTC

[GitHub] spark pull request: [SPARK-3309] [PySpark] Put all public API in _...

GitHub user davies opened a pull request:

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

    [SPARK-3309] [PySpark] Put all public API in __all__

    Put all public API in __all__, also put them all in pyspark.__init__.py, then we can got all the documents for public API by `pydoc pyspark`. It also can be used by other programs (such as Sphinx or Epydoc) to generate only documents for public APIs.

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

    $ git pull https://github.com/davies/spark public

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

    https://github.com/apache/spark/pull/2205.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 #2205
    
----
commit 6caab21f380ed6d995bb44875447f2c1ac96cb06
Author: Davies Liu <da...@gmail.com>
Date:   2014-08-29T18:23:38Z

    add public interfaces into pyspark.__init__.py

commit 6281b482fc2f1984b4e1a541e76700a853142fa3
Author: Davies Liu <da...@gmail.com>
Date:   2014-08-29T18:57:45Z

    fix doc for 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53922778
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19489/consoleFull) for   PR 2205 at commit [`6281b48`](https://github.com/apache/spark/commit/6281b482fc2f1984b4e1a541e76700a853142fa3).
     * This patch **fails** 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r16922689
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -61,13 +61,16 @@
     
     from pyspark.conf import SparkConf
     from pyspark.context import SparkContext
    -from pyspark.sql import SQLContext
     from pyspark.rdd import RDD
    -from pyspark.sql import SchemaRDD
    -from pyspark.sql import Row
    +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row
     from pyspark.files import SparkFiles
     from pyspark.storagelevel import StorageLevel
    +from pyspark.accumulators import Accumulator, AccumulatorParam
    +from pyspark.broadcast import Broadcast
    +from pyspark.serializers import MarshalSerializer, PickleSerializer
     
    -
    -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD",
    -           "SparkFiles", "StorageLevel", "Row"]
    +__all__ = [
    +    "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",
    +    "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer",
    +    "SQLContext", "HiveContext", "SchemaRDD", "Row",
    --- End diff --
    
    I'm not sure we want the SQL things here. They should be called pyspark.sql.Row instead of pyspark.Row.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r17071220
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -94,6 +94,9 @@
     from pyspark.serializers import read_int, PickleSerializer
     
     
    +__all__ = ['Accumulator', 'AccumulatorParam']
    --- End diff --
    
    Fair enough, since we can always add it later if we decide to.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r17070879
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -94,6 +94,9 @@
     from pyspark.serializers import read_int, PickleSerializer
     
     
    +__all__ = ['Accumulator', 'AccumulatorParam']
    --- End diff --
    
    Maybe `AddingAccumulatorParam` should also be public?


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53927045
  
    The failed cases are not related to this.
    
    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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53949189
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19515/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * This patch **fails** 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53948396
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19515/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53941658
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19502/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class Sqrt(child: Expression) extends UnaryExpression `



---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r16922982
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -61,13 +61,16 @@
     
     from pyspark.conf import SparkConf
     from pyspark.context import SparkContext
    -from pyspark.sql import SQLContext
     from pyspark.rdd import RDD
    -from pyspark.sql import SchemaRDD
    -from pyspark.sql import Row
    +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row
     from pyspark.files import SparkFiles
     from pyspark.storagelevel import StorageLevel
    +from pyspark.accumulators import Accumulator, AccumulatorParam
    +from pyspark.broadcast import Broadcast
    +from pyspark.serializers import MarshalSerializer, PickleSerializer
     
    -
    -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD",
    -           "SparkFiles", "StorageLevel", "Row"]
    +__all__ = [
    +    "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",
    +    "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer",
    +    "SQLContext", "HiveContext", "SchemaRDD", "Row",
    --- End diff --
    
    There are already there, if we remove them now, it will break something, especially SQLContext and HiveContext.


---
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-3309] [PySpark] Put all public API in _...

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

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


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r17070804
  
    --- Diff: python/pyspark/broadcast.py ---
    @@ -59,11 +63,20 @@ def __init__(self, bid, value, java_broadcast=None,
             """
             self.bid = bid
             if path is None:
    -            self.value = value
    +            self._value = value
             self._jbroadcast = java_broadcast
             self._pickle_registry = pickle_registry
             self.path = path
     
    +    @property
    --- End diff --
    
    This is a good refactoring, especially how it allows `value` to be documented.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54255285
  
    @mateiz fixed, 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54259512
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19640/consoleFull) for   PR 2205 at commit [`c6c5567`](https://github.com/apache/spark/commit/c6c5567b13077c6571480325e8cb135c05559dd0).
     * This patch **passes** 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54201897
  
    The failed case ((streaming/test:test) sbt.TestsFailedException) is not related to this PR.
    
    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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r17071170
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -94,6 +94,9 @@
     from pyspark.serializers import read_int, PickleSerializer
     
     
    +__all__ = ['Accumulator', 'AccumulatorParam']
    --- End diff --
    
    I think `AddingAccumulatorParam` is just part of implementation details, only `Accumulator` and `AccumulatorParam` are used in API.
    
    Right now, I would like to keep the minimized public classes, it's easy to add 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53933841
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19494/consoleFull) for   PR 2205 at commit [`7e3016a`](https://github.com/apache/spark/commit/7e3016a270a558ce537043a906171bcb9a0deba1).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54480937
  
    > I'm really looking forward to Sphinx docs further down the road, so thanks for doing this!
    
    Same here!


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r16923130
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -61,13 +61,16 @@
     
     from pyspark.conf import SparkConf
     from pyspark.context import SparkContext
    -from pyspark.sql import SQLContext
     from pyspark.rdd import RDD
    -from pyspark.sql import SchemaRDD
    -from pyspark.sql import Row
    +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row
     from pyspark.files import SparkFiles
     from pyspark.storagelevel import StorageLevel
    +from pyspark.accumulators import Accumulator, AccumulatorParam
    +from pyspark.broadcast import Broadcast
    +from pyspark.serializers import MarshalSerializer, PickleSerializer
     
    -
    -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD",
    -           "SparkFiles", "StorageLevel", "Row"]
    +__all__ = [
    +    "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",
    +    "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer",
    +    "SQLContext", "HiveContext", "SchemaRDD", "Row",
    --- End diff --
    
    I think these are just convenience imports, right?  The documentation should probably use the full package names.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53948334
  
    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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54253897
  
    Hey Davies, so with the SQL change, I also noticed we have some error messages that say "please use pyspark.Row". They should say pyspark.sql.Row 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54226570
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19605/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * This patch **passes** 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53918058
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19489/consoleFull) for   PR 2205 at commit [`6281b48`](https://github.com/apache/spark/commit/6281b482fc2f1984b4e1a541e76700a853142fa3).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#discussion_r16923970
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -61,13 +61,16 @@
     
     from pyspark.conf import SparkConf
     from pyspark.context import SparkContext
    -from pyspark.sql import SQLContext
     from pyspark.rdd import RDD
    -from pyspark.sql import SchemaRDD
    -from pyspark.sql import Row
    +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row
     from pyspark.files import SparkFiles
     from pyspark.storagelevel import StorageLevel
    +from pyspark.accumulators import Accumulator, AccumulatorParam
    +from pyspark.broadcast import Broadcast
    +from pyspark.serializers import MarshalSerializer, PickleSerializer
     
    -
    -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD",
    -           "SparkFiles", "StorageLevel", "Row"]
    +__all__ = [
    +    "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",
    +    "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer",
    +    "SQLContext", "HiveContext", "SchemaRDD", "Row",
    --- End diff --
    
    yeah, I had removed them from __all__, but still keep the import for back compatibility.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53937444
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19502/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54255622
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19640/consoleFull) for   PR 2205 at commit [`c6c5567`](https://github.com/apache/spark/commit/c6c5567b13077c6571480325e8cb135c05559dd0).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53938109
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19494/consoleFull) for   PR 2205 at commit [`7e3016a`](https://github.com/apache/spark/commit/7e3016a270a558ce537043a906171bcb9a0deba1).
     * This patch **passes** 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54218518
  
    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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54346577
  
    This looks good to me.  I'm really looking forward to Sphinx docs further down the road, so thanks for doing this!
    
    Merging into master.


---
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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-54219327
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19605/consoleFull) for   PR 2205 at commit [`f7b35be`](https://github.com/apache/spark/commit/f7b35bea0e6ddce823327da56cab1cfa9dbf419e).
     * 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-3309] [PySpark] Put all public API in _...

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

    https://github.com/apache/spark/pull/2205#issuecomment-53917493
  
    There are bunch of types defined in pyspark.sql, should we also put them in pyspark? They are supposed to be developer APIs, so I think it's fine to leave them in pyspark.sql.


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