diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index ec05c18d4f0626e285c624821167e9fb3a092552..46c1707cb6c37e63caab1bce6fb245ece0aaf417 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -250,11 +250,50 @@ class Column(object): raise TypeError("Column is not iterable") # string methods + _rlike_doc = """ + Return 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. + + :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 = """ + Return a Boolean :class:`Column` based on a string match. + + :param other: string at end 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. + + :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() + [] + """ + contains = _bin_op("contains") - rlike = _bin_op("rlike") - like = _bin_op("like") - startswith = _bin_op("startsWith") - endswith = _bin_op("endsWith") + 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)) @ignore_unicode_prefix @since(1.3) @@ -303,8 +342,27 @@ class Column(object): desc = _unary_op("desc", "Returns a sort expression based on the" " descending order of the given column name.") - isNull = _unary_op("isNull", "True if the current expression is null.") - isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") + _isNull_doc = """ + True if the current expression is null. Often combined with + :func:`DataFrame.filter` to select rows with null values. + + >>> 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() + [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. + + >>> 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() + [Row(height=80, name=u'Tom')] + """ + + isNull = ignore_unicode_prefix(_unary_op("isNull", _isNull_doc)) + isNotNull = ignore_unicode_prefix(_unary_op("isNotNull", _isNotNull_doc)) @since(1.3) def alias(self, *alias, **kwargs):