Skip to content

Commit 0c1e762

Browse files
Update rdd pipe method for checkCode
use boolean checkCode rather than more complicated mode optional argument. Also add param to docstring
1 parent ab9a2e1 commit 0c1e762

File tree

1 file changed

+4
-13
lines changed

1 file changed

+4
-13
lines changed

python/pyspark/rdd.py

Lines changed: 4 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -687,24 +687,15 @@ def groupBy(self, f, numPartitions=None):
687687
return self.map(lambda x: (f(x), x)).groupByKey(numPartitions)
688688

689689
@ignore_unicode_prefix
690-
def pipe(self, command, env={}, mode='permissive'):
690+
def pipe(self, command, env={}, checkCode=False):
691691
"""
692692
Return an RDD created by piping elements to a forked external process.
693693
694694
>>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect()
695695
[u'1', u'2', u'', u'3']
696+
697+
:param checkCode: whether or not to check the return value of the shell command.
696698
"""
697-
if mode == 'permissive':
698-
def fail_condition(x):
699-
return False
700-
elif mode == 'strict':
701-
def fail_condition(x):
702-
return x != 0
703-
elif mode == 'grep':
704-
def fail_condition(x):
705-
return x != 0 and x != 1
706-
else:
707-
raise ValueError("mode must be one of 'permissive', 'strict' or 'grep'.")
708699

709700
def func(iterator):
710701
pipe = Popen(
@@ -719,7 +710,7 @@ def pipe_objs(out):
719710

720711
def check_return_code():
721712
pipe.wait()
722-
if fail_condition(pipe.returncode):
713+
if checkCode and pipe.returncode:
723714
raise Exception("Pipe function `%s' exited "
724715
"with error code %d" % (command, pipe.returncode))
725716
else:

0 commit comments

Comments
 (0)