2015-02-09 23:49:22 -05:00
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
|
|
|
# this work for additional information regarding copyright ownership.
|
|
|
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
|
|
|
# (the "License"); you may not use this file except in compliance with
|
|
|
|
# the License. You may obtain a copy of the License at
|
|
|
|
#
|
|
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
#
|
|
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
# See the License for the specific language governing permissions and
|
|
|
|
# limitations under the License.
|
|
|
|
#
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
import sys
|
2016-05-04 20:39:30 -04:00
|
|
|
import warnings
|
2015-04-16 19:20:57 -04:00
|
|
|
|
2018-03-18 07:24:14 -04:00
|
|
|
from pyspark import since, _NoValue
|
2016-04-28 13:55:48 -04:00
|
|
|
from pyspark.sql.session import _monkey_patch_RDD, SparkSession
|
2015-02-09 23:49:22 -05:00
|
|
|
from pyspark.sql.dataframe import DataFrame
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark.sql.readwriter import DataFrameReader
|
|
|
|
from pyspark.sql.streaming import DataStreamReader
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql.udf import UDFRegistration # noqa: F401
|
2015-06-30 19:17:46 -04:00
|
|
|
from pyspark.sql.utils import install_exception_handler
|
2015-02-11 15:13:16 -05:00
|
|
|
|
2020-03-27 02:51:15 -04:00
|
|
|
__all__ = ["SQLContext", "HiveContext"]
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
|
|
|
|
class SQLContext(object):
|
2016-05-20 00:53:26 -04:00
|
|
|
"""The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x.
|
|
|
|
|
|
|
|
As of Spark 2.0, this is replaced by :class:`SparkSession`. However, we are keeping the class
|
|
|
|
here for backward compatibility.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as
|
2015-02-09 23:49:22 -05:00
|
|
|
tables, execute SQL over tables, cache tables, and read parquet files.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 3.0.0
|
|
|
|
Use :func:`SparkSession.builder.getOrCreate()` instead.
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
sparkContext : :class:`SparkContext`
|
|
|
|
The :class:`SparkContext` backing this SQLContext.
|
|
|
|
sparkSession : :class:`SparkSession`
|
|
|
|
The :class:`SparkSession` around which this SQLContext wraps.
|
|
|
|
jsqlContext : optional
|
|
|
|
An optional JVM Scala SQLContext. If set, we do not instantiate a new
|
2015-02-09 23:49:22 -05:00
|
|
|
SQLContext in the JVM, instead we make all calls to this object.
|
2020-11-02 20:00:49 -05:00
|
|
|
This is only for internal.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
|
|
|
>>> from datetime import datetime
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> sqlContext = SQLContext(sc)
|
|
|
|
>>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1,
|
|
|
|
... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1),
|
|
|
|
... time=datetime(2014, 8, 1, 14, 1, 5))])
|
|
|
|
>>> df = allTypes.toDF()
|
|
|
|
>>> df.createOrReplaceTempView("allTypes")
|
|
|
|
>>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a '
|
|
|
|
... 'from allTypes where b and i > 0').collect()
|
|
|
|
[Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \
|
|
|
|
dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)]
|
|
|
|
>>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect()
|
|
|
|
[(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
|
2015-03-31 21:31:36 -04:00
|
|
|
"""
|
|
|
|
|
2015-10-19 19:18:20 -04:00
|
|
|
_instantiatedContext = None
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
def __init__(self, sparkContext, sparkSession=None, jsqlContext=None):
|
2020-09-16 13:13:47 -04:00
|
|
|
if sparkSession is None:
|
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.",
|
|
|
|
DeprecationWarning)
|
[SPARK-30861][PYTHON][SQL] Deprecate constructor of SQLContext and getOrCreate in SQLContext at PySpark
### What changes were proposed in this pull request?
This PR proposes to deprecate the APIs at `SQLContext` removed in SPARK-25908. We should remove equivalent APIs; however, seems we missed to deprecate.
While I am here, I fix one more issue. After SPARK-25908, `sc._jvm.SQLContext.getOrCreate` dose not exist anymore. So,
```python
from pyspark.sql import SQLContext
from pyspark import SparkContext
sc = SparkContext.getOrCreate()
SQLContext.getOrCreate(sc).range(10).show()
```
throws an exception as below:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/context.py", line 110, in getOrCreate
jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc())
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1516, in __getattr__
py4j.protocol.Py4JError: org.apache.spark.sql.SQLContext.getOrCreate does not exist in the JVM
```
After this PR:
```
/.../spark/python/pyspark/sql/context.py:113: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
+---+
| id|
+---+
| 0|
| 1|
| 2|
| 3|
| 4|
| 5|
| 6|
| 7|
| 8|
| 9|
+---+
```
In case of the constructor of `SQLContext`, after this PR:
```python
from pyspark.sql import SQLContext
sc = SparkContext.getOrCreate()
SQLContext(sc)
```
```
/.../spark/python/pyspark/sql/context.py:77: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
```
### Why are the changes needed?
To promote to use SparkSession, and keep the API party consistent with Scala side.
### Does this PR introduce any user-facing change?
Yes, it will show deprecation warning to users.
### How was this patch tested?
Manually tested as described above. Unittests were also added.
Closes #27614 from HyukjinKwon/SPARK-30861.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 21:17:47 -05:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
self._sc = sparkContext
|
|
|
|
self._jsc = self._sc._jsc
|
|
|
|
self._jvm = self._sc._jvm
|
2016-04-28 13:55:48 -04:00
|
|
|
if sparkSession is None:
|
2017-01-13 05:35:12 -05:00
|
|
|
sparkSession = SparkSession.builder.getOrCreate()
|
2016-04-28 13:55:48 -04:00
|
|
|
if jsqlContext is None:
|
|
|
|
jsqlContext = sparkSession._jwrapped
|
|
|
|
self.sparkSession = sparkSession
|
|
|
|
self._jsqlContext = jsqlContext
|
|
|
|
_monkey_patch_RDD(self.sparkSession)
|
2015-06-30 19:17:46 -04:00
|
|
|
install_exception_handler()
|
2020-02-19 22:21:24 -05:00
|
|
|
if (SQLContext._instantiatedContext is None
|
|
|
|
or SQLContext._instantiatedContext._sc._jsc is None):
|
2015-10-19 19:18:20 -04:00
|
|
|
SQLContext._instantiatedContext = self
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
@property
|
|
|
|
def _ssql_ctx(self):
|
|
|
|
"""Accessor for the JVM Spark SQL context.
|
|
|
|
|
|
|
|
Subclasses can override this property to provide their own
|
|
|
|
JVM Contexts.
|
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self._jsqlContext
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2018-07-02 02:35:37 -04:00
|
|
|
@property
|
|
|
|
def _conf(self):
|
|
|
|
"""Accessor for the JVM SQL-specific configurations"""
|
|
|
|
return self.sparkSession._jsparkSession.sessionState().conf()
|
|
|
|
|
2015-10-19 19:18:20 -04:00
|
|
|
@classmethod
|
|
|
|
def getOrCreate(cls, sc):
|
|
|
|
"""
|
|
|
|
Get the existing SQLContext or create a new one with given SparkContext.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
[SPARK-30861][PYTHON][SQL] Deprecate constructor of SQLContext and getOrCreate in SQLContext at PySpark
### What changes were proposed in this pull request?
This PR proposes to deprecate the APIs at `SQLContext` removed in SPARK-25908. We should remove equivalent APIs; however, seems we missed to deprecate.
While I am here, I fix one more issue. After SPARK-25908, `sc._jvm.SQLContext.getOrCreate` dose not exist anymore. So,
```python
from pyspark.sql import SQLContext
from pyspark import SparkContext
sc = SparkContext.getOrCreate()
SQLContext.getOrCreate(sc).range(10).show()
```
throws an exception as below:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/context.py", line 110, in getOrCreate
jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc())
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1516, in __getattr__
py4j.protocol.Py4JError: org.apache.spark.sql.SQLContext.getOrCreate does not exist in the JVM
```
After this PR:
```
/.../spark/python/pyspark/sql/context.py:113: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
+---+
| id|
+---+
| 0|
| 1|
| 2|
| 3|
| 4|
| 5|
| 6|
| 7|
| 8|
| 9|
+---+
```
In case of the constructor of `SQLContext`, after this PR:
```python
from pyspark.sql import SQLContext
sc = SparkContext.getOrCreate()
SQLContext(sc)
```
```
/.../spark/python/pyspark/sql/context.py:77: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
```
### Why are the changes needed?
To promote to use SparkSession, and keep the API party consistent with Scala side.
### Does this PR introduce any user-facing change?
Yes, it will show deprecation warning to users.
### How was this patch tested?
Manually tested as described above. Unittests were also added.
Closes #27614 from HyukjinKwon/SPARK-30861.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 21:17:47 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 3.0.0
|
|
|
|
Use :func:`SparkSession.builder.getOrCreate()` instead.
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
sc : :class:`SparkContext`
|
2015-10-19 19:18:20 -04:00
|
|
|
"""
|
[SPARK-30861][PYTHON][SQL] Deprecate constructor of SQLContext and getOrCreate in SQLContext at PySpark
### What changes were proposed in this pull request?
This PR proposes to deprecate the APIs at `SQLContext` removed in SPARK-25908. We should remove equivalent APIs; however, seems we missed to deprecate.
While I am here, I fix one more issue. After SPARK-25908, `sc._jvm.SQLContext.getOrCreate` dose not exist anymore. So,
```python
from pyspark.sql import SQLContext
from pyspark import SparkContext
sc = SparkContext.getOrCreate()
SQLContext.getOrCreate(sc).range(10).show()
```
throws an exception as below:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/context.py", line 110, in getOrCreate
jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc())
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1516, in __getattr__
py4j.protocol.Py4JError: org.apache.spark.sql.SQLContext.getOrCreate does not exist in the JVM
```
After this PR:
```
/.../spark/python/pyspark/sql/context.py:113: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
+---+
| id|
+---+
| 0|
| 1|
| 2|
| 3|
| 4|
| 5|
| 6|
| 7|
| 8|
| 9|
+---+
```
In case of the constructor of `SQLContext`, after this PR:
```python
from pyspark.sql import SQLContext
sc = SparkContext.getOrCreate()
SQLContext(sc)
```
```
/.../spark/python/pyspark/sql/context.py:77: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
```
### Why are the changes needed?
To promote to use SparkSession, and keep the API party consistent with Scala side.
### Does this PR introduce any user-facing change?
Yes, it will show deprecation warning to users.
### How was this patch tested?
Manually tested as described above. Unittests were also added.
Closes #27614 from HyukjinKwon/SPARK-30861.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 21:17:47 -05:00
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.",
|
|
|
|
DeprecationWarning)
|
|
|
|
|
2020-02-19 22:21:24 -05:00
|
|
|
if (cls._instantiatedContext is None
|
|
|
|
or SQLContext._instantiatedContext._sc._jsc is None):
|
[SPARK-30861][PYTHON][SQL] Deprecate constructor of SQLContext and getOrCreate in SQLContext at PySpark
### What changes were proposed in this pull request?
This PR proposes to deprecate the APIs at `SQLContext` removed in SPARK-25908. We should remove equivalent APIs; however, seems we missed to deprecate.
While I am here, I fix one more issue. After SPARK-25908, `sc._jvm.SQLContext.getOrCreate` dose not exist anymore. So,
```python
from pyspark.sql import SQLContext
from pyspark import SparkContext
sc = SparkContext.getOrCreate()
SQLContext.getOrCreate(sc).range(10).show()
```
throws an exception as below:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/context.py", line 110, in getOrCreate
jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc())
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1516, in __getattr__
py4j.protocol.Py4JError: org.apache.spark.sql.SQLContext.getOrCreate does not exist in the JVM
```
After this PR:
```
/.../spark/python/pyspark/sql/context.py:113: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
+---+
| id|
+---+
| 0|
| 1|
| 2|
| 3|
| 4|
| 5|
| 6|
| 7|
| 8|
| 9|
+---+
```
In case of the constructor of `SQLContext`, after this PR:
```python
from pyspark.sql import SQLContext
sc = SparkContext.getOrCreate()
SQLContext(sc)
```
```
/.../spark/python/pyspark/sql/context.py:77: DeprecationWarning: Deprecated in 3.0.0. Use SparkSession.builder.getOrCreate() instead.
DeprecationWarning)
```
### Why are the changes needed?
To promote to use SparkSession, and keep the API party consistent with Scala side.
### Does this PR introduce any user-facing change?
Yes, it will show deprecation warning to users.
### How was this patch tested?
Manually tested as described above. Unittests were also added.
Closes #27614 from HyukjinKwon/SPARK-30861.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 21:17:47 -05:00
|
|
|
jsqlContext = sc._jvm.SparkSession.builder().sparkContext(
|
|
|
|
sc._jsc.sc()).getOrCreate().sqlContext()
|
2016-04-28 13:55:48 -04:00
|
|
|
sparkSession = SparkSession(sc, jsqlContext.sparkSession())
|
|
|
|
cls(sc, sparkSession, jsqlContext)
|
2015-10-19 19:18:20 -04:00
|
|
|
return cls._instantiatedContext
|
|
|
|
|
|
|
|
def newSession(self):
|
|
|
|
"""
|
|
|
|
Returns a new SQLContext as new session, that has separate SQLConf,
|
2016-05-17 21:01:59 -04:00
|
|
|
registered temporary views and UDFs, but shared SparkContext and
|
2015-10-19 19:18:20 -04:00
|
|
|
table cache.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
.. versionadded:: 1.6.0
|
2015-10-19 19:18:20 -04:00
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self.__class__(self._sc, self.sparkSession.newSession())
|
2015-10-19 19:18:20 -04:00
|
|
|
|
2015-02-10 20:29:52 -05:00
|
|
|
def setConf(self, key, value):
|
|
|
|
"""Sets the given Spark SQL configuration property.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
.. versionadded:: 1.3.0
|
2015-02-10 20:29:52 -05:00
|
|
|
"""
|
2016-04-29 23:46:07 -04:00
|
|
|
self.sparkSession.conf.set(key, value)
|
2015-02-10 20:29:52 -05:00
|
|
|
|
2018-03-18 07:24:14 -04:00
|
|
|
def getConf(self, key, defaultValue=_NoValue):
|
2015-02-10 20:29:52 -05:00
|
|
|
"""Returns the value of Spark SQL configuration property for the given key.
|
|
|
|
|
2018-03-18 07:24:14 -04:00
|
|
|
If the key is not set and defaultValue is set, return
|
|
|
|
defaultValue. If the key is not set and defaultValue is not set, return
|
2016-04-24 01:38:36 -04:00
|
|
|
the system default value.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-04-24 01:38:36 -04:00
|
|
|
>>> sqlContext.getConf("spark.sql.shuffle.partitions")
|
2020-07-13 22:22:44 -04:00
|
|
|
'200'
|
|
|
|
>>> sqlContext.getConf("spark.sql.shuffle.partitions", "10")
|
|
|
|
'10'
|
|
|
|
>>> sqlContext.setConf("spark.sql.shuffle.partitions", "50")
|
|
|
|
>>> sqlContext.getConf("spark.sql.shuffle.partitions", "10")
|
|
|
|
'50'
|
2015-02-10 20:29:52 -05:00
|
|
|
"""
|
2016-04-29 23:46:07 -04:00
|
|
|
return self.sparkSession.conf.get(key, defaultValue)
|
2015-02-10 20:29:52 -05:00
|
|
|
|
2015-03-30 18:47:00 -04:00
|
|
|
@property
|
|
|
|
def udf(self):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Returns a :class:`UDFRegistration` for UDF registration.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.1
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`UDFRegistration`
|
2015-06-03 03:23:34 -04:00
|
|
|
"""
|
2018-01-18 00:51:05 -05:00
|
|
|
return self.sparkSession.udf
|
2015-03-30 18:47:00 -04:00
|
|
|
|
2015-06-03 14:28:18 -04:00
|
|
|
def range(self, start, end=None, step=1, numPartitions=None):
|
2015-05-19 00:43:12 -04:00
|
|
|
"""
|
2016-07-28 17:57:15 -04:00
|
|
|
Create a :class:`DataFrame` with single :class:`pyspark.sql.types.LongType` column named
|
|
|
|
``id``, containing elements in a range from ``start`` to ``end`` (exclusive) with
|
|
|
|
step value ``step``.
|
2015-05-19 00:43:12 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
start : int
|
|
|
|
the start value
|
|
|
|
end : int, optional
|
|
|
|
the end value (exclusive)
|
|
|
|
step : int, optional
|
|
|
|
the incremental step (default: 1)
|
|
|
|
numPartitions : int, optional
|
|
|
|
the number of partitions of the DataFrame
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-05-19 00:43:12 -04:00
|
|
|
>>> sqlContext.range(1, 7, 2).collect()
|
|
|
|
[Row(id=1), Row(id=3), Row(id=5)]
|
2015-06-03 14:28:18 -04:00
|
|
|
|
2015-06-03 17:19:10 -04:00
|
|
|
If only one argument is specified, it will be used as the end value.
|
|
|
|
|
2015-06-03 14:28:18 -04:00
|
|
|
>>> sqlContext.range(3).collect()
|
|
|
|
[Row(id=0), Row(id=1), Row(id=2)]
|
2015-05-19 00:43:12 -04:00
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self.sparkSession.range(start, end, step, numPartitions)
|
2015-05-19 00:43:12 -04:00
|
|
|
|
2018-01-16 06:20:33 -05:00
|
|
|
def registerFunction(self, name, f, returnType=None):
|
2018-01-18 00:51:05 -05:00
|
|
|
"""An alias for :func:`spark.udf.register`.
|
|
|
|
See :meth:`pyspark.sql.UDFRegistration.register`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.2.0
|
|
|
|
|
|
|
|
.. deprecated:: 2.3.0
|
|
|
|
Use :func:`spark.udf.register` instead.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2018-01-18 00:51:05 -05:00
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 2.3.0. Use spark.udf.register instead.",
|
|
|
|
DeprecationWarning)
|
|
|
|
return self.sparkSession.udf.register(name, f, returnType)
|
2015-06-29 16:20:55 -04:00
|
|
|
|
2016-10-14 18:50:35 -04:00
|
|
|
def registerJavaFunction(self, name, javaClassName, returnType=None):
|
2018-01-18 00:51:05 -05:00
|
|
|
"""An alias for :func:`spark.udf.registerJavaFunction`.
|
|
|
|
See :meth:`pyspark.sql.UDFRegistration.registerJavaFunction`.
|
2016-10-14 18:50:35 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
.. deprecated:: 2.3.0
|
|
|
|
Use :func:`spark.udf.registerJavaFunction` instead.
|
2016-10-14 18:50:35 -04:00
|
|
|
"""
|
2018-01-18 00:51:05 -05:00
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 2.3.0. Use spark.udf.registerJavaFunction instead.",
|
|
|
|
DeprecationWarning)
|
|
|
|
return self.sparkSession.udf.registerJavaFunction(name, javaClassName, returnType)
|
2017-07-05 13:59:10 -04:00
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
# TODO(andrew): delete this once we refactor things to take in SparkSession
|
2015-02-11 15:13:16 -05:00
|
|
|
def _inferSchema(self, rdd, samplingRatio=None):
|
2015-06-29 16:20:55 -04:00
|
|
|
"""
|
|
|
|
Infer schema from an RDD of Row or tuple.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
rdd : :class:`RDD`
|
|
|
|
an RDD of Row or tuple
|
|
|
|
samplingRatio : float, optional
|
|
|
|
sampling ratio, or no sampling (default)
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`pyspark.sql.types.StructType`
|
2015-06-29 16:20:55 -04:00
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self.sparkSession._inferSchema(rdd, samplingRatio)
|
2015-07-30 01:30:49 -04:00
|
|
|
|
2016-08-15 15:41:27 -04:00
|
|
|
def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True):
|
2015-02-10 22:40:12 -05:00
|
|
|
"""
|
2016-03-08 17:00:03 -05:00
|
|
|
Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`.
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
When ``schema`` is a list of column names, the type of each column
|
|
|
|
will be inferred from ``data``.
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
When ``schema`` is ``None``, it will try to infer the schema (column names and types)
|
|
|
|
from ``data``, which should be an RDD of :class:`Row`,
|
|
|
|
or :class:`namedtuple`, or :class:`dict`.
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2016-07-29 17:07:03 -04:00
|
|
|
When ``schema`` is :class:`pyspark.sql.types.DataType` or a datatype string it must match
|
|
|
|
the real data, or an exception will be thrown at runtime. If the given schema is not
|
2016-07-28 17:57:15 -04:00
|
|
|
:class:`pyspark.sql.types.StructType`, it will be wrapped into a
|
|
|
|
:class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value",
|
|
|
|
each record will also be wrapped into a tuple, which can be converted to row later.
|
2016-03-08 17:00:03 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
If schema inference is needed, ``samplingRatio`` is used to determined the ratio of
|
|
|
|
rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``.
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
.. versionchanged:: 2.0.0
|
|
|
|
The ``schema`` parameter can be a :class:`pyspark.sql.types.DataType` or a
|
|
|
|
datatype string after 2.0.
|
|
|
|
If it's not a :class:`pyspark.sql.types.StructType`, it will be wrapped into a
|
|
|
|
:class:`pyspark.sql.types.StructType` and each record will also be wrapped into a tuple.
|
|
|
|
|
|
|
|
.. versionchanged:: 2.1.0
|
|
|
|
Added verifySchema.
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
data : :class:`RDD` or iterable
|
|
|
|
an RDD of any kind of SQL data representation(e.g. :class:`Row`,
|
2016-07-28 17:57:15 -04:00
|
|
|
:class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, or
|
|
|
|
:class:`pandas.DataFrame`.
|
2020-11-02 20:00:49 -05:00
|
|
|
schema : :class:`pyspark.sql.types.DataType`, str or list, optional
|
|
|
|
a :class:`pyspark.sql.types.DataType` or a datatype string or a list of
|
2016-07-28 17:57:15 -04:00
|
|
|
column names, default is None. The data type string format equals to
|
|
|
|
:class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can
|
|
|
|
omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use
|
|
|
|
``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`.
|
|
|
|
We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`.
|
2020-11-02 20:00:49 -05:00
|
|
|
samplingRatio : float, optional
|
|
|
|
the sample ratio of rows used for inferring
|
|
|
|
verifySchema : bool, optional
|
|
|
|
verify data types of every row against schema. Enabled by default.
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
2016-08-15 15:41:27 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-02-11 15:13:16 -05:00
|
|
|
>>> l = [('Alice', 1)]
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.createDataFrame(l).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(_1='Alice', _2=1)]
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.createDataFrame(l, ['name', 'age']).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2015-02-11 15:13:16 -05:00
|
|
|
|
|
|
|
>>> d = [{'name': 'Alice', 'age': 1}]
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.createDataFrame(d).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(age=1, name='Alice')]
|
2015-02-11 15:13:16 -05:00
|
|
|
|
|
|
|
>>> rdd = sc.parallelize(l)
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.createDataFrame(rdd).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(_1='Alice', _2=1)]
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> df = sqlContext.createDataFrame(rdd, ['name', 'age'])
|
2015-02-10 22:40:12 -05:00
|
|
|
>>> df.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2015-02-10 22:40:12 -05:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> Person = Row('name', 'age')
|
|
|
|
>>> person = rdd.map(lambda r: Person(*r))
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> df2 = sqlContext.createDataFrame(person)
|
2015-02-10 22:40:12 -05:00
|
|
|
>>> df2.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2015-02-10 22:40:12 -05:00
|
|
|
|
|
|
|
>>> from pyspark.sql.types import *
|
|
|
|
>>> schema = StructType([
|
|
|
|
... StructField("name", StringType(), True),
|
|
|
|
... StructField("age", IntegerType(), True)])
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> df3 = sqlContext.createDataFrame(rdd, schema)
|
2015-02-10 22:40:12 -05:00
|
|
|
>>> df3.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2015-02-20 18:35:05 -05:00
|
|
|
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.createDataFrame(df.toPandas()).collect() # doctest: +SKIP
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2015-11-12 18:42:30 -05:00
|
|
|
>>> sqlContext.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP
|
2015-06-30 23:35:46 -04:00
|
|
|
[Row(0=1, 1=2)]
|
2016-03-08 17:00:03 -05:00
|
|
|
|
|
|
|
>>> sqlContext.createDataFrame(rdd, "a: string, b: int").collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(a='Alice', b=1)]
|
2016-03-08 17:00:03 -05:00
|
|
|
>>> rdd = rdd.map(lambda row: row[1])
|
|
|
|
>>> sqlContext.createDataFrame(rdd, "int").collect()
|
|
|
|
[Row(value=1)]
|
|
|
|
>>> sqlContext.createDataFrame(rdd, "boolean").collect() # doctest: +IGNORE_EXCEPTION_DETAIL
|
|
|
|
Traceback (most recent call last):
|
|
|
|
...
|
2016-04-14 04:42:15 -04:00
|
|
|
Py4JJavaError: ...
|
2015-02-10 22:40:12 -05:00
|
|
|
"""
|
2016-08-15 15:41:27 -04:00
|
|
|
return self.sparkSession.createDataFrame(data, schema, samplingRatio, verifySchema)
|
2015-02-10 22:40:12 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
def registerDataFrameAsTable(self, df, tableName):
|
|
|
|
"""Registers the given :class:`DataFrame` as a temporary table in the catalog.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2016-05-12 03:51:53 -04:00
|
|
|
df.createOrReplaceTempView(tableName)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-11-26 22:15:22 -05:00
|
|
|
def dropTempTable(self, tableName):
|
2019-11-01 14:55:29 -04:00
|
|
|
""" Remove the temporary table from catalog.
|
2015-11-26 22:15:22 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-11-26 22:15:22 -05:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
|
|
|
>>> sqlContext.dropTempTable("table1")
|
|
|
|
"""
|
2016-05-12 03:51:53 -04:00
|
|
|
self.sparkSession.catalog.dropTempView(tableName)
|
2015-11-26 22:15:22 -05:00
|
|
|
|
2020-03-27 02:51:15 -04:00
|
|
|
def createExternalTable(self, tableName, path=None, source=None, schema=None, **options):
|
|
|
|
"""Creates an external table based on the dataset in a data source.
|
|
|
|
|
|
|
|
It returns the DataFrame associated with the external table.
|
|
|
|
|
|
|
|
The data source is specified by the ``source`` and a set of ``options``.
|
|
|
|
If ``source`` is not specified, the default data source configured by
|
|
|
|
``spark.sql.sources.default`` will be used.
|
|
|
|
|
|
|
|
Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and
|
|
|
|
created external table.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
2020-03-27 02:51:15 -04:00
|
|
|
"""
|
|
|
|
return self.sparkSession.catalog.createExternalTable(
|
|
|
|
tableName, path, source, schema, **options)
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
def sql(self, sqlQuery):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a :class:`DataFrame` representing the result of the given query.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.0.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
|
|
|
>>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1")
|
2015-02-09 23:49:22 -05:00
|
|
|
>>> df2.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self.sparkSession.sql(sqlQuery)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
def table(self, tableName):
|
2017-04-06 12:09:43 -04:00
|
|
|
"""Returns the specified table or view as a :class:`DataFrame`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.0.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
|
|
|
>>> df2 = sqlContext.table("table1")
|
2015-02-09 23:49:22 -05:00
|
|
|
>>> sorted(df.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
"""
|
2016-04-28 13:55:48 -04:00
|
|
|
return self.sparkSession.table(tableName)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-02-12 21:08:01 -05:00
|
|
|
def tables(self, dbName=None):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a :class:`DataFrame` containing names of tables in the given database.
|
2015-02-12 21:08:01 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
If ``dbName`` is not specified, the current database will be used.
|
2015-02-12 21:08:01 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
The returned DataFrame has two columns: ``tableName`` and ``isTemporary``
|
|
|
|
(a column with :class:`BooleanType` indicating if a table is a temporary one or not).
|
2015-02-12 21:08:01 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
dbName: str, optional
|
|
|
|
name of the database to use.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
|
|
|
>>> df2 = sqlContext.tables()
|
2015-02-12 21:08:01 -05:00
|
|
|
>>> df2.filter("tableName = 'table1'").first()
|
2020-07-13 22:22:44 -04:00
|
|
|
Row(database='', tableName='table1', isTemporary=True)
|
2015-02-12 21:08:01 -05:00
|
|
|
"""
|
|
|
|
if dbName is None:
|
|
|
|
return DataFrame(self._ssql_ctx.tables(), self)
|
|
|
|
else:
|
|
|
|
return DataFrame(self._ssql_ctx.tables(dbName), self)
|
|
|
|
|
|
|
|
def tableNames(self, dbName=None):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a list of names of tables in the database ``dbName``.
|
2015-02-12 21:08:01 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
dbName: str
|
|
|
|
name of the database to use. Default to the current database.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
list
|
|
|
|
list of table names, in string
|
2015-02-12 21:08:01 -05:00
|
|
|
|
2015-04-08 16:31:45 -04:00
|
|
|
>>> sqlContext.registerDataFrameAsTable(df, "table1")
|
|
|
|
>>> "table1" in sqlContext.tableNames()
|
2015-02-12 21:08:01 -05:00
|
|
|
True
|
2016-03-25 01:59:35 -04:00
|
|
|
>>> "table1" in sqlContext.tableNames("default")
|
2015-02-12 21:08:01 -05:00
|
|
|
True
|
|
|
|
"""
|
|
|
|
if dbName is None:
|
|
|
|
return [name for name in self._ssql_ctx.tableNames()]
|
|
|
|
else:
|
|
|
|
return [name for name in self._ssql_ctx.tableNames(dbName)]
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.0)
|
2015-02-09 23:49:22 -05:00
|
|
|
def cacheTable(self, tableName):
|
|
|
|
"""Caches the specified table in-memory."""
|
|
|
|
self._ssql_ctx.cacheTable(tableName)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.0)
|
2015-02-09 23:49:22 -05:00
|
|
|
def uncacheTable(self, tableName):
|
|
|
|
"""Removes the specified table from the in-memory cache."""
|
|
|
|
self._ssql_ctx.uncacheTable(tableName)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-20 03:20:02 -05:00
|
|
|
def clearCache(self):
|
|
|
|
"""Removes all cached tables from the in-memory cache. """
|
|
|
|
self._ssql_ctx.clearCache()
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
@property
|
|
|
|
def read(self):
|
|
|
|
"""
|
|
|
|
Returns a :class:`DataFrameReader` that can be used to read data
|
|
|
|
in as a :class:`DataFrame`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrameReader`
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
return DataFrameReader(self)
|
|
|
|
|
2016-06-14 20:58:45 -04:00
|
|
|
@property
|
|
|
|
def readStream(self):
|
|
|
|
"""
|
|
|
|
Returns a :class:`DataStreamReader` that can be used to read data streams
|
|
|
|
as a streaming :class:`DataFrame`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-14 20:58:45 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataStreamReader`
|
2016-06-28 21:33:37 -04:00
|
|
|
|
2016-06-30 19:51:11 -04:00
|
|
|
>>> text_sdf = sqlContext.readStream.text(tempfile.mkdtemp())
|
2016-06-28 21:33:37 -04:00
|
|
|
>>> text_sdf.isStreaming
|
|
|
|
True
|
2016-06-14 20:58:45 -04:00
|
|
|
"""
|
2016-06-28 21:33:37 -04:00
|
|
|
return DataStreamReader(self)
|
2016-06-14 20:58:45 -04:00
|
|
|
|
2016-04-28 18:22:28 -04:00
|
|
|
@property
|
|
|
|
def streams(self):
|
2016-06-15 13:46:02 -04:00
|
|
|
"""Returns a :class:`StreamingQueryManager` that allows managing all the
|
|
|
|
:class:`StreamingQuery` StreamingQueries active on `this` context.
|
2016-06-14 05:12:29 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
from pyspark.sql.streaming import StreamingQueryManager
|
|
|
|
return StreamingQueryManager(self._ssql_ctx.streams())
|
2016-04-28 18:22:28 -04:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2020-03-27 02:51:15 -04:00
|
|
|
class HiveContext(SQLContext):
|
|
|
|
"""A variant of Spark SQL that integrates with data stored in Hive.
|
|
|
|
|
|
|
|
Configuration for Hive is read from ``hive-site.xml`` on the classpath.
|
|
|
|
It supports running both SQL and HiveQL commands.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 2.0.0
|
|
|
|
Use SparkSession.builder.enableHiveSupport().getOrCreate().
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
sparkContext : :class:`SparkContext`
|
|
|
|
The SparkContext to wrap.
|
|
|
|
jhiveContext : optional
|
|
|
|
An optional JVM Scala HiveContext. If set, we do not instantiate a new
|
2020-03-27 02:51:15 -04:00
|
|
|
:class:`HiveContext` in the JVM, instead we make all calls to this object.
|
2020-11-02 20:00:49 -05:00
|
|
|
This is only for internal use.
|
2020-03-27 02:51:15 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, sparkContext, jhiveContext=None):
|
|
|
|
warnings.warn(
|
|
|
|
"HiveContext is deprecated in Spark 2.0.0. Please use " +
|
|
|
|
"SparkSession.builder.enableHiveSupport().getOrCreate() instead.",
|
|
|
|
DeprecationWarning)
|
|
|
|
if jhiveContext is None:
|
|
|
|
sparkContext._conf.set("spark.sql.catalogImplementation", "hive")
|
|
|
|
sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate()
|
|
|
|
else:
|
|
|
|
sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession())
|
|
|
|
SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext)
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def _createForTesting(cls, sparkContext):
|
|
|
|
"""(Internal use only) Create a new HiveContext for testing.
|
|
|
|
|
|
|
|
All test code that touches HiveContext *must* go through this method. Otherwise,
|
|
|
|
you may end up launching multiple derby instances and encounter with incredibly
|
|
|
|
confusing error messages.
|
|
|
|
"""
|
|
|
|
jsc = sparkContext._jsc.sc()
|
|
|
|
jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc, False)
|
|
|
|
return cls(sparkContext, jtestHive)
|
|
|
|
|
|
|
|
def refreshTable(self, tableName):
|
|
|
|
"""Invalidate and refresh all the cached the metadata of the given
|
|
|
|
table. For performance reasons, Spark SQL or the external data source
|
|
|
|
library it uses might cache certain metadata about a table, such as the
|
|
|
|
location of blocks. When those change outside of Spark SQL, users should
|
|
|
|
call this function to invalidate the cache.
|
|
|
|
"""
|
|
|
|
self._ssql_ctx.refreshTable(tableName)
|
|
|
|
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
def _test():
|
2015-06-03 03:23:34 -04:00
|
|
|
import os
|
2015-02-09 23:49:22 -05:00
|
|
|
import doctest
|
2016-06-28 21:33:37 -04:00
|
|
|
import tempfile
|
2015-02-09 23:49:22 -05:00
|
|
|
from pyspark.context import SparkContext
|
|
|
|
from pyspark.sql import Row, SQLContext
|
|
|
|
import pyspark.sql.context
|
2015-06-03 03:23:34 -04:00
|
|
|
|
|
|
|
os.chdir(os.environ["SPARK_HOME"])
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
globs = pyspark.sql.context.__dict__.copy()
|
|
|
|
sc = SparkContext('local[4]', 'PythonTest')
|
2016-06-28 21:33:37 -04:00
|
|
|
globs['tempfile'] = tempfile
|
|
|
|
globs['os'] = os
|
2015-02-09 23:49:22 -05:00
|
|
|
globs['sc'] = sc
|
2015-04-08 16:31:45 -04:00
|
|
|
globs['sqlContext'] = SQLContext(sc)
|
2015-02-10 22:40:12 -05:00
|
|
|
globs['rdd'] = rdd = sc.parallelize(
|
2015-02-09 23:49:22 -05:00
|
|
|
[Row(field1=1, field2="row1"),
|
|
|
|
Row(field1=2, field2="row2"),
|
|
|
|
Row(field1=3, field2="row3")]
|
|
|
|
)
|
2015-02-14 02:03:22 -05:00
|
|
|
globs['df'] = rdd.toDF()
|
2015-02-09 23:49:22 -05:00
|
|
|
jsonStrings = [
|
|
|
|
'{"field1": 1, "field2": "row1", "field3":{"field4":11}}',
|
2019-01-17 20:40:39 -05:00
|
|
|
'{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},"field6":[{"field7": "row2"}]}',
|
|
|
|
'{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}'
|
2015-02-09 23:49:22 -05:00
|
|
|
]
|
|
|
|
globs['jsonStrings'] = jsonStrings
|
|
|
|
globs['json'] = sc.parallelize(jsonStrings)
|
|
|
|
(failure_count, test_count) = doctest.testmod(
|
2015-02-24 23:51:55 -05:00
|
|
|
pyspark.sql.context, globs=globs,
|
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
|
2015-02-09 23:49:22 -05:00
|
|
|
globs['sc'].stop()
|
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|