Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
25 changes: 19 additions & 6 deletions python/pyspark/sql/column.py
Original file line number Diff line number Diff line change
Expand Up @@ -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')]
"""
Expand All @@ -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()
Expand Down Expand Up @@ -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)]
"""
Expand All @@ -320,6 +323,7 @@ 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')]
>>> df.select(df.age.cast(StringType()).alias('ages')).collect()
Expand All @@ -344,6 +348,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))|
Expand All @@ -366,6 +371,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|
Expand All @@ -391,6 +397,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|
Expand All @@ -412,9 +419,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 NULLS FIRST UnspecifiedFrame)|
+-----+---+-----------------------------------------------------------------------------+
| Bob| 3| 1|
| Bob| 5| 2|
|Alice| 2| 1|
+-----+---+-----------------------------------------------------------------------------+
"""
from pyspark.sql.window import WindowSpec
if not isinstance(window, WindowSpec):
Expand Down Expand Up @@ -442,9 +457,7 @@ 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())]))
globs['spark'] = spark

(failure_count, test_count) = doctest.testmod(
pyspark.sql.column, globs=globs,
Expand Down
33 changes: 15 additions & 18 deletions python/pyspark/sql/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,8 @@ def registerDataFrameAsTable(self, df, tableName):

Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`.

>>> df = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
"""
df.createOrReplaceTempView(tableName)
Expand All @@ -346,6 +348,8 @@ def registerDataFrameAsTable(self, df, tableName):
def dropTempTable(self, tableName):
""" Remove the temp table from catalog.

>>> df = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> sqlContext.dropTempTable("table1")
"""
Expand Down Expand Up @@ -376,6 +380,8 @@ def sql(self, sqlQuery):

:return: :class:`DataFrame`

>>> df = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1")
>>> df2.collect()
Expand All @@ -389,6 +395,8 @@ def table(self, tableName):

:return: :class:`DataFrame`

>>> df = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> df2 = sqlContext.table("table1")
>>> sorted(df.collect()) == sorted(df2.collect())
Expand All @@ -409,6 +417,8 @@ def tables(self, dbName=None):
:param dbName: string, name of the database to use.
:return: :class:`DataFrame`

>>> df = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> df2 = sqlContext.tables()
>>> df2.filter("tableName = 'table1'").first()
Expand All @@ -426,6 +436,8 @@ 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 = sqlContext.createDataFrame([(1, 'row1'), (2, 'row2'), (3, 'row3')],
... ['field1', 'field2'])
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> "table1" in sqlContext.tableNames()
True
Expand Down Expand Up @@ -474,6 +486,7 @@ def readStream(self):

:return: :class:`DataStreamReader`

>>> import tempfile
>>> text_sdf = sqlContext.readStream.text(tempfile.mkdtemp())
>>> text_sdf.isStreaming
True
Expand Down Expand Up @@ -553,34 +566,18 @@ def register(self, name, f, returnType=StringType()):
def _test():
import os
import doctest
import tempfile
from pyspark.context import SparkContext
from pyspark.sql import Row, SQLContext
from pyspark.sql import 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)
Expand Down
Loading