diff options
author | Davies Liu <davies@databricks.com> | 2015-02-11 12:13:16 -0800 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2015-02-11 12:13:16 -0800 |
commit | b694eb9c2fefeaa33891d3e61f9bea369bc09984 (patch) | |
tree | 0618924c6564e41ab27676415e79467216d4832f /python/pyspark/sql/dataframe.py | |
parent | 1ac099e3e00ddb01af8e6e3a84c70f8363f04b5c (diff) | |
download | spark-b694eb9c2fefeaa33891d3e61f9bea369bc09984.tar.gz spark-b694eb9c2fefeaa33891d3e61f9bea369bc09984.tar.bz2 spark-b694eb9c2fefeaa33891d3e61f9bea369bc09984.zip |
[SPARK-5677] [SPARK-5734] [SQL] [PySpark] Python DataFrame API remaining tasks
1. DataFrame.renameColumn
2. DataFrame.show() and _repr_
3. Use simpleString() rather than jsonValue in DataFrame.dtypes
4. createDataFrame from local Python data, including pandas.DataFrame
Author: Davies Liu <davies@databricks.com>
Closes #4528 from davies/df3 and squashes the following commits:
014acea [Davies Liu] fix typo
6ba526e [Davies Liu] fix tests
46f5f95 [Davies Liu] address comments
6cbc154 [Davies Liu] dataframe.show() and improve dtypes
6f94f25 [Davies Liu] create DataFrame from local Python data
Diffstat (limited to 'python/pyspark/sql/dataframe.py')
-rw-r--r-- | python/pyspark/sql/dataframe.py | 42 |
1 files changed, 39 insertions, 3 deletions
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 3eef0cc376..3eb56ed74c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -236,6 +236,24 @@ class DataFrame(object): """ print (self._jdf.schema().treeString()) + def show(self): + """ + Print the first 20 rows. + + >>> df.show() + age name + 2 Alice + 5 Bob + >>> df + age name + 2 Alice + 5 Bob + """ + print (self) + + def __repr__(self): + return self._jdf.showString() + def count(self): """Return the number of elements in this RDD. @@ -380,9 +398,9 @@ class DataFrame(object): """Return all column names and their data types as a list. >>> df.dtypes - [('age', 'integer'), ('name', 'string')] + [('age', 'int'), ('name', 'string')] """ - return [(str(f.name), f.dataType.jsonValue()) for f in self.schema().fields] + return [(str(f.name), f.dataType.simpleString()) for f in self.schema().fields] @property def columns(self): @@ -606,6 +624,17 @@ class DataFrame(object): """ return self.select('*', col.alias(colName)) + def renameColumn(self, existing, new): + """ Rename an existing column to a new name + + >>> df.renameColumn('age', 'age2').collect() + [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] + """ + cols = [Column(_to_java_column(c), self.sql_ctx).alias(new) + if c == existing else c + for c in self.columns] + return self.select(*cols) + def to_pandas(self): """ Collect all the rows and return a `pandas.DataFrame`. @@ -885,6 +914,12 @@ class Column(DataFrame): jc = self._jc.cast(jdt) return Column(jc, self.sql_ctx) + def __repr__(self): + if self._jdf.isComputable(): + return self._jdf.samples() + else: + return 'Column<%s>' % self._jdf.toString() + def to_pandas(self): """ Return a pandas.Series from the column @@ -1030,7 +1065,8 @@ def _test(): globs['df'] = sqlCtx.inferSchema(rdd2) globs['df2'] = sqlCtx.inferSchema(rdd3) (failure_count, test_count) = doctest.testmod( - pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS) + pyspark.sql.dataframe, globs=globs, + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) globs['sc'].stop() if failure_count: exit(-1) |