2015-05-15 23:09:15 -04: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.
|
|
|
|
#
|
|
|
|
|
|
|
|
import sys
|
2017-02-14 12:57:43 -05:00
|
|
|
import json
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
if sys.version >= '3':
|
|
|
|
basestring = str
|
|
|
|
long = int
|
|
|
|
|
2016-03-14 22:25:49 -04:00
|
|
|
from pyspark import copy_func, since
|
2015-05-15 23:09:15 -04:00
|
|
|
from pyspark.context import SparkContext
|
|
|
|
from pyspark.rdd import ignore_unicode_prefix
|
|
|
|
from pyspark.sql.types import *
|
|
|
|
|
2017-01-30 12:01:02 -05:00
|
|
|
__all__ = ["Column"]
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
|
|
|
|
def _create_column_from_literal(literal):
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return sc._jvm.functions.lit(literal)
|
|
|
|
|
|
|
|
|
|
|
|
def _create_column_from_name(name):
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return sc._jvm.functions.col(name)
|
|
|
|
|
|
|
|
|
|
|
|
def _to_java_column(col):
|
|
|
|
if isinstance(col, Column):
|
|
|
|
jcol = col._jc
|
[SPARK-19165][PYTHON][SQL] PySpark APIs using columns as arguments should validate input types for column
## What changes were proposed in this pull request?
While preparing to take over https://github.com/apache/spark/pull/16537, I realised a (I think) better approach to make the exception handling in one point.
This PR proposes to fix `_to_java_column` in `pyspark.sql.column`, which most of functions in `functions.py` and some other APIs use. This `_to_java_column` basically looks not working with other types than `pyspark.sql.column.Column` or string (`str` and `unicode`).
If this is not `Column`, then it calls `_create_column_from_name` which calls `functions.col` within JVM:
https://github.com/apache/spark/blob/42b9eda80e975d970c3e8da4047b318b83dd269f/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L76
And it looks we only have `String` one with `col`.
So, these should work:
```python
>>> from pyspark.sql.column import _to_java_column, Column
>>> _to_java_column("a")
JavaObject id=o28
>>> _to_java_column(u"a")
JavaObject id=o29
>>> _to_java_column(spark.range(1).id)
JavaObject id=o33
```
whereas these do not:
```python
>>> _to_java_column(1)
```
```
...
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.col. Trace:
py4j.Py4JException: Method col([class java.lang.Integer]) does not exist
...
```
```python
>>> _to_java_column([])
```
```
...
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.col. Trace:
py4j.Py4JException: Method col([class java.util.ArrayList]) does not exist
...
```
```python
>>> class A(): pass
>>> _to_java_column(A())
```
```
...
AttributeError: 'A' object has no attribute '_get_object_id'
```
Meaning most of functions using `_to_java_column` such as `udf` or `to_json` or some other APIs throw an exception as below:
```python
>>> from pyspark.sql.functions import udf
>>> udf(lambda x: x)(None)
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling z:org.apache.spark.sql.functions.col.
: java.lang.NullPointerException
...
```
```python
>>> from pyspark.sql.functions import to_json
>>> to_json(None)
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling z:org.apache.spark.sql.functions.col.
: java.lang.NullPointerException
...
```
**After this PR**:
```python
>>> from pyspark.sql.functions import udf
>>> udf(lambda x: x)(None)
...
```
```
TypeError: Invalid argument, not a string or column: None of type <type 'NoneType'>. For column literals, use 'lit', 'array', 'struct' or 'create_map' functions.
```
```python
>>> from pyspark.sql.functions import to_json
>>> to_json(None)
```
```
...
TypeError: Invalid argument, not a string or column: None of type <type 'NoneType'>. For column literals, use 'lit', 'array', 'struct' or 'create_map' functions.
```
## How was this patch tested?
Unit tests added in `python/pyspark/sql/tests.py` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Author: zero323 <zero323@users.noreply.github.com>
Closes #19027 from HyukjinKwon/SPARK-19165.
2017-08-24 07:29:03 -04:00
|
|
|
elif isinstance(col, basestring):
|
2015-05-15 23:09:15 -04:00
|
|
|
jcol = _create_column_from_name(col)
|
[SPARK-19165][PYTHON][SQL] PySpark APIs using columns as arguments should validate input types for column
## What changes were proposed in this pull request?
While preparing to take over https://github.com/apache/spark/pull/16537, I realised a (I think) better approach to make the exception handling in one point.
This PR proposes to fix `_to_java_column` in `pyspark.sql.column`, which most of functions in `functions.py` and some other APIs use. This `_to_java_column` basically looks not working with other types than `pyspark.sql.column.Column` or string (`str` and `unicode`).
If this is not `Column`, then it calls `_create_column_from_name` which calls `functions.col` within JVM:
https://github.com/apache/spark/blob/42b9eda80e975d970c3e8da4047b318b83dd269f/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L76
And it looks we only have `String` one with `col`.
So, these should work:
```python
>>> from pyspark.sql.column import _to_java_column, Column
>>> _to_java_column("a")
JavaObject id=o28
>>> _to_java_column(u"a")
JavaObject id=o29
>>> _to_java_column(spark.range(1).id)
JavaObject id=o33
```
whereas these do not:
```python
>>> _to_java_column(1)
```
```
...
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.col. Trace:
py4j.Py4JException: Method col([class java.lang.Integer]) does not exist
...
```
```python
>>> _to_java_column([])
```
```
...
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.col. Trace:
py4j.Py4JException: Method col([class java.util.ArrayList]) does not exist
...
```
```python
>>> class A(): pass
>>> _to_java_column(A())
```
```
...
AttributeError: 'A' object has no attribute '_get_object_id'
```
Meaning most of functions using `_to_java_column` such as `udf` or `to_json` or some other APIs throw an exception as below:
```python
>>> from pyspark.sql.functions import udf
>>> udf(lambda x: x)(None)
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling z:org.apache.spark.sql.functions.col.
: java.lang.NullPointerException
...
```
```python
>>> from pyspark.sql.functions import to_json
>>> to_json(None)
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling z:org.apache.spark.sql.functions.col.
: java.lang.NullPointerException
...
```
**After this PR**:
```python
>>> from pyspark.sql.functions import udf
>>> udf(lambda x: x)(None)
...
```
```
TypeError: Invalid argument, not a string or column: None of type <type 'NoneType'>. For column literals, use 'lit', 'array', 'struct' or 'create_map' functions.
```
```python
>>> from pyspark.sql.functions import to_json
>>> to_json(None)
```
```
...
TypeError: Invalid argument, not a string or column: None of type <type 'NoneType'>. For column literals, use 'lit', 'array', 'struct' or 'create_map' functions.
```
## How was this patch tested?
Unit tests added in `python/pyspark/sql/tests.py` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Author: zero323 <zero323@users.noreply.github.com>
Closes #19027 from HyukjinKwon/SPARK-19165.
2017-08-24 07:29:03 -04:00
|
|
|
else:
|
|
|
|
raise TypeError(
|
|
|
|
"Invalid argument, not a string or column: "
|
|
|
|
"{0} of type {1}. "
|
|
|
|
"For column literals, use 'lit', 'array', 'struct' or 'create_map' "
|
|
|
|
"function.".format(col, type(col)))
|
2015-05-15 23:09:15 -04:00
|
|
|
return jcol
|
|
|
|
|
|
|
|
|
|
|
|
def _to_seq(sc, cols, converter=None):
|
|
|
|
"""
|
|
|
|
Convert a list of Column (or names) into a JVM Seq of Column.
|
|
|
|
|
|
|
|
An optional `converter` could be used to convert items in `cols`
|
|
|
|
into JVM Column objects.
|
|
|
|
"""
|
|
|
|
if converter:
|
|
|
|
cols = [converter(c) for c in cols]
|
|
|
|
return sc._jvm.PythonUtils.toSeq(cols)
|
|
|
|
|
|
|
|
|
2015-08-25 07:33:13 -04:00
|
|
|
def _to_list(sc, cols, converter=None):
|
|
|
|
"""
|
|
|
|
Convert a list of Column (or names) into a JVM (Scala) List of Column.
|
|
|
|
|
|
|
|
An optional `converter` could be used to convert items in `cols`
|
|
|
|
into JVM Column objects.
|
|
|
|
"""
|
|
|
|
if converter:
|
|
|
|
cols = [converter(c) for c in cols]
|
|
|
|
return sc._jvm.PythonUtils.toList(cols)
|
|
|
|
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
def _unary_op(name, doc="unary operator"):
|
|
|
|
""" Create a method for given unary operator """
|
|
|
|
def _(self):
|
|
|
|
jc = getattr(self._jc, name)()
|
|
|
|
return Column(jc)
|
|
|
|
_.__doc__ = doc
|
|
|
|
return _
|
|
|
|
|
|
|
|
|
|
|
|
def _func_op(name, doc=''):
|
|
|
|
def _(self):
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = getattr(sc._jvm.functions, name)(self._jc)
|
|
|
|
return Column(jc)
|
|
|
|
_.__doc__ = doc
|
|
|
|
return _
|
|
|
|
|
|
|
|
|
2015-09-11 18:19:04 -04:00
|
|
|
def _bin_func_op(name, reverse=False, doc="binary function"):
|
|
|
|
def _(self, other):
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
fn = getattr(sc._jvm.functions, name)
|
|
|
|
jc = other._jc if isinstance(other, Column) else _create_column_from_literal(other)
|
|
|
|
njc = fn(self._jc, jc) if not reverse else fn(jc, self._jc)
|
|
|
|
return Column(njc)
|
|
|
|
_.__doc__ = doc
|
|
|
|
return _
|
|
|
|
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
def _bin_op(name, doc="binary operator"):
|
|
|
|
""" Create a method for given binary operator
|
|
|
|
"""
|
|
|
|
def _(self, other):
|
|
|
|
jc = other._jc if isinstance(other, Column) else other
|
|
|
|
njc = getattr(self._jc, name)(jc)
|
|
|
|
return Column(njc)
|
|
|
|
_.__doc__ = doc
|
|
|
|
return _
|
|
|
|
|
|
|
|
|
|
|
|
def _reverse_op(name, doc="binary operator"):
|
|
|
|
""" Create a method for binary operator (this object is on right side)
|
|
|
|
"""
|
|
|
|
def _(self, other):
|
|
|
|
jother = _create_column_from_literal(other)
|
|
|
|
jc = getattr(jother, name)(self._jc)
|
|
|
|
return Column(jc)
|
|
|
|
_.__doc__ = doc
|
|
|
|
return _
|
|
|
|
|
|
|
|
|
|
|
|
class Column(object):
|
|
|
|
|
|
|
|
"""
|
|
|
|
A column in a DataFrame.
|
|
|
|
|
|
|
|
:class:`Column` instances can be created by::
|
|
|
|
|
|
|
|
# 1. Select a column out of a DataFrame
|
|
|
|
|
|
|
|
df.colName
|
|
|
|
df["colName"]
|
|
|
|
|
|
|
|
# 2. Create from an expression
|
|
|
|
df.colName + 1
|
|
|
|
1 / df.colName
|
2015-05-21 02:05:54 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.3
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, jc):
|
|
|
|
self._jc = jc
|
|
|
|
|
|
|
|
# arithmetic operators
|
|
|
|
__neg__ = _func_op("negate")
|
|
|
|
__add__ = _bin_op("plus")
|
|
|
|
__sub__ = _bin_op("minus")
|
|
|
|
__mul__ = _bin_op("multiply")
|
|
|
|
__div__ = _bin_op("divide")
|
|
|
|
__truediv__ = _bin_op("divide")
|
|
|
|
__mod__ = _bin_op("mod")
|
|
|
|
__radd__ = _bin_op("plus")
|
|
|
|
__rsub__ = _reverse_op("minus")
|
|
|
|
__rmul__ = _bin_op("multiply")
|
|
|
|
__rdiv__ = _reverse_op("divide")
|
|
|
|
__rtruediv__ = _reverse_op("divide")
|
|
|
|
__rmod__ = _reverse_op("mod")
|
2015-09-11 18:19:04 -04:00
|
|
|
__pow__ = _bin_func_op("pow")
|
|
|
|
__rpow__ = _bin_func_op("pow", reverse=True)
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
# logistic operators
|
|
|
|
__eq__ = _bin_op("equalTo")
|
|
|
|
__ne__ = _bin_op("notEqual")
|
|
|
|
__lt__ = _bin_op("lt")
|
|
|
|
__le__ = _bin_op("leq")
|
|
|
|
__ge__ = _bin_op("geq")
|
|
|
|
__gt__ = _bin_op("gt")
|
|
|
|
|
2017-05-01 12:43:32 -04:00
|
|
|
_eqNullSafe_doc = """
|
|
|
|
Equality test that is safe for null values.
|
|
|
|
|
|
|
|
:param other: a value or :class:`Column`
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df1 = spark.createDataFrame([
|
|
|
|
... Row(id=1, value='foo'),
|
|
|
|
... Row(id=2, value=None)
|
|
|
|
... ])
|
|
|
|
>>> df1.select(
|
|
|
|
... df1['value'] == 'foo',
|
|
|
|
... df1['value'].eqNullSafe('foo'),
|
|
|
|
... df1['value'].eqNullSafe(None)
|
|
|
|
... ).show()
|
|
|
|
+-------------+---------------+----------------+
|
|
|
|
|(value = foo)|(value <=> foo)|(value <=> NULL)|
|
|
|
|
+-------------+---------------+----------------+
|
|
|
|
| true| true| false|
|
|
|
|
| null| false| true|
|
|
|
|
+-------------+---------------+----------------+
|
|
|
|
>>> df2 = spark.createDataFrame([
|
|
|
|
... Row(value = 'bar'),
|
|
|
|
... Row(value = None)
|
|
|
|
... ])
|
|
|
|
>>> df1.join(df2, df1["value"] == df2["value"]).count()
|
|
|
|
0
|
|
|
|
>>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count()
|
|
|
|
1
|
|
|
|
>>> df2 = spark.createDataFrame([
|
|
|
|
... Row(id=1, value=float('NaN')),
|
|
|
|
... Row(id=2, value=42.0),
|
|
|
|
... Row(id=3, value=None)
|
|
|
|
... ])
|
|
|
|
>>> df2.select(
|
|
|
|
... df2['value'].eqNullSafe(None),
|
|
|
|
... df2['value'].eqNullSafe(float('NaN')),
|
|
|
|
... df2['value'].eqNullSafe(42.0)
|
|
|
|
... ).show()
|
|
|
|
+----------------+---------------+----------------+
|
|
|
|
|(value <=> NULL)|(value <=> NaN)|(value <=> 42.0)|
|
|
|
|
+----------------+---------------+----------------+
|
|
|
|
| false| true| false|
|
|
|
|
| false| false| true|
|
|
|
|
| true| false| false|
|
|
|
|
+----------------+---------------+----------------+
|
|
|
|
|
|
|
|
.. note:: Unlike Pandas, PySpark doesn't consider NaN values to be NULL.
|
|
|
|
See the `NaN Semantics`_ for details.
|
|
|
|
.. _NaN Semantics:
|
|
|
|
https://spark.apache.org/docs/latest/sql-programming-guide.html#nan-semantics
|
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
"""
|
|
|
|
eqNullSafe = _bin_op("eqNullSafe", _eqNullSafe_doc)
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
# `and`, `or`, `not` cannot be overloaded in Python,
|
|
|
|
# so use bitwise operators as boolean operators
|
|
|
|
__and__ = _bin_op('and')
|
|
|
|
__or__ = _bin_op('or')
|
|
|
|
__invert__ = _func_op('not')
|
|
|
|
__rand__ = _bin_op("and")
|
|
|
|
__ror__ = _bin_op("or")
|
|
|
|
|
|
|
|
# container operators
|
[SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator against column
## What changes were proposed in this pull request?
This PR proposes to remove incorrect implementation that has been not executed so far (at least from Spark 1.5.2) for `in` operator and throw a correct exception rather than saying it is a bool. I tested the codes above in 1.5.2, 1.6.3, 2.1.0 and in the master branch as below:
**1.5.2**
```python
>>> df = sqlContext.createDataFrame([[1]])
>>> 1 in df._1
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 418, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```
**1.6.3**
```python
>>> 1 in sqlContext.range(1).id
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 447, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```
**2.1.0**
```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 426, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```
**Current Master**
```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```
**After**
```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__
raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' "
ValueError: Cannot apply 'in' operator against a column: please use 'contains' in a string column or 'array_contains' function for an array column.
```
In more details,
It seems the implementation intended to support this
```python
1 in df.column
```
However, currently, it throws an exception as below:
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```
What happens here is as below:
```python
class Column(object):
def __contains__(self, item):
print "I am contains"
return Column()
def __nonzero__(self):
raise Exception("I am nonzero.")
>>> 1 in Column()
I am contains
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "<stdin>", line 6, in __nonzero__
Exception: I am nonzero.
```
It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` is being called against `Column()` to make this a bool (or int to be specific).
It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and `__contains__` forcing the the return into a bool unlike other operators. There are few references about this as below:
https://bugs.python.org/issue16011
http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378
http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777
It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to make this working because these force the return type as a bool as below:
```python
class Column(object):
def __contains__(self, item):
print "I am contains"
return Column()
def __nonzero__(self):
return "a"
>>> 1 in Column()
I am contains
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
TypeError: __nonzero__ should return bool or int, returned str
```
## How was this patch tested?
Added unit tests in `tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17160 from HyukjinKwon/SPARK-19701.
2017-03-05 21:04:52 -05:00
|
|
|
def __contains__(self, item):
|
|
|
|
raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' "
|
|
|
|
"in a string column or 'array_contains' function for an array column.")
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
# bitwise operators
|
2017-04-29 16:46:40 -04:00
|
|
|
_bitwiseOR_doc = """
|
|
|
|
Compute bitwise OR of this expression with another expression.
|
|
|
|
|
|
|
|
:param other: a value or :class:`Column` to calculate bitwise or(|) against
|
|
|
|
this :class:`Column`.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(a=170, b=75)])
|
|
|
|
>>> df.select(df.a.bitwiseOR(df.b)).collect()
|
|
|
|
[Row((a | b)=235)]
|
|
|
|
"""
|
|
|
|
_bitwiseAND_doc = """
|
|
|
|
Compute bitwise AND of this expression with another expression.
|
|
|
|
|
|
|
|
:param other: a value or :class:`Column` to calculate bitwise and(&) against
|
|
|
|
this :class:`Column`.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(a=170, b=75)])
|
|
|
|
>>> df.select(df.a.bitwiseAND(df.b)).collect()
|
|
|
|
[Row((a & b)=10)]
|
|
|
|
"""
|
|
|
|
_bitwiseXOR_doc = """
|
|
|
|
Compute bitwise XOR of this expression with another expression.
|
|
|
|
|
|
|
|
:param other: a value or :class:`Column` to calculate bitwise xor(^) against
|
|
|
|
this :class:`Column`.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(a=170, b=75)])
|
|
|
|
>>> df.select(df.a.bitwiseXOR(df.b)).collect()
|
|
|
|
[Row((a ^ b)=225)]
|
|
|
|
"""
|
|
|
|
|
|
|
|
bitwiseOR = _bin_op("bitwiseOR", _bitwiseOR_doc)
|
|
|
|
bitwiseAND = _bin_op("bitwiseAND", _bitwiseAND_doc)
|
|
|
|
bitwiseXOR = _bin_op("bitwiseXOR", _bitwiseXOR_doc)
|
2015-05-15 23:09:15 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-05-15 23:09:15 -04:00
|
|
|
def getItem(self, key):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
An expression that gets an item at position ``ordinal`` out of a list,
|
|
|
|
or gets an item by key out of a dict.
|
2015-05-15 23:09:15 -04:00
|
|
|
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 2], {"key": "value"})], ["l", "d"])
|
2015-05-15 23:09:15 -04:00
|
|
|
>>> df.select(df.l.getItem(0), df.d.getItem("key")).show()
|
|
|
|
+----+------+
|
|
|
|
|l[0]|d[key]|
|
|
|
|
+----+------+
|
|
|
|
| 1| value|
|
|
|
|
+----+------+
|
|
|
|
>>> df.select(df.l[0], df.d["key"]).show()
|
|
|
|
+----+------+
|
|
|
|
|l[0]|d[key]|
|
|
|
|
+----+------+
|
|
|
|
| 1| value|
|
|
|
|
+----+------+
|
|
|
|
"""
|
|
|
|
return self[key]
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-05-15 23:09:15 -04:00
|
|
|
def getField(self, name):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
An expression that gets a field by name in a StructField.
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df = spark.createDataFrame([Row(r=Row(a=1, b="b"))])
|
2015-05-15 23:09:15 -04:00
|
|
|
>>> df.select(df.r.getField("b")).show()
|
2016-02-21 09:53:15 -05:00
|
|
|
+---+
|
|
|
|
|r.b|
|
|
|
|
+---+
|
|
|
|
| b|
|
|
|
|
+---+
|
2015-05-15 23:09:15 -04:00
|
|
|
>>> df.select(df.r.a).show()
|
2016-02-21 09:53:15 -05:00
|
|
|
+---+
|
|
|
|
|r.a|
|
|
|
|
+---+
|
|
|
|
| 1|
|
|
|
|
+---+
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
return self[name]
|
|
|
|
|
|
|
|
def __getattr__(self, item):
|
|
|
|
if item.startswith("__"):
|
|
|
|
raise AttributeError(item)
|
|
|
|
return self.getField(item)
|
|
|
|
|
2017-02-13 18:23:56 -05:00
|
|
|
def __getitem__(self, k):
|
|
|
|
if isinstance(k, slice):
|
|
|
|
if k.step is not None:
|
|
|
|
raise ValueError("slice with step is not supported.")
|
|
|
|
return self.substr(k.start, k.stop)
|
|
|
|
else:
|
|
|
|
return _bin_op("apply")(self, k)
|
|
|
|
|
2015-09-02 16:36:36 -04:00
|
|
|
def __iter__(self):
|
|
|
|
raise TypeError("Column is not iterable")
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
# string methods
|
2017-04-29 16:46:40 -04:00
|
|
|
_contains_doc = """
|
|
|
|
Contains the other element. Returns a boolean :class:`Column` based on a string match.
|
|
|
|
|
|
|
|
:param other: string in line
|
|
|
|
|
|
|
|
>>> df.filter(df.name.contains('o')).collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
|
|
|
"""
|
2017-04-22 22:58:54 -04:00
|
|
|
_rlike_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex
|
|
|
|
match.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
:param other: an extended regex expression
|
|
|
|
|
|
|
|
>>> df.filter(df.name.rlike('ice$')).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
"""
|
|
|
|
_like_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
:param other: a SQL LIKE pattern
|
|
|
|
|
|
|
|
See :func:`rlike` for a regex version
|
|
|
|
|
|
|
|
>>> df.filter(df.name.like('Al%')).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
"""
|
|
|
|
_startswith_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
String starts with. Returns a boolean :class:`Column` based on a string match.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
2017-04-29 16:46:40 -04:00
|
|
|
:param other: string at start of line (do not use a regex `^`)
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
>>> df.filter(df.name.startswith('Al')).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
>>> df.filter(df.name.startswith('^Al')).collect()
|
|
|
|
[]
|
|
|
|
"""
|
|
|
|
_endswith_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
String ends with. Returns a boolean :class:`Column` based on a string match.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
:param other: string at end of line (do not use a regex `$`)
|
|
|
|
|
|
|
|
>>> df.filter(df.name.endswith('ice')).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
>>> df.filter(df.name.endswith('ice$')).collect()
|
|
|
|
[]
|
|
|
|
"""
|
|
|
|
|
2017-04-29 16:46:40 -04:00
|
|
|
contains = ignore_unicode_prefix(_bin_op("contains", _contains_doc))
|
2017-04-22 22:58:54 -04:00
|
|
|
rlike = ignore_unicode_prefix(_bin_op("rlike", _rlike_doc))
|
|
|
|
like = ignore_unicode_prefix(_bin_op("like", _like_doc))
|
|
|
|
startswith = ignore_unicode_prefix(_bin_op("startsWith", _startswith_doc))
|
|
|
|
endswith = ignore_unicode_prefix(_bin_op("endsWith", _endswith_doc))
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-05-15 23:09:15 -04:00
|
|
|
def substr(self, startPos, length):
|
|
|
|
"""
|
2015-05-23 11:30:05 -04:00
|
|
|
Return a :class:`Column` which is a substring of the column.
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
:param startPos: start position (int or Column)
|
|
|
|
:param length: length of the substring (int or Column)
|
|
|
|
|
|
|
|
>>> df.select(df.name.substr(1, 3).alias("col")).collect()
|
|
|
|
[Row(col=u'Ali'), Row(col=u'Bob')]
|
|
|
|
"""
|
|
|
|
if type(startPos) != type(length):
|
2017-08-15 22:19:15 -04:00
|
|
|
raise TypeError(
|
|
|
|
"startPos and length must be the same type. "
|
|
|
|
"Got {startPos_t} and {length_t}, respectively."
|
|
|
|
.format(
|
|
|
|
startPos_t=type(startPos),
|
|
|
|
length_t=type(length),
|
|
|
|
))
|
|
|
|
if isinstance(startPos, int):
|
2015-05-15 23:09:15 -04:00
|
|
|
jc = self._jc.substr(startPos, length)
|
|
|
|
elif isinstance(startPos, Column):
|
|
|
|
jc = self._jc.substr(startPos._jc, length._jc)
|
|
|
|
else:
|
|
|
|
raise TypeError("Unexpected type: %s" % type(startPos))
|
|
|
|
return Column(jc)
|
|
|
|
|
2015-08-06 13:39:16 -04:00
|
|
|
@ignore_unicode_prefix
|
|
|
|
@since(1.5)
|
|
|
|
def isin(self, *cols):
|
|
|
|
"""
|
|
|
|
A boolean expression that is evaluated to true if the value of this
|
|
|
|
expression is contained by the evaluated values of the arguments.
|
|
|
|
|
|
|
|
>>> df[df.name.isin("Bob", "Mike")].collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
|
|
|
>>> df[df.age.isin([1, 2, 3])].collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, set)):
|
|
|
|
cols = cols[0]
|
|
|
|
cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols]
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-06 13:39:16 -04:00
|
|
|
jc = getattr(self._jc, "isin")(_to_seq(sc, cols))
|
2015-05-15 23:09:15 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
# order
|
2017-04-29 16:46:40 -04:00
|
|
|
_asc_doc = """
|
2018-04-08 00:09:06 -04:00
|
|
|
Returns a sort expression based on ascending order of the column.
|
2017-04-29 16:46:40 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
2018-04-08 00:09:06 -04:00
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"])
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df.select(df.name).orderBy(df.name.asc()).collect()
|
|
|
|
[Row(name=u'Alice'), Row(name=u'Tom')]
|
|
|
|
"""
|
2018-04-08 00:09:06 -04:00
|
|
|
_asc_nulls_first_doc = """
|
|
|
|
Returns a sort expression based on ascending order of the column, and null values
|
|
|
|
return before non-null values.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"])
|
|
|
|
>>> df.select(df.name).orderBy(df.name.asc_nulls_first()).collect()
|
|
|
|
[Row(name=None), Row(name=u'Alice'), Row(name=u'Tom')]
|
|
|
|
|
|
|
|
.. versionadded:: 2.4
|
|
|
|
"""
|
|
|
|
_asc_nulls_last_doc = """
|
|
|
|
Returns a sort expression based on ascending order of the column, and null values
|
|
|
|
appear after non-null values.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"])
|
|
|
|
>>> df.select(df.name).orderBy(df.name.asc_nulls_last()).collect()
|
|
|
|
[Row(name=u'Alice'), Row(name=u'Tom'), Row(name=None)]
|
|
|
|
|
|
|
|
.. versionadded:: 2.4
|
|
|
|
"""
|
2017-04-29 16:46:40 -04:00
|
|
|
_desc_doc = """
|
2018-04-08 00:09:06 -04:00
|
|
|
Returns a sort expression based on the descending order of the column.
|
2017-04-29 16:46:40 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
2018-04-08 00:09:06 -04:00
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"])
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df.select(df.name).orderBy(df.name.desc()).collect()
|
|
|
|
[Row(name=u'Tom'), Row(name=u'Alice')]
|
|
|
|
"""
|
2018-04-08 00:09:06 -04:00
|
|
|
_desc_nulls_first_doc = """
|
|
|
|
Returns a sort expression based on the descending order of the column, and null values
|
|
|
|
appear before non-null values.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"])
|
|
|
|
>>> df.select(df.name).orderBy(df.name.desc_nulls_first()).collect()
|
|
|
|
[Row(name=None), Row(name=u'Tom'), Row(name=u'Alice')]
|
|
|
|
|
|
|
|
.. versionadded:: 2.4
|
|
|
|
"""
|
|
|
|
_desc_nulls_last_doc = """
|
|
|
|
Returns a sort expression based on the descending order of the column, and null values
|
|
|
|
appear after non-null values.
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"])
|
|
|
|
>>> df.select(df.name).orderBy(df.name.desc_nulls_last()).collect()
|
|
|
|
[Row(name=u'Tom'), Row(name=u'Alice'), Row(name=None)]
|
|
|
|
|
|
|
|
.. versionadded:: 2.4
|
|
|
|
"""
|
2017-04-29 16:46:40 -04:00
|
|
|
|
|
|
|
asc = ignore_unicode_prefix(_unary_op("asc", _asc_doc))
|
2018-04-08 00:09:06 -04:00
|
|
|
asc_nulls_first = ignore_unicode_prefix(_unary_op("asc_nulls_first", _asc_nulls_first_doc))
|
|
|
|
asc_nulls_last = ignore_unicode_prefix(_unary_op("asc_nulls_last", _asc_nulls_last_doc))
|
2017-04-29 16:46:40 -04:00
|
|
|
desc = ignore_unicode_prefix(_unary_op("desc", _desc_doc))
|
2018-04-08 00:09:06 -04:00
|
|
|
desc_nulls_first = ignore_unicode_prefix(_unary_op("desc_nulls_first", _desc_nulls_first_doc))
|
|
|
|
desc_nulls_last = ignore_unicode_prefix(_unary_op("desc_nulls_last", _desc_nulls_last_doc))
|
2015-05-15 23:09:15 -04:00
|
|
|
|
2017-04-22 22:58:54 -04:00
|
|
|
_isNull_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
True if the current expression is null.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
|
|
|
|
>>> df.filter(df.height.isNull()).collect()
|
2017-04-22 22:58:54 -04:00
|
|
|
[Row(height=None, name=u'Alice')]
|
|
|
|
"""
|
|
|
|
_isNotNull_doc = """
|
2017-04-29 16:46:40 -04:00
|
|
|
True if the current expression is NOT null.
|
2017-04-22 22:58:54 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
2017-04-29 16:46:40 -04:00
|
|
|
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
|
|
|
|
>>> df.filter(df.height.isNotNull()).collect()
|
2017-04-22 22:58:54 -04:00
|
|
|
[Row(height=80, name=u'Tom')]
|
|
|
|
"""
|
|
|
|
|
|
|
|
isNull = ignore_unicode_prefix(_unary_op("isNull", _isNull_doc))
|
|
|
|
isNotNull = ignore_unicode_prefix(_unary_op("isNotNull", _isNotNull_doc))
|
2015-05-15 23:09:15 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2017-02-14 12:57:43 -05:00
|
|
|
def alias(self, *alias, **kwargs):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
Returns this column aliased with a new name or names (in the case of expressions that
|
2015-05-15 23:09:15 -04:00
|
|
|
return more than one column, such as explode).
|
|
|
|
|
2017-02-14 12:57:43 -05:00
|
|
|
:param alias: strings of desired column names (collects all positional arguments passed)
|
|
|
|
:param metadata: a dict of information to be stored in ``metadata`` attribute of the
|
|
|
|
corresponding :class: `StructField` (optional, keyword only argument)
|
|
|
|
|
|
|
|
.. versionchanged:: 2.2
|
|
|
|
Added optional ``metadata`` argument.
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
>>> df.select(df.age.alias("age2")).collect()
|
|
|
|
[Row(age2=2), Row(age2=5)]
|
2017-02-14 12:57:43 -05:00
|
|
|
>>> df.select(df.age.alias("age3", metadata={'max': 99})).schema['age3'].metadata['max']
|
|
|
|
99
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
|
2017-02-14 12:57:43 -05:00
|
|
|
metadata = kwargs.pop('metadata', None)
|
|
|
|
assert not kwargs, 'Unexpected kwargs where passed: %s' % kwargs
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-05-15 23:09:15 -04:00
|
|
|
if len(alias) == 1:
|
2017-02-14 12:57:43 -05:00
|
|
|
if metadata:
|
|
|
|
jmeta = sc._jvm.org.apache.spark.sql.types.Metadata.fromJson(
|
|
|
|
json.dumps(metadata))
|
|
|
|
return Column(getattr(self._jc, "as")(alias[0], jmeta))
|
|
|
|
else:
|
|
|
|
return Column(getattr(self._jc, "as")(alias[0]))
|
2015-05-15 23:09:15 -04:00
|
|
|
else:
|
2017-02-14 12:57:43 -05:00
|
|
|
if metadata:
|
|
|
|
raise ValueError('metadata can only be provided for a single column')
|
2015-05-15 23:09:15 -04:00
|
|
|
return Column(getattr(self._jc, "as")(_to_seq(sc, list(alias))))
|
|
|
|
|
2016-03-23 02:43:09 -04:00
|
|
|
name = copy_func(alias, sinceversion=2.0, doc=":func:`name` is an alias for :func:`alias`.")
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-05-15 23:09:15 -04:00
|
|
|
def cast(self, dataType):
|
2015-05-23 11:30:05 -04:00
|
|
|
""" Convert the column into type ``dataType``.
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
>>> df.select(df.age.cast("string").alias('ages')).collect()
|
|
|
|
[Row(ages=u'2'), Row(ages=u'5')]
|
|
|
|
>>> df.select(df.age.cast(StringType()).alias('ages')).collect()
|
|
|
|
[Row(ages=u'2'), Row(ages=u'5')]
|
|
|
|
"""
|
|
|
|
if isinstance(dataType, basestring):
|
|
|
|
jc = self._jc.cast(dataType)
|
|
|
|
elif isinstance(dataType, DataType):
|
2016-08-25 02:36:04 -04:00
|
|
|
from pyspark.sql import SparkSession
|
|
|
|
spark = SparkSession.builder.getOrCreate()
|
|
|
|
jdt = spark._jsparkSession.parseDataType(dataType.json())
|
2015-05-15 23:09:15 -04:00
|
|
|
jc = self._jc.cast(jdt)
|
|
|
|
else:
|
|
|
|
raise TypeError("unexpected type: %s" % type(dataType))
|
|
|
|
return Column(jc)
|
|
|
|
|
2016-03-14 22:25:49 -04:00
|
|
|
astype = copy_func(cast, sinceversion=1.4, doc=":func:`astype` is an alias for :func:`cast`.")
|
2015-05-21 14:50:39 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-05-15 23:09:15 -04:00
|
|
|
def between(self, lowerBound, upperBound):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
A boolean expression that is evaluated to true if the value of this
|
2015-05-15 23:09:15 -04:00
|
|
|
expression is between the given columns.
|
2015-06-02 16:38:06 -04:00
|
|
|
|
|
|
|
>>> df.select(df.name, df.age.between(2, 4)).show()
|
2016-02-21 09:53:15 -05:00
|
|
|
+-----+---------------------------+
|
|
|
|
| name|((age >= 2) AND (age <= 4))|
|
|
|
|
+-----+---------------------------+
|
|
|
|
|Alice| true|
|
|
|
|
| Bob| false|
|
|
|
|
+-----+---------------------------+
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
return (self >= lowerBound) & (self <= upperBound)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-15 23:09:15 -04:00
|
|
|
def when(self, condition, value):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
Evaluates a list of conditions and returns one of multiple possible result expressions.
|
2015-05-15 23:09:15 -04:00
|
|
|
If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
|
|
|
|
|
|
|
|
See :func:`pyspark.sql.functions.when` for example usage.
|
|
|
|
|
|
|
|
:param condition: a boolean :class:`Column` expression.
|
|
|
|
:param value: a literal value, or a :class:`Column` expression.
|
2015-06-02 16:38:06 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import functions as F
|
|
|
|
>>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show()
|
2016-01-13 15:44:35 -05:00
|
|
|
+-----+------------------------------------------------------------+
|
|
|
|
| name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END|
|
|
|
|
+-----+------------------------------------------------------------+
|
|
|
|
|Alice| -1|
|
|
|
|
| Bob| 1|
|
|
|
|
+-----+------------------------------------------------------------+
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
if not isinstance(condition, Column):
|
|
|
|
raise TypeError("condition should be a Column")
|
|
|
|
v = value._jc if isinstance(value, Column) else value
|
2015-06-02 16:38:06 -04:00
|
|
|
jc = self._jc.when(condition._jc, v)
|
2015-05-15 23:09:15 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-15 23:09:15 -04:00
|
|
|
def otherwise(self, value):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
Evaluates a list of conditions and returns one of multiple possible result expressions.
|
2015-05-15 23:09:15 -04:00
|
|
|
If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
|
|
|
|
|
|
|
|
See :func:`pyspark.sql.functions.when` for example usage.
|
|
|
|
|
|
|
|
:param value: a literal value, or a :class:`Column` expression.
|
2015-06-02 16:38:06 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import functions as F
|
|
|
|
>>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show()
|
2016-01-13 15:44:35 -05:00
|
|
|
+-----+-------------------------------------+
|
|
|
|
| name|CASE WHEN (age > 3) THEN 1 ELSE 0 END|
|
|
|
|
+-----+-------------------------------------+
|
|
|
|
|Alice| 0|
|
|
|
|
| Bob| 1|
|
|
|
|
+-----+-------------------------------------+
|
2015-05-15 23:09:15 -04:00
|
|
|
"""
|
|
|
|
v = value._jc if isinstance(value, Column) else value
|
2015-06-02 16:38:06 -04:00
|
|
|
jc = self._jc.otherwise(v)
|
2015-05-15 23:09:15 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
2015-05-23 11:30:05 -04:00
|
|
|
@since(1.4)
|
|
|
|
def over(self, window):
|
|
|
|
"""
|
|
|
|
Define a windowing column.
|
|
|
|
|
|
|
|
:param window: a :class:`WindowSpec`
|
|
|
|
:return: a Column
|
|
|
|
|
|
|
|
>>> from pyspark.sql import Window
|
2019-06-12 22:04:41 -04:00
|
|
|
>>> window = Window.partitionBy("name").orderBy("age") \
|
|
|
|
.rowsBetween(Window.unboundedPreceding, Window.currentRow)
|
2015-05-23 11:30:05 -04:00
|
|
|
>>> from pyspark.sql.functions import rank, min
|
2019-06-12 22:04:41 -04:00
|
|
|
>>> df.withColumn("rank", rank().over(window)) \
|
|
|
|
.withColumn("min", min('age').over(window)).show()
|
|
|
|
+---+-----+----+---+
|
|
|
|
|age| name|rank|min|
|
|
|
|
+---+-----+----+---+
|
|
|
|
| 5| Bob| 1| 5|
|
|
|
|
| 2|Alice| 1| 2|
|
|
|
|
+---+-----+----+---+
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
from pyspark.sql.window import WindowSpec
|
|
|
|
if not isinstance(window, WindowSpec):
|
|
|
|
raise TypeError("window should be WindowSpec")
|
|
|
|
jc = self._jc.over(window._jspec)
|
|
|
|
return Column(jc)
|
|
|
|
|
2015-06-23 18:51:16 -04:00
|
|
|
def __nonzero__(self):
|
|
|
|
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
|
|
|
|
"'~' for 'not' when building DataFrame boolean expressions.")
|
|
|
|
__bool__ = __nonzero__
|
|
|
|
|
2015-05-15 23:09:15 -04:00
|
|
|
def __repr__(self):
|
|
|
|
return 'Column<%s>' % self._jc.toString().encode('utf8')
|
|
|
|
|
|
|
|
|
|
|
|
def _test():
|
|
|
|
import doctest
|
2016-05-23 21:14:48 -04:00
|
|
|
from pyspark.sql import SparkSession
|
2015-05-15 23:09:15 -04:00
|
|
|
import pyspark.sql.column
|
|
|
|
globs = pyspark.sql.column.__dict__.copy()
|
2016-05-23 21:14:48 -04:00
|
|
|
spark = SparkSession.builder\
|
|
|
|
.master("local[4]")\
|
|
|
|
.appName("sql.column tests")\
|
|
|
|
.getOrCreate()
|
|
|
|
sc = spark.sparkContext
|
2017-04-29 16:46:40 -04:00
|
|
|
globs['spark'] = spark
|
2015-05-15 23:09:15 -04:00
|
|
|
globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \
|
|
|
|
.toDF(StructType([StructField('age', IntegerType()),
|
|
|
|
StructField('name', StringType())]))
|
|
|
|
|
|
|
|
(failure_count, test_count) = doctest.testmod(
|
|
|
|
pyspark.sql.column, globs=globs,
|
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
|
2016-05-23 21:14:48 -04:00
|
|
|
spark.stop()
|
2015-05-15 23:09:15 -04:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-05-15 23:09:15 -04:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|