-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-18069][Doc] improve python API docstrings #15053
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -254,6 +254,7 @@ def substr(self, startPos, length): | |
| :param startPos: start position (int or Column) | ||
| :param length: length of the substring (int or Column) | ||
|
|
||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.name.substr(1, 3).alias("col")).collect() | ||
| [Row(col=u'Ali'), Row(col=u'Bob')] | ||
| """ | ||
|
|
@@ -276,6 +277,7 @@ 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 = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df[df.name.isin("Bob", "Mike")].collect() | ||
| [Row(age=5, name=u'Bob')] | ||
| >>> df[df.age.isin([1, 2, 3])].collect() | ||
|
|
@@ -303,6 +305,7 @@ def alias(self, *alias): | |
| Returns this column aliased with a new name or names (in the case of expressions that | ||
| return more than one column, such as explode). | ||
|
|
||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.age.alias("age2")).collect() | ||
| [Row(age2=2), Row(age2=5)] | ||
| """ | ||
|
|
@@ -320,10 +323,13 @@ def alias(self, *alias): | |
| def cast(self, dataType): | ||
| """ Convert the column into type ``dataType``. | ||
|
|
||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.age.cast("string").alias('ages')).collect() | ||
| [Row(ages=u'2'), Row(ages=u'5')] | ||
| [Row(ages='2'), Row(ages='5')] | ||
|
|
||
| >>> from pyspark.sql.types import StringType | ||
| >>> df.select(df.age.cast(StringType()).alias('ages')).collect() | ||
| [Row(ages=u'2'), Row(ages=u'5')] | ||
| [Row(ages='2'), Row(ages='5')] | ||
|
Member
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. Here, too. Let us please sweep it back. |
||
| """ | ||
| if isinstance(dataType, basestring): | ||
| jc = self._jc.cast(dataType) | ||
|
|
@@ -344,6 +350,7 @@ def between(self, lowerBound, upperBound): | |
| A boolean expression that is evaluated to true if the value of this | ||
| expression is between the given columns. | ||
|
|
||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.name, df.age.between(2, 4)).show() | ||
| +-----+---------------------------+ | ||
| | name|((age >= 2) AND (age <= 4))| | ||
|
|
@@ -366,6 +373,7 @@ def when(self, condition, value): | |
| :param value: a literal value, or a :class:`Column` expression. | ||
|
|
||
| >>> from pyspark.sql import functions as F | ||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show() | ||
| +-----+------------------------------------------------------------+ | ||
| | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END| | ||
|
|
@@ -391,6 +399,7 @@ def otherwise(self, value): | |
| :param value: a literal value, or a :class:`Column` expression. | ||
|
|
||
| >>> from pyspark.sql import functions as F | ||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob')], ['age', 'name']) | ||
| >>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show() | ||
| +-----+-------------------------------------+ | ||
| | name|CASE WHEN (age > 3) THEN 1 ELSE 0 END| | ||
|
|
@@ -412,9 +421,17 @@ def over(self, window): | |
| :return: a Column | ||
|
|
||
| >>> from pyspark.sql import Window | ||
| >>> window = Window.partitionBy("name").orderBy("age").rowsBetween(-1, 1) | ||
| >>> from pyspark.sql.functions import rank, min | ||
| >>> # df.select(rank().over(window), min('age').over(window)) | ||
| >>> window = Window.partitionBy("name").orderBy("age") | ||
| >>> from pyspark.sql.functions import rank | ||
| >>> df = spark.createDataFrame([(2, 'Alice'), (5, 'Bob'), (3, 'Bob')], ['age', 'name']) | ||
| >>> df.select('name', 'age', rank().over(window)).show() | ||
| +-----+---+-----------------------------------------------------------------+ | ||
| | name|age|RANK() OVER (PARTITION BY name ORDER BY age ASC UnspecifiedFrame)| | ||
| +-----+---+-----------------------------------------------------------------+ | ||
| | Bob| 3| 1| | ||
| | Bob| 5| 2| | ||
| |Alice| 2| 1| | ||
| +-----+---+-----------------------------------------------------------------+ | ||
| """ | ||
| from pyspark.sql.window import WindowSpec | ||
| if not isinstance(window, WindowSpec): | ||
|
|
@@ -442,10 +459,6 @@ def _test(): | |
| .getOrCreate() | ||
| sc = spark.sparkContext | ||
| globs['sc'] = sc | ||
| 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) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -338,6 +338,7 @@ def registerDataFrameAsTable(self, df, tableName): | |
|
|
||
| Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`. | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| """ | ||
| df.createOrReplaceTempView(tableName) | ||
|
|
@@ -346,6 +347,7 @@ def registerDataFrameAsTable(self, df, tableName): | |
| def dropTempTable(self, tableName): | ||
| """ Remove the temp table from catalog. | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| >>> sqlContext.dropTempTable("table1") | ||
| """ | ||
|
|
@@ -376,10 +378,11 @@ def sql(self, sqlQuery): | |
|
|
||
| :return: :class:`DataFrame` | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| >>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1") | ||
| >>> df2.collect() | ||
| [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] | ||
| [Row(f1=1, f2='row1'), Row(f1=2, f2='row2')] | ||
| """ | ||
| return self.sparkSession.sql(sqlQuery) | ||
|
|
||
|
|
@@ -389,6 +392,7 @@ def table(self, tableName): | |
|
|
||
| :return: :class:`DataFrame` | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| >>> df2 = sqlContext.table("table1") | ||
| >>> sorted(df.collect()) == sorted(df2.collect()) | ||
|
|
@@ -409,10 +413,11 @@ def tables(self, dbName=None): | |
| :param dbName: string, name of the database to use. | ||
| :return: :class:`DataFrame` | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| >>> df2 = sqlContext.tables() | ||
| >>> df2.filter("tableName = 'table1'").first() | ||
| Row(database=u'', tableName=u'table1', isTemporary=True) | ||
| Row(tableName='table1', isTemporary=True) | ||
| """ | ||
| if dbName is None: | ||
| return DataFrame(self._ssql_ctx.tables(), self) | ||
|
|
@@ -426,6 +431,7 @@ def tableNames(self, dbName=None): | |
| :param dbName: string, name of the database to use. Default to the current database. | ||
| :return: list of table names, in string | ||
|
|
||
| >>> df = spark.createDataFrame([(1, 'row1'), (2, 'row2')], ['field1', 'field2']) | ||
| >>> sqlContext.registerDataFrameAsTable(df, "table1") | ||
| >>> "table1" in sqlContext.tableNames() | ||
| True | ||
|
|
@@ -474,6 +480,7 @@ def readStream(self): | |
|
|
||
| :return: :class:`DataStreamReader` | ||
|
|
||
| >>> import tempfile | ||
| >>> text_sdf = sqlContext.readStream.text(tempfile.mkdtemp()) | ||
| >>> text_sdf.isStreaming | ||
| True | ||
|
|
@@ -553,34 +560,16 @@ def register(self, name, f, returnType=StringType()): | |
| def _test(): | ||
|
Member
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. Let's leave the newline here. Usually the block of imports has a newline before the codes (I believe pep8 will complain). |
||
| import os | ||
| import doctest | ||
| import tempfile | ||
| from pyspark.context import SparkContext | ||
| from pyspark.sql import Row, SQLContext | ||
| import pyspark.sql.context | ||
|
|
||
| os.chdir(os.environ["SPARK_HOME"]) | ||
|
|
||
| globs = pyspark.sql.context.__dict__.copy() | ||
| sc = SparkContext('local[4]', 'PythonTest') | ||
| globs['tempfile'] = tempfile | ||
| globs['os'] = os | ||
|
|
||
| globs['sc'] = sc | ||
| globs['sqlContext'] = SQLContext(sc) | ||
| globs['rdd'] = rdd = sc.parallelize( | ||
| [Row(field1=1, field2="row1"), | ||
| Row(field1=2, field2="row2"), | ||
| Row(field1=3, field2="row3")] | ||
| ) | ||
| globs['df'] = rdd.toDF() | ||
| jsonStrings = [ | ||
| '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', | ||
| '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' | ||
| '"field6":[{"field7": "row2"}]}', | ||
| '{"field1" : null, "field2": "row3", ' | ||
| '"field3":{"field4":33, "field5": []}}' | ||
| ] | ||
| globs['jsonStrings'] = jsonStrings | ||
| globs['json'] = sc.parallelize(jsonStrings) | ||
| (failure_count, test_count) = doctest.testmod( | ||
| pyspark.sql.context, globs=globs, | ||
| optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) | ||
|
|
||
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.
If I remember correctly, this test breaks the tests in Python 2.x. Let's use unicodes here.
Uh oh!
There was an error while loading. Please reload this page.
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.
FYI, the unicode string in Python 3 does not have prefix 'u'. So, this is being handled via
@ignore_unicode_prefix.