[SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark

## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
This commit is contained in:
hyukjinkwon 2017-04-29 13:46:40 -07:00 committed by Holden Karau
parent 70f1bcd7bc
commit d228cd0b02
3 changed files with 101 additions and 40 deletions

View file

@ -185,9 +185,43 @@ class Column(object):
"in a string column or 'array_contains' function for an array column.")
# bitwise operators
bitwiseOR = _bin_op("bitwiseOR")
bitwiseAND = _bin_op("bitwiseAND")
bitwiseXOR = _bin_op("bitwiseXOR")
_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)
@since(1.3)
def getItem(self, key):
@ -195,7 +229,7 @@ class Column(object):
An expression that gets an item at position ``ordinal`` out of a list,
or gets an item by key out of a dict.
>>> df = sc.parallelize([([1, 2], {"key": "value"})]).toDF(["l", "d"])
>>> df = spark.createDataFrame([([1, 2], {"key": "value"})], ["l", "d"])
>>> df.select(df.l.getItem(0), df.d.getItem("key")).show()
+----+------+
|l[0]|d[key]|
@ -217,7 +251,7 @@ class Column(object):
An expression that gets a field by name in a StructField.
>>> from pyspark.sql import Row
>>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF()
>>> df = spark.createDataFrame([Row(r=Row(a=1, b="b"))])
>>> df.select(df.r.getField("b")).show()
+---+
|r.b|
@ -250,8 +284,17 @@ class Column(object):
raise TypeError("Column is not iterable")
# string methods
_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')]
"""
_rlike_doc = """
Return a Boolean :class:`Column` based on a regex match.
SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex
match.
:param other: an extended regex expression
@ -259,7 +302,7 @@ class Column(object):
[Row(age=2, name=u'Alice')]
"""
_like_doc = """
Return a Boolean :class:`Column` based on a SQL LIKE match.
SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match.
:param other: a SQL LIKE pattern
@ -269,9 +312,9 @@ class Column(object):
[Row(age=2, name=u'Alice')]
"""
_startswith_doc = """
Return a Boolean :class:`Column` based on a string match.
String starts with. Returns a boolean :class:`Column` based on a string match.
:param other: string at end of line (do not use a regex `^`)
:param other: string at start of line (do not use a regex `^`)
>>> df.filter(df.name.startswith('Al')).collect()
[Row(age=2, name=u'Alice')]
@ -279,7 +322,7 @@ class Column(object):
[]
"""
_endswith_doc = """
Return a Boolean :class:`Column` based on matching end of string.
String ends with. Returns a boolean :class:`Column` based on a string match.
:param other: string at end of line (do not use a regex `$`)
@ -289,7 +332,7 @@ class Column(object):
[]
"""
contains = _bin_op("contains")
contains = ignore_unicode_prefix(_bin_op("contains", _contains_doc))
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))
@ -337,27 +380,40 @@ class Column(object):
return Column(jc)
# order
asc = _unary_op("asc", "Returns a sort expression based on the"
" ascending order of the given column name.")
desc = _unary_op("desc", "Returns a sort expression based on the"
" descending order of the given column name.")
_isNull_doc = """
True if the current expression is null. Often combined with
:func:`DataFrame.filter` to select rows with null values.
_asc_doc = """
Returns a sort expression based on the ascending order of the given column name
>>> from pyspark.sql import Row
>>> df2 = sc.parallelize([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)]).toDF()
>>> df2.filter(df2.height.isNull()).collect()
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
>>> df.select(df.name).orderBy(df.name.asc()).collect()
[Row(name=u'Alice'), Row(name=u'Tom')]
"""
_desc_doc = """
Returns a sort expression based on the descending order of the given column name.
>>> from pyspark.sql import Row
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
>>> df.select(df.name).orderBy(df.name.desc()).collect()
[Row(name=u'Tom'), Row(name=u'Alice')]
"""
asc = ignore_unicode_prefix(_unary_op("asc", _asc_doc))
desc = ignore_unicode_prefix(_unary_op("desc", _desc_doc))
_isNull_doc = """
True if the current expression is null.
>>> from pyspark.sql import Row
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
>>> df.filter(df.height.isNull()).collect()
[Row(height=None, name=u'Alice')]
"""
_isNotNull_doc = """
True if the current expression is null. Often combined with
:func:`DataFrame.filter` to select rows with non-null values.
True if the current expression is NOT null.
>>> from pyspark.sql import Row
>>> df2 = sc.parallelize([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)]).toDF()
>>> df2.filter(df2.height.isNotNull()).collect()
>>> df = spark.createDataFrame([Row(name=u'Tom', height=80), Row(name=u'Alice', height=None)])
>>> df.filter(df.height.isNotNull()).collect()
[Row(height=80, name=u'Tom')]
"""
@ -527,7 +583,7 @@ def _test():
.appName("sql.column tests")\
.getOrCreate()
sc = spark.sparkContext
globs['sc'] = sc
globs['spark'] = spark
globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \
.toDF(StructType([StructField('age', IntegerType()),
StructField('name', StringType())]))

View file

@ -86,7 +86,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet
}
/**
* A function that calculates bitwise xor of two numbers.
* A function that calculates bitwise xor({@literal ^}) of two numbers.
*
* Code generation inherited from BinaryArithmetic.
*/

View file

@ -779,7 +779,7 @@ class Column(val expr: Expression) extends Logging {
def isin(list: Any*): Column = withExpr { In(expr, list.map(lit(_).expr)) }
/**
* SQL like expression.
* SQL like expression. Returns a boolean column based on a SQL LIKE match.
*
* @group expr_ops
* @since 1.3.0
@ -787,7 +787,8 @@ class Column(val expr: Expression) extends Logging {
def like(literal: String): Column = withExpr { Like(expr, lit(literal).expr) }
/**
* SQL RLIKE expression (LIKE with Regex).
* SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex
* match.
*
* @group expr_ops
* @since 1.3.0
@ -838,7 +839,7 @@ class Column(val expr: Expression) extends Logging {
}
/**
* Contains the other element.
* Contains the other element. Returns a boolean column based on a string match.
*
* @group expr_ops
* @since 1.3.0
@ -846,7 +847,7 @@ class Column(val expr: Expression) extends Logging {
def contains(other: Any): Column = withExpr { Contains(expr, lit(other).expr) }
/**
* String starts with.
* String starts with. Returns a boolean column based on a string match.
*
* @group expr_ops
* @since 1.3.0
@ -854,7 +855,7 @@ class Column(val expr: Expression) extends Logging {
def startsWith(other: Column): Column = withExpr { StartsWith(expr, lit(other).expr) }
/**
* String starts with another string literal.
* String starts with another string literal. Returns a boolean column based on a string match.
*
* @group expr_ops
* @since 1.3.0
@ -862,7 +863,7 @@ class Column(val expr: Expression) extends Logging {
def startsWith(literal: String): Column = this.startsWith(lit(literal))
/**
* String ends with.
* String ends with. Returns a boolean column based on a string match.
*
* @group expr_ops
* @since 1.3.0
@ -870,7 +871,7 @@ class Column(val expr: Expression) extends Logging {
def endsWith(other: Column): Column = withExpr { EndsWith(expr, lit(other).expr) }
/**
* String ends with another string literal.
* String ends with another string literal. Returns a boolean column based on a string match.
*
* @group expr_ops
* @since 1.3.0
@ -1008,7 +1009,7 @@ class Column(val expr: Expression) extends Logging {
def cast(to: String): Column = cast(CatalystSqlParser.parseDataType(to))
/**
* Returns an ordering used in sorting.
* Returns a sort expression based on the descending order of the column.
* {{{
* // Scala
* df.sort(df("age").desc)
@ -1023,7 +1024,8 @@ class Column(val expr: Expression) extends Logging {
def desc: Column = withExpr { SortOrder(expr, Descending) }
/**
* Returns a descending ordering used in sorting, where null values appear before non-null values.
* Returns a sort expression based on the descending order of the column,
* and null values appear before non-null values.
* {{{
* // Scala: sort a DataFrame by age column in descending order and null values appearing first.
* df.sort(df("age").desc_nulls_first)
@ -1038,7 +1040,8 @@ class Column(val expr: Expression) extends Logging {
def desc_nulls_first: Column = withExpr { SortOrder(expr, Descending, NullsFirst, Set.empty) }
/**
* Returns a descending ordering used in sorting, where null values appear after non-null values.
* Returns a sort expression based on the descending order of the column,
* and null values appear after non-null values.
* {{{
* // Scala: sort a DataFrame by age column in descending order and null values appearing last.
* df.sort(df("age").desc_nulls_last)
@ -1053,7 +1056,7 @@ class Column(val expr: Expression) extends Logging {
def desc_nulls_last: Column = withExpr { SortOrder(expr, Descending, NullsLast, Set.empty) }
/**
* Returns an ascending ordering used in sorting.
* Returns a sort expression based on ascending order of the column.
* {{{
* // Scala: sort a DataFrame by age column in ascending order.
* df.sort(df("age").asc)
@ -1068,7 +1071,8 @@ class Column(val expr: Expression) extends Logging {
def asc: Column = withExpr { SortOrder(expr, Ascending) }
/**
* Returns an ascending ordering used in sorting, where null values appear before non-null values.
* Returns a sort expression based on ascending order of the column,
* and null values return before non-null values.
* {{{
* // Scala: sort a DataFrame by age column in ascending order and null values appearing first.
* df.sort(df("age").asc_nulls_last)
@ -1083,7 +1087,8 @@ class Column(val expr: Expression) extends Logging {
def asc_nulls_first: Column = withExpr { SortOrder(expr, Ascending, NullsFirst, Set.empty) }
/**
* Returns an ordering used in sorting, where null values appear after non-null values.
* Returns a sort expression based on ascending order of the column,
* and null values appear after non-null values.
* {{{
* // Scala: sort a DataFrame by age column in ascending order and null values appearing last.
* df.sort(df("age").asc_nulls_last)