-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark #17737
Changes from all commits
bb5de1f
92347de
af8ac74
2815ff1
eaeb456
0fd9e37
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -185,17 +185,51 @@ def __contains__(self, item): | |
"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): | ||
""" | ||
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 @@ def getField(self, name): | |
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,16 +284,25 @@ def __iter__(self): | |
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 | ||
|
||
>>> df.filter(df.name.rlike('ice$')).collect() | ||
[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,17 +312,17 @@ def __iter__(self): | |
[Row(age=2, name=u'Alice')] | ||
""" | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
_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')] | ||
>>> df.filter(df.name.startswith('^Al')).collect() | ||
[] | ||
""" | ||
_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. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems to be form the function above, but its correct in the code so no worries |
||
|
||
:param other: string at end of line (do not use a regex `$`) | ||
|
||
|
@@ -289,7 +332,7 @@ def __iter__(self): | |
[] | ||
""" | ||
|
||
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 @@ def isin(self, *cols): | |
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.") | ||
_asc_doc = """ | ||
Returns a sort expression based on the ascending 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.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. Often combined with | ||
:func:`DataFrame.filter` to select rows with null values. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
True if the current expression is 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.isNull()).collect() | ||
>>> 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 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I removed |
||
globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you want to update the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we could. I think this is not related with Python documentation fix BTW. |
||
.toDF(StructType([StructField('age', IntegerType()), | ||
StructField('name', StringType())])) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Matching it up with
|
||
* | ||
* Code generation inherited from BinaryArithmetic. | ||
*/ | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -779,15 +779,16 @@ 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 | ||
*/ | ||
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,39 +839,39 @@ 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 | ||
*/ | ||
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 | ||
*/ | ||
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 | ||
*/ | ||
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 | ||
*/ | ||
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. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This and the similar instances below are matched with
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you want to include example usages, as in the python documentation? E.g. for rlike,
I made four examples for It would also be helpful for the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea, that sounds good in a way but the downside of adding examples is to maintain and keep them up to date. Let's leave them out here as this PR targets to fix Python documentation. |
||
* {{{ | ||
* // 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) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.