-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-7735] [pyspark] Raise Exception on non-zero exit from pipe commands #6262
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 1 commit
Commits
Show all changes
24 commits
Select commit
Hold shift + click to select a range
f552d49
Catch non-zero exit from pipe commands
megatron-me-uk 5745d85
Remove space to fix style
megatron-me-uk 45f4977
fix line too long style error
megatron-me-uk 0974f98
add space between words in multiline string
megatron-me-uk 1b3dc4e
fix missing space around operator style
megatron-me-uk 8db4073
Add a test for rdd pipe functions
megatron-me-uk cc1a73d
fix style issues in pipe test
megatron-me-uk 3ab8c7a
remove whitespace for style
megatron-me-uk 3344a21
wrap assertRaises with QuietTest
megatron-me-uk 491d3fc
Pass a function handle to assertRaises
megatron-me-uk 4153b02
fix list.sort returns None
megatron-me-uk 8ed89a6
Chain generators to prevent potential deadlock
megatron-me-uk 0486ae3
style fixes
megatron-me-uk 8a9ef9c
make check_return_code an iterator
megatron-me-uk a0c0161
fix generator issue
megatron-me-uk 34fcdc3
add optional argument 'mode' for rdd.pipe
megatron-me-uk a307d13
update rdd tests to test pipe modes
megatron-me-uk b0ac3a4
Merge pull request #1 from megatron-me-uk/megatron-me-uk-patch-1
megatron-me-uk eb4801c
fix fail_condition
megatron-me-uk ab9a2e1
Update rdd pipe tests for checkCode
megatron-me-uk 0c1e762
Update rdd pipe method for checkCode
megatron-me-uk 574b564
Merge pull request #2 from megatron-me-uk/patch-4
megatron-me-uk 98fa101
fix blank line style error
megatron-me-uk 04ae1d5
Remove spurious empty line
megatron-me-uk File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
fix list.sort returns None
- Loading branch information
commit 4153b02fdb0ce1c691ec6893c6ad70e990a4e02b
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you also add a test for
grep? It may exit with no-zero codeThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I can do although I think a non-zero code from grep will raise an Exception in the scala implementation. See (that I just found):
https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
I see that grep will return 1 if it doesn't match any lines in a partition. Raising an exception in this case may well be annoying but other shell functions can return 1 on errors (for example http://www.postgresql.org/docs/8.3/static/app-psql.html). I wonder what the best solution is here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could we have an option for this? At least, we should have an option to support
grepon a partition without any matching lines.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This issue can be worked around using
grep target; test $? -le 1although maybe not the best solution.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NAVER - http://www.naver.com/
[email protected] 님께 보내신 메일 <Re: [spark] [SPARK-7735] [pyspark] Raise Exception on non-zero exit from pipe commands (#6262)> 이 다음과 같은 이유로 전송 실패했습니다.
받는 사람이 회원님의 메일을 수신차단 하였습니다.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@davies, I guess that your comment about
grepis due to the fact that the fix here might break programs that used to work in case they relied on the old behavior? I guess that one option might be to add a configuration flag for controlling the exit-code-handling behavior and to update the documentation to address this. Any thoughts?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, that's something in my mind. Maybe add a option to
RDD.pipe()would be friendly, having the default behavior as before.