Skip to content

Commit 085b55f

Browse files
committed
Move input format tests to tests.py and clean up docs
1 parent 43eb728 commit 085b55f

File tree

2 files changed

+196
-30
lines changed

2 files changed

+196
-30
lines changed

python/pyspark/context.py

Lines changed: 62 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -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()

python/pyspark/tests.py

Lines changed: 134 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -202,6 +202,140 @@ def func(x):
202202
self.sc.parallelize([1]).foreach(func)
203203

204204

205+
class TestInputFormat(PySparkTestCase):
206+
207+
def setUp(self):
208+
PySparkTestCase.setUp(self)
209+
self.tempdir = tempfile.NamedTemporaryFile(delete=False)
210+
os.unlink(self.tempdir.name)
211+
self.sc._jvm.WriteInputFormatTestDataGenerator.generateData(self.tempdir.name, self.sc._jsc)
212+
213+
def tearDown(self):
214+
PySparkTestCase.tearDown(self)
215+
shutil.rmtree(self.tempdir.name)
216+
217+
def test_sequencefiles(self):
218+
basepath = self.tempdir.name
219+
ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/",
220+
"org.apache.hadoop.io.IntWritable",
221+
"org.apache.hadoop.io.Text").collect())
222+
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
223+
self.assertEqual(ints, ei)
224+
225+
doubles = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfdouble/",
226+
"org.apache.hadoop.io.DoubleWritable",
227+
"org.apache.hadoop.io.Text").collect())
228+
ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')]
229+
self.assertEqual(doubles, ed)
230+
231+
text = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sftext/",
232+
"org.apache.hadoop.io.Text",
233+
"org.apache.hadoop.io.Text").collect())
234+
et = [(u'1', u'aa'),
235+
(u'1', u'aa'),
236+
(u'2', u'aa'),
237+
(u'2', u'bb'),
238+
(u'2', u'bb'),
239+
(u'3', u'cc')]
240+
self.assertEqual(text, et)
241+
242+
bools = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbool/",
243+
"org.apache.hadoop.io.IntWritable",
244+
"org.apache.hadoop.io.BooleanWritable").collect())
245+
eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)]
246+
self.assertEqual(bools, eb)
247+
248+
nulls = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfnull/",
249+
"org.apache.hadoop.io.IntWritable",
250+
"org.apache.hadoop.io.BooleanWritable").collect())
251+
en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)]
252+
self.assertEqual(nulls, en)
253+
254+
maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/",
255+
"org.apache.hadoop.io.IntWritable",
256+
"org.apache.hadoop.io.MapWritable").collect())
257+
em = [(1, {2.0: u'aa'}),
258+
(1, {3.0: u'bb'}),
259+
(2, {1.0: u'aa'}),
260+
(2, {1.0: u'cc'}),
261+
(2, {3.0: u'bb'}),
262+
(3, {2.0: u'dd'})]
263+
self.assertEqual(maps, em)
264+
265+
clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/",
266+
"org.apache.hadoop.io.Text",
267+
"org.apache.spark.api.python.TestWritable").collect())
268+
ec = (u'1',
269+
{u'__class__': u'org.apache.spark.api.python.TestWritable',
270+
u'double': 54.0, u'int': 123, u'str': u'test1'})
271+
self.assertEqual(clazz[0], ec)
272+
273+
def test_oldhadoop(self):
274+
basepath = self.tempdir.name
275+
ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/",
276+
"org.apache.hadoop.mapred.SequenceFileInputFormat",
277+
"org.apache.hadoop.io.IntWritable",
278+
"org.apache.hadoop.io.Text").collect())
279+
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
280+
self.assertEqual(ints, ei)
281+
282+
hello = self.sc.hadoopFile("python/test_support/hello.txt",
283+
"org.apache.hadoop.mapred.TextInputFormat",
284+
"org.apache.hadoop.io.LongWritable",
285+
"org.apache.hadoop.io.Text").collect()
286+
result = [(0, u'Hello World!')]
287+
self.assertEqual(hello, result)
288+
289+
def test_newhadoop(self):
290+
basepath = self.tempdir.name
291+
ints = sorted(self.sc.newAPIHadoopFile(
292+
basepath + "/sftestdata/sfint/",
293+
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
294+
"org.apache.hadoop.io.IntWritable",
295+
"org.apache.hadoop.io.Text").collect())
296+
ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')]
297+
self.assertEqual(ints, ei)
298+
299+
hello = self.sc.newAPIHadoopFile("python/test_support/hello.txt",
300+
"org.apache.hadoop.mapreduce.lib.input.TextInputFormat",
301+
"org.apache.hadoop.io.LongWritable",
302+
"org.apache.hadoop.io.Text").collect()
303+
result = [(0, u'Hello World!')]
304+
self.assertEqual(hello, result)
305+
306+
def test_newolderror(self):
307+
basepath = self.tempdir.name
308+
newFromOld = self.sc.hadoopFile(
309+
basepath + "/sftestdata/sfint/",
310+
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
311+
"org.apache.hadoop.io.IntWritable",
312+
"org.apache.hadoop.io.Text")
313+
self.assertRaises(Exception, lambda: newFromOld.collect())
314+
315+
oldFromNew = self.sc.newAPIHadoopFile(basepath + "/sftestdata/sfint/",
316+
"org.apache.hadoop.mapred.SequenceFileInputFormat",
317+
"org.apache.hadoop.io.IntWritable",
318+
"org.apache.hadoop.io.Text")
319+
self.assertRaises(Exception, lambda: oldFromNew.collect())
320+
321+
def test_bad_inputs(self):
322+
basepath = self.tempdir.name
323+
self.assertRaises(Exception, lambda: self.sc.sequenceFile(
324+
basepath + "/sftestdata/sfint/",
325+
"org.apache.hadoop.io.NotValidWritable",
326+
"org.apache.hadoop.io.Text"))
327+
self.assertRaises(Exception, lambda: self.sc.hadoopFile(
328+
basepath + "/sftestdata/sfint/",
329+
"org.apache.hadoop.mapred.NotValidInputFormat",
330+
"org.apache.hadoop.io.IntWritable",
331+
"org.apache.hadoop.io.Text"))
332+
self.assertRaises(Exception, lambda: self.sc.newAPIHadoopFile(
333+
basepath + "/sftestdata/sfint/",
334+
"org.apache.hadoop.mapreduce.lib.input.NotValidInputFormat",
335+
"org.apache.hadoop.io.IntWritable",
336+
"org.apache.hadoop.io.Text"))
337+
338+
205339
class TestDaemon(unittest.TestCase):
206340
def connect(self, port):
207341
from socket import socket, AF_INET, SOCK_STREAM

0 commit comments

Comments
 (0)