@@ -342,32 +342,21 @@ def sequenceFile(self, path, keyClass, valueClass, keyConverter=None, valueConve
342342 Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS,
343343 a local file system (available on all nodes), or any Hadoop-supported file system URI.
344344 The mechanism is as follows:
345- 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes
345+ 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key
346+ and value Writable classes
346347 2. Serialization is attempted via Pyrolite pickling
347348 3. If this fails, the fallback is to call 'toString' on each key and value
348349 4. C{PickleSerializer} is used to deserialize pickled objects on the Python side
349350
350- @param path:
351- @param keyClass:
352- @param valueClass:
353- @param keyWrapper:
354- @param valueWrapper:
355- @param minSplits:
356- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sfint/").collect())
357- [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
358- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sfdouble/").collect())
359- [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')]
360- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sftext/").collect())
361- [(u'1', u'aa'), (u'1', u'aa'), (u'2', u'aa'), (u'2', u'bb'), (u'2', u'bb'), (u'3', u'cc')]
362- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sfbool/").collect())
363- [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)]
364- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sfnull/").collect())
365- [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)]
366- >>> sorted(sc.sequenceFile(tempdir + "/sftestdata/sfmap/").collect())
367- [(1, {2.0: u'aa'}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), (2, {3.0: u'bb'}), (3, {2.0: u'dd'})]
368- >>> r = sorted(sc.sequenceFile(tempdir + "/sftestdata/sfclass").collect())[0]
369- >>> r == (u'1', {u'__class__': u'org.apache.spark.api.python.TestWritable', u'double': 54.0, u'int': 123, u'str': u'test1'})
370- True
351+ @param path: path to sequncefile
352+ @param keyClass: fully qualified classname of key Writable class
353+ (e.g. "org.apache.hadoop.io.Text")
354+ @param valueClass: fully qualified classname of value Writable class
355+ (e.g. "org.apache.hadoop.io.LongWritable")
356+ @param keyConverter:
357+ @param valueConverter:
358+ @param minSplits: minimum splits in dataset
359+ (default min(2, sc.defaultParallelism))
371360 """
372361 minSplits = minSplits or min (self .defaultParallelism , 2 )
373362 jrdd = self ._jvm .PythonRDD .sequenceFile (self ._jsc , path , keyClass , valueClass ,
@@ -383,6 +372,18 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv
383372
384373 A Hadoop configuration can be passed in as a Python dict. This will be converted into a
385374 Configuration in Java
375+
376+ @param path: path to Hadoop file
377+ @param inputFormatClass: fully qualified classname of Hadoop InputFormat
378+ (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat")
379+ @param keyClass: fully qualified classname of key Writable class
380+ (e.g. "org.apache.hadoop.io.Text")
381+ @param valueClass: fully qualified classname of value Writable class
382+ (e.g. "org.apache.hadoop.io.LongWritable")
383+ @param keyConverter: (None by default)
384+ @param valueConverter: (None by default)
385+ @param conf: Hadoop configuration, passed in as a dict
386+ (None by default)
386387 """
387388 jconf = self ._dictToJavaMap (conf )
388389 jrdd = self ._jvm .PythonRDD .newAPIHadoopFile (self ._jsc , path , inputFormatClass , keyClass ,
@@ -393,9 +394,20 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N
393394 valueConverter = None , conf = None ):
394395 """
395396 Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary
396- Hadoop configuration,
397- which is passed in as a Python dict. This will be converted into a Configuration in Java.
397+ Hadoop configuration, which is passed in as a Python dict.
398+ This will be converted into a Configuration in Java.
398399 The mechanism is the same as for sc.sequenceFile.
400+
401+ @param inputFormatClass: fully qualified classname of Hadoop InputFormat
402+ (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat")
403+ @param keyClass: fully qualified classname of key Writable class
404+ (e.g. "org.apache.hadoop.io.Text")
405+ @param valueClass: fully qualified classname of value Writable class
406+ (e.g. "org.apache.hadoop.io.LongWritable")
407+ @param keyConverter: (None by default)
408+ @param valueConverter: (None by default)
409+ @param conf: Hadoop configuration, passed in as a dict
410+ (None by default)
399411 """
400412 jconf = self ._dictToJavaMap (conf )
401413 jrdd = self ._jvm .PythonRDD .newAPIHadoopRDD (self ._jsc , inputFormatClass , keyClass ,
@@ -410,11 +422,21 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=
410422 The mechanism is the same as for sc.sequenceFile.
411423
412424 A Hadoop configuration can be passed in as a Python dict. This will be converted into a
413- Configuration in Java
425+ Configuration in Java.
426+
427+ @param path: path to Hadoop file
428+ @param inputFormatClass: fully qualified classname of Hadoop InputFormat
429+ (e.g. "org.apache.hadoop.mapred.TextInputFormat")
430+ @param keyClass: fully qualified classname of key Writable class
431+ (e.g. "org.apache.hadoop.io.Text")
432+ @param valueClass: fully qualified classname of value Writable class
433+ (e.g. "org.apache.hadoop.io.LongWritable")
434+ @param keyConverter: (None by default)
435+ @param valueConverter: (None by default)
436+ @param conf: Hadoop configuration, passed in as a dict
437+ (None by default)
414438 """
415439 jconf = self ._dictToJavaMap (conf )
416- for k , v in conf .iteritems ():
417- jconf [k ] = v
418440 jrdd = self ._jvm .PythonRDD .hadoopFile (self ._jsc , path , inputFormatClass , keyClass ,
419441 valueClass , keyConverter , valueConverter , jconf )
420442 return RDD (jrdd , self , PickleSerializer ())
@@ -423,9 +445,20 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None,
423445 valueConverter = None , conf = None ):
424446 """
425447 Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary
426- Hadoop configuration,
427- which is passed in as a Python dict. This will be converted into a Configuration in Java.
448+ Hadoop configuration, which is passed in as a Python dict.
449+ This will be converted into a Configuration in Java.
428450 The mechanism is the same as for sc.sequenceFile.
451+
452+ @param inputFormatClass: fully qualified classname of Hadoop InputFormat
453+ (e.g. "org.apache.hadoop.mapred.TextInputFormat")
454+ @param keyClass: fully qualified classname of key Writable class
455+ (e.g. "org.apache.hadoop.io.Text")
456+ @param valueClass: fully qualified classname of value Writable class
457+ (e.g. "org.apache.hadoop.io.LongWritable")
458+ @param keyConverter: (None by default)
459+ @param valueConverter: (None by default)
460+ @param conf: Hadoop configuration, passed in as a dict
461+ (None by default)
429462 """
430463 jconf = self ._dictToJavaMap (conf )
431464 jrdd = self ._jvm .PythonRDD .hadoopRDD (self ._jsc , inputFormatClass , keyClass , valueClass ,
@@ -674,7 +707,6 @@ def _test():
674707 globs = globals ().copy ()
675708 globs ['sc' ] = SparkContext ('local[4]' , 'PythonTest' , batchSize = 2 )
676709 globs ['tempdir' ] = tempfile .mkdtemp ()
677- globs ['sc' ]._jvm .WriteInputFormatTestDataGenerator .generateData (globs ['tempdir' ], globs ['sc' ]._jsc )
678710 atexit .register (lambda : shutil .rmtree (globs ['tempdir' ]))
679711 (failure_count , test_count ) = doctest .testmod (globs = globs , optionflags = doctest .ELLIPSIS )
680712 globs ['sc' ].stop ()
0 commit comments