aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/dataframe.py
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2015-02-17 13:48:38 -0800
committerMichael Armbrust <michael@databricks.com>2015-02-17 13:48:38 -0800
commit3df85dccbc8fd1ba19bbcdb8d359c073b1494d98 (patch)
tree955d4ded491392807c0f8fab861d8241624b57cb /python/pyspark/sql/dataframe.py
parent445a755b884885b88c1778fd56a3151045b0b0ed (diff)
downloadspark-3df85dccbc8fd1ba19bbcdb8d359c073b1494d98.tar.gz
spark-3df85dccbc8fd1ba19bbcdb8d359c073b1494d98.tar.bz2
spark-3df85dccbc8fd1ba19bbcdb8d359c073b1494d98.zip
[SPARK-5871] output explain in Python
Author: Davies Liu <davies@databricks.com> Closes #4658 from davies/explain and squashes the following commits: db87ea2 [Davies Liu] output explain in Python
Diffstat (limited to 'python/pyspark/sql/dataframe.py')
-rw-r--r--python/pyspark/sql/dataframe.py23
1 files changed, 20 insertions, 3 deletions
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 841724095f..388033d385 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -244,8 +244,25 @@ class DataFrame(object):
debugging purpose.
If extended is False, only prints the physical plan.
- """
- self._jdf.explain(extended)
+
+ >>> df.explain()
+ PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at SQLContext.scala:...
+
+ >>> df.explain(True)
+ == Parsed Logical Plan ==
+ ...
+ == Analyzed Logical Plan ==
+ ...
+ == Optimized Logical Plan ==
+ ...
+ == Physical Plan ==
+ ...
+ == RDD ==
+ """
+ if extended:
+ print self._jdf.queryExecution().toString()
+ else:
+ print self._jdf.queryExecution().executedPlan().toString()
def isLocal(self):
"""
@@ -1034,7 +1051,7 @@ def _test():
Row(name='Bob', age=5, height=85)]).toDF()
(failure_count, test_count) = doctest.testmod(
pyspark.sql.dataframe, globs=globs,
- optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
+ optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
globs['sc'].stop()
if failure_count:
exit(-1)