aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/session.py
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19055][SQL][PYSPARK] Fix SparkSession initialization when ↵Liang-Chi Hsieh2017-01-121-6/+10
| | | | | | | | | | | | | | | | | | | | | | SparkContext is stopped ## What changes were proposed in this pull request? In SparkSession initialization, we store created the instance of SparkSession into a class variable _instantiatedContext. Next time we can use SparkSession.builder.getOrCreate() to retrieve the existing SparkSession instance. However, when the active SparkContext is stopped and we create another new SparkContext to use, the existing SparkSession is still associated with the stopped SparkContext. So the operations with this existing SparkSession will be failed. We need to detect such case in SparkSession and renew the class variable _instantiatedContext if needed. ## How was this patch tested? New test added in PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16454 from viirya/fix-pyspark-sparksession.
* [SPARK-17720][SQL] introduce static SQL confWenchen Fan2016-10-111-1/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897. Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf. ## How was this patch tested? new tests in SQLConfSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #15295 from cloud-fan/global-conf.
* [SPARK-17261] [PYSPARK] Using HiveContext after re-creating SparkContext in ↵Jeff Zhang2016-09-021-0/+1
| | | | | | | | | | | | | | | | | | | | | Spark 2.0 throws "Java.lang.illegalStateException: Cannot call methods on a stopped sparkContext" ## What changes were proposed in this pull request? Set SparkSession._instantiatedContext as None so that we can recreate SparkSession again. ## How was this patch tested? Tested manually using the following command in pyspark shell ``` spark.stop() spark = SparkSession.builder.enableHiveSupport().getOrCreate() spark.sql("show databases").show() ``` Author: Jeff Zhang <zjffdu@apache.org> Closes #14857 from zjffdu/SPARK-17261.
* [SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schemaDavies Liu2016-08-151-16/+13
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In 2.0, we verify the data type against schema for every row for safety, but with performance cost, this PR make it optional. When we verify the data type for StructType, it does not support all the types we support in infer schema (for example, dict), this PR fix that to make them consistent. For Row object which is created using named arguments, the order of fields are sorted by name, they may be not different than the order in provided schema, this PR fix that by ignore the order of fields in this case. ## How was this patch tested? Created regression tests for them. Author: Davies Liu <davies@databricks.com> Closes #14469 from davies/py_dict.
* [SPARK-16772][PYTHON][DOCS] Restore "datatype string" to Python API docstringsNicholas Chammas2016-07-291-6/+4
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR corrects [an error made in an earlier PR](https://github.com/apache/spark/pull/14393/files#r72843069). ## How was this patch tested? ```sh $ ./dev/lint-python PEP8 checks passed. rm -rf _build/* pydoc checks passed. ``` I also built the docs and confirmed that they looked good in my browser. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14408 from nchammas/SPARK-16772.
* [SPARK-16772] Correct API doc references to PySpark classes + formatting fixesNicholas Chammas2016-07-281-18/+23
| | | | | | | | | | | | | | | | | | ## What's Been Changed The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems. For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module. You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown. ## Testing I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14393 from nchammas/python-docstring-fixes.
* [SPARK-16503] SparkSession should provide Spark versionLiwei Lin2016-07-131-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch enables SparkSession to provide spark version. ## How was this patch tested? Manual test: ``` scala> sc.version res0: String = 2.1.0-SNAPSHOT scala> spark.version res1: String = 2.1.0-SNAPSHOT ``` ``` >>> sc.version u'2.1.0-SNAPSHOT' >>> spark.version u'2.1.0-SNAPSHOT' ``` Author: Liwei Lin <lwlin7@gmail.com> Closes #14165 from lw-lin/add-version.
* [MINOR][PYSPARK][DOC] Fix wrongly formatted examples in PySpark documentationhyukjinkwon2016-07-061-6/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes wrongly formatted examples in PySpark documentation as below: - **`SparkSession`** - **Before** ![2016-07-06 11 34 41](https://cloud.githubusercontent.com/assets/6477701/16605847/ae939526-436d-11e6-8ab8-6ad578362425.png) - **After** ![2016-07-06 11 33 56](https://cloud.githubusercontent.com/assets/6477701/16605845/ace9ee78-436d-11e6-8923-b76d4fc3e7c3.png) - **`Builder`** - **Before** ![2016-07-06 11 34 44](https://cloud.githubusercontent.com/assets/6477701/16605844/aba60dbc-436d-11e6-990a-c87bc0281c6b.png) - **After** ![2016-07-06 1 26 37](https://cloud.githubusercontent.com/assets/6477701/16607562/586704c0-437d-11e6-9483-e0af93d8f74e.png) This PR also fixes several similar instances across the documentation in `sql` PySpark module. ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #14063 from HyukjinKwon/minor-pyspark-builder.
* [SPARK-16266][SQL][STREAING] Moved DataStreamReader/Writer from pyspark.sql ↵Tathagata Das2016-06-281-1/+2
| | | | | | | | | | | | | | | | | | to pyspark.sql.streaming ## What changes were proposed in this pull request? - Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging - Exposed the necessary classes in sql.streaming package so that they appear in the docs - Added pyspark.sql.streaming module to the docs ## How was this patch tested? - updated unit tests. - generated docs for testing visibility of pyspark.sql.streaming classes. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13955 from tdas/SPARK-16266.
* [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set ↵Yin Huai2016-06-281-0/+7
| | | | | | | | | | | | | | to the existing Scala SparkContext's SparkConf ## What changes were proposed in this pull request? When we create a SparkSession at the Python side, it is possible that a SparkContext has been created. For this case, we need to set configs of the SparkSession builder to the Scala SparkContext's SparkConf (we need to do so because conf changes on a active Python SparkContext will not be propagated to the JVM side). Otherwise, we may create a wrong SparkSession (e.g. Hive support is not enabled even if enableHiveSupport is called). ## How was this patch tested? New tests and manual tests. Author: Yin Huai <yhuai@databricks.com> Closes #13931 from yhuai/SPARK-16224.
* [SPARK-15803] [PYSPARK] Support with statement syntax for SparkSessionJeff Zhang2016-06-171-0/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Support with statement syntax for SparkSession in pyspark ## How was this patch tested? Manually verify it. Although I can add unit test for it, it would affect other unit test because the SparkContext is stopped after the with statement. Author: Jeff Zhang <zjffdu@apache.org> Closes #13541 from zjffdu/SPARK-15803.
* [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQueryTathagata Das2016-06-151-5/+5
| | | | | | | | | | Renamed for simplicity, so that its obvious that its related to streaming. Existing unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13673 from tdas/SPARK-15953.
* [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream ↵Tathagata Das2016-06-141-1/+16
| | | | | | | | | | | | | | | | and writeStream for streaming DFs ## What changes were proposed in this pull request? Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams. - [x] Python API!! ## How was this patch tested? Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13653 from tdas/SPARK-15933.
* [SPARK-15935][PYSPARK] Enable test for sql/streaming.py and fix these testsShixiong Zhu2016-06-141-0/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR just enables tests for sql/streaming.py and also fixes the failures. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13655 from zsxwing/python-streaming-test.
* [MINOR] Fix Typos 'an -> a'Zheng RuiFeng2016-06-061-1/+1
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `an -> a` Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one. ## How was this patch tested? manual tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #13515 from zhengruifeng/an_a.
* [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession ↵Eric Liang2016-05-261-1/+3
| | | | | | | | | | | | | | | | builder in pyspark ## What changes were proposed in this pull request? Also sets confs in the underlying sc when using SparkSession.builder.getOrCreate(). This is a bug-fix from a post-merge comment in https://github.com/apache/spark/pull/13289 ## How was this patch tested? Python doc-tests. Author: Eric Liang <ekl@databricks.com> Closes #13309 from ericl/spark-15520-1.
* [SPARK-15520][SQL] SparkSession builder in python should also allow ↵Eric Liang2016-05-251-11/+24
| | | | | | | | | | | | | | | | | | overriding confs of existing sessions ## What changes were proposed in this pull request? This fixes the python SparkSession builder to allow setting confs correctly. This was a leftover TODO from https://github.com/apache/spark/pull/13200. ## How was this patch tested? Python doc tests. cc andrewor14 Author: Eric Liang <ekl@databricks.com> Closes #13289 from ericl/spark-15520.
* [SPARK-15417][SQL][PYTHON] PySpark shell always uses in-memory catalogAndrew Or2016-05-191-0/+6
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? There is no way to use the Hive catalog in `pyspark-shell`. This is because we used to create a `SparkContext` before calling `SparkSession.enableHiveSupport().getOrCreate()`, which just gets the existing `SparkContext` instead of creating a new one. As a result, `spark.sql.catalogImplementation` was never propagated. ## How was this patch tested? Manual. Author: Andrew Or <andrew@databricks.com> Closes #13203 from andrewor14/fix-pyspark-shell.
* [SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate ↵Reynold Xin2016-05-191-3/+14
| | | | | | | | | | | | | | | | config options to existing sessions if specified ## What changes were proposed in this pull request? Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that. This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession. ## How was this patch tested? Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches. Author: Reynold Xin <rxin@databricks.com> Closes #13200 from rxin/SPARK-15075.
* [SPARK-15171][SQL] Remove the references to deprecated method ↵Sean Zhong2016-05-181-1/+1
| | | | | | | | | | | | | | | | | dataset.registerTempTable ## What changes were proposed in this pull request? Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`. ## How was this patch tested? This PR only changes the unit test code, examples, and comments. It should be safe. This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged. Author: Sean Zhong <seanzhong@databricks.com> Closes #13098 from clockfly/spark-15171-remove-deprecation.
* [SPARK-15244] [PYTHON] Type of column name created with createDataFrame is ↵Dongjoon Hyun2016-05-171-0/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | not consistent. ## What changes were proposed in this pull request? **createDataFrame** returns inconsistent types for column names. ```python >>> from pyspark.sql.types import StructType, StructField, StringType >>> schema = StructType([StructField(u"col", StringType())]) >>> df1 = spark.createDataFrame([("a",)], schema) >>> df1.columns # "col" is str ['col'] >>> df2 = spark.createDataFrame([("a",)], [u"col"]) >>> df2.columns # "col" is unicode [u'col'] ``` The reason is only **StructField** has the following code. ``` if not isinstance(name, str): name = name.encode('utf-8') ``` This PR adds the same logic into **createDataFrame** for consistency. ``` if isinstance(schema, list): schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] ``` ## How was this patch tested? Pass the Jenkins test (with new python doctest) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13097 from dongjoon-hyun/SPARK-15244.
* [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempViewSean Zhong2016-05-121-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView. ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #12945 from clockfly/spark-15171.
* [SPARK-15072][SQL][PYSPARK] FollowUp: Remove SparkSession.withHiveSupport in ↵Sandeep Singh2016-05-111-10/+0
| | | | | | | | | | | | | | | PySpark ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/12851 Remove `SparkSession.withHiveSupport` in PySpark and instead use `SparkSession.builder. enableHiveSupport` ## How was this patch tested? Existing tests. Author: Sandeep Singh <sandeep@techaddict.me> Closes #13063 from techaddict/SPARK-15072-followup.
* [SPARK-15126][SQL] RuntimeConfig.set should return UnitReynold Xin2016-05-041-3/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we return RuntimeConfig itself to facilitate chaining. However, it makes the output in interactive environments (e.g. notebooks, scala repl) weird because it'd show the response of calling set as a RuntimeConfig itself. ## How was this patch tested? Updated unit tests. Author: Reynold Xin <rxin@databricks.com> Closes #12902 from rxin/SPARK-15126.
* [SPARK-15084][PYTHON][SQL] Use builder pattern to create SparkSession in ↵Dongjoon Hyun2016-05-031-1/+90
| | | | | | | | | | | | | | | | PySpark. ## What changes were proposed in this pull request? This is a python port of corresponding Scala builder pattern code. `sql.py` is modified as a target example case. ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12860 from dongjoon-hyun/SPARK-15084.
* [SPARK-15012][SQL] Simplify configuration API furtherAndrew Or2016-04-291-29/+0
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. Remove all the `spark.setConf` etc. Just expose `spark.conf` 2. Make `spark.conf` take in things set in the core `SparkConf` as well, otherwise users may get confused This was done for both the Python and Scala APIs. ## How was this patch tested? `SQLConfSuite`, python tests. This one fixes the failed tests in #12787 Closes #12787 Author: Andrew Or <andrew@databricks.com> Author: Yin Huai <yhuai@databricks.com> Closes #12798 from yhuai/conf-api.
* [SPARK-14988][PYTHON] SparkSession API follow-upsAndrew Or2016-04-291-2/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Addresses comments in #12765. ## How was this patch tested? Python tests. Author: Andrew Or <andrew@databricks.com> Closes #12784 from andrewor14/python-followup.
* [SPARK-14988][PYTHON] SparkSession catalog and conf APIAndrew Or2016-04-291-80/+59
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The `catalog` and `conf` APIs were exposed in `SparkSession` in #12713 and #12669. This patch adds those to the python API. ## How was this patch tested? Python tests. Author: Andrew Or <andrew@databricks.com> Closes #12765 from andrewor14/python-spark-session-more.
* [SPARK-14945][PYTHON] SparkSession Python APIAndrew Or2016-04-281-0/+525
## What changes were proposed in this pull request? ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /__ / .__/\_,_/_/ /_/\_\ version 2.0.0-SNAPSHOT /_/ Using Python version 2.7.5 (default, Mar 9 2014 22:15:05) SparkSession available as 'spark'. >>> spark <pyspark.sql.session.SparkSession object at 0x101f3bfd0> >>> spark.sql("SHOW TABLES").show() ... +---------+-----------+ |tableName|isTemporary| +---------+-----------+ | src| false| +---------+-----------+ >>> spark.range(1, 10, 2).show() +---+ | id| +---+ | 1| | 3| | 5| | 7| | 9| +---+ ``` **Note**: This API is NOT complete in its current state. In particular, for now I left out the `conf` and `catalog` APIs, which were added later in Scala. These will be added later before 2.0. ## How was this patch tested? Python tests. Author: Andrew Or <andrew@databricks.com> Closes #12746 from andrewor14/python-spark-session.