@@ -146,14 +146,28 @@ def table(self, tableName):
146146 return self ._df (self ._jreader .table (tableName ))
147147
148148 @since (1.4 )
149- def parquet (self , * path ):
149+ def parquet (self , * paths ):
150150 """Loads a Parquet file, returning the result as a :class:`DataFrame`.
151151
152152 >>> df = sqlContext.read.parquet('python/test_support/sql/parquet_partitioned')
153153 >>> df.dtypes
154154 [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')]
155155 """
156- return self ._df (self ._jreader .parquet (_to_seq (self ._sqlContext ._sc , path )))
156+ return self ._df (self ._jreader .parquet (_to_seq (self ._sqlContext ._sc , paths )))
157+
158+ @since (1.5 )
159+ def orc (self , path ):
160+ """
161+ Loads an ORC file, returning the result as a :class:`DataFrame`.
162+
163+ ::Note: Currently ORC support is only available together with
164+ :class:`HiveContext`.
165+
166+ >>> df = hiveContext.read.orc('python/test_support/sql/orc_partitioned')
167+ >>> df.dtypes
168+ [('a', 'bigint'), ('b', 'int'), ('c', 'int')]
169+ """
170+ return self ._df (self ._jreader .orc (path ))
157171
158172 @since (1.4 )
159173 def jdbc (self , url , table , column = None , lowerBound = None , upperBound = None , numPartitions = None ,
@@ -378,6 +392,29 @@ def parquet(self, path, mode=None, partitionBy=None):
378392 self .partitionBy (partitionBy )
379393 self ._jwrite .parquet (path )
380394
395+ def orc (self , path , mode = None , partitionBy = None ):
396+ """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
397+
398+ ::Note: Currently ORC support is only available together with
399+ :class:`HiveContext`.
400+
401+ :param path: the path in any Hadoop supported file system
402+ :param mode: specifies the behavior of the save operation when data already exists.
403+
404+ * ``append``: Append contents of this :class:`DataFrame` to existing data.
405+ * ``overwrite``: Overwrite existing data.
406+ * ``ignore``: Silently ignore this operation if data already exists.
407+ * ``error`` (default case): Throw an exception if data already exists.
408+ :param partitionBy: names of partitioning columns
409+
410+ >>> orc_df = hiveContext.read.orc('python/test_support/sql/orc_partitioned')
411+ >>> orc_df.write.orc(os.path.join(tempfile.mkdtemp(), 'data'))
412+ """
413+ self .mode (mode )
414+ if partitionBy is not None :
415+ self .partitionBy (partitionBy )
416+ self ._jwrite .orc (path )
417+
381418 @since (1.4 )
382419 def jdbc (self , url , table , mode = None , properties = {}):
383420 """Saves the content of the :class:`DataFrame` to a external database table via JDBC.
@@ -408,7 +445,7 @@ def _test():
408445 import os
409446 import tempfile
410447 from pyspark .context import SparkContext
411- from pyspark .sql import Row , SQLContext
448+ from pyspark .sql import Row , SQLContext , HiveContext
412449 import pyspark .sql .readwriter
413450
414451 os .chdir (os .environ ["SPARK_HOME" ])
@@ -420,6 +457,7 @@ def _test():
420457 globs ['os' ] = os
421458 globs ['sc' ] = sc
422459 globs ['sqlContext' ] = SQLContext (sc )
460+ globs ['hiveContext' ] = HiveContext (sc )
423461 globs ['df' ] = globs ['sqlContext' ].read .parquet ('python/test_support/sql/parquet_partitioned' )
424462
425463 (failure_count , test_count ) = doctest .testmod (
0 commit comments