Skip to content

Conversation

@andrewor14
Copy link
Contributor

This reverts a change introduced in 3870248, which redirected all stderr to the OS pipe instead of directly to the bin/pyspark shell output. This causes a simple job to hang in two ways:

  1. If the cluster is not configured correctly or does not have enough resources, the job hangs without producing any output, because the relevant warning messages are masked.
  2. If the stderr volume is large, this could lead to a deadlock if we redirect everything to the OS pipe. From the python docs:
Note Do not use stdout=PIPE or stderr=PIPE with this function as that can deadlock
based on the child process output volume. Use Popen with the communicate() method
when you need pipes.

Note that we cannot remove stdout=PIPE in a similar way, because we currently use it to communicate the py4j port. However, it should be fine (as it has been for a long time) because we do not produce a ton of traffic through stdout.

That commit was not merged in branch-1.0, so this fix is for master only.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@andrewor14 andrewor14 changed the title [SPARK-2242] HOTFIX: Do not mask pyspark stderr from output [SPARK-2242] HOTFIX: pyspark shell hangs on simple job Jun 23, 2014
andrewor14 referenced this pull request Jun 23, 2014
If the gateway process fails to start correctly (e.g., because JAVA_HOME isn't set correctly, there's no Spark jar, etc.), right now pyspark fails because of a very difficult-to-understand error, where we try to parse stdout to get the port where Spark started and there's nothing there. This commit properly catches the error and throws an exception that includes the stderr output for much easier debugging.

Thanks to @shivaram and @stogers for helping to fix this issue!

Author: Kay Ousterhout <[email protected]>

Closes #383 from kayousterhout/pyspark and squashes the following commits:

36dd54b [Kay Ousterhout] [SPARK-1466] Raise exception if Gateway process doesn't start.
@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16015/

@AmplabJenkins
Copy link

Merged build finished.

@AmplabJenkins
Copy link

Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16016/

@andrewor14
Copy link
Contributor Author

Jenkins test this please

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16017/

@kayousterhout
Copy link
Contributor

Sorry about this and thanks for fixing it @andrewor14 ! Just one change -- I think you should also delete proc.stderr.readLines() on line 57, since that won't return anything now.

@kayousterhout
Copy link
Contributor

BTW did you look into using communicate() at all? If not I'll look into that to do a long-term fix later today.

@andrewor14
Copy link
Contributor Author

Hm no I have not, though I'm not sure if we can use communicate() because I think that waits till the end of the process before returning any stdout or stderr, so that's not quite applicable to the shell (I could be totally making this up, but that was my impression when I ran into something similar before)

@andrewor14
Copy link
Contributor Author

Unless we do a communicate() only for the except case. It would still be good to print out why "Launching GatewayServer failed with exit code"

@kayousterhout
Copy link
Contributor

Yeah the latter thing is what I was thinking

Before:
  ValueError: invalid literal for int() with base 10
After:
  Launching GatewayServer failed because of stdout interference.
  Silence the following and try again.
@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@andrewor14
Copy link
Contributor Author

Looks like if an exception is thrown because of casting the output to int (rather than reading the output of the process itself), then the process exit code returned is None because the process hasn't terminated yet. We should add a separate except for bad casting, which happens if there is other output other than the py4j port, and this is the most likely cause of an exception in the first place. I added this in the latest commit.

I haven't invested a ton of time on figuring out how to use communicate() correctly, though I'm not entirely sure if we need it, actually. Because our error messages actually surface on stdout, and our output messages on stderr, I think what we have now is sufficient as a fix. In the long term we might want to avoid using stdout to communicate the py4j port entirely, but maybe instead write it to a temporary file so pyspark doesn't die on us if we echo a seemingly benign "hello" somewhere :)

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished.

@AmplabJenkins
Copy link

Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16040/

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@andrewor14
Copy link
Contributor Author

@mattf The second issue that your changes don't address is that the existing code also masks stderr output, which contains important Spark logging information. Also, I tried your patch out, and bin/pyspark still hangs when an exception is thrown.

@andrewor14
Copy link
Contributor Author

Here's the output of the latest commit in the event of stdout interference:

//... stack trace here
Exception: Launching GatewayServer failed! (Warning: unexpected output detected.)

Dearly beloved, did you forget to silence this message?
Here's how you do it. First you climb the ladder of--zzzz
52254

@kayousterhout
Copy link
Contributor

This looks perfect!

On Tue, Jun 24, 2014 at 12:44 PM, andrewor14 [email protected]
wrote:

Here's the output of the latest commit in the event of stdout interference:

//... stack trace here
Exception: Launching GatewayServer failed! (Warning: unexpected output detected.)

Dearly beloved, did you forget to silence this message?
Here's how you do it. First you climb the ladder of--zzzz
52254


Reply to this email directly or view it on GitHub
#1178 (comment).

@AmplabJenkins
Copy link

Merged build finished.

@AmplabJenkins
Copy link

Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16064/

@andrewor14
Copy link
Contributor Author

Jenkins, test this please

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished.

@AmplabJenkins
Copy link

Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16065/

@andrewor14
Copy link
Contributor Author

Jenkins, test this please

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16069/

@mattf
Copy link

mattf commented Jun 25, 2014

@andrewor14 what's the reproducer for the "hangs when an exception is thrown" case?

@andrewor14
Copy link
Contributor Author

@mattf try adding the following lines to bin/spark-class (anywhere near the lines with SPARK_MEM is fine):

echo "Hello. This goes to stdout..."
echo "and interferes with pyspark reading the py4j port as an int"

What pyspark tries to do is to read the string "Hello. This goes to stdout..." as an int and throws an exception. I think whether it hangs depends on the environment, but on mine I ran into the deadlock the python docs warned against.

@mattf
Copy link

mattf commented Jun 25, 2014

@andrewor14 thanks, i've been able to reproduce a hang when spark-class outputs something other than the port #

@mengxr
Copy link
Contributor

mengxr commented Jun 25, 2014

This looks good to me. I'm going merge it since pyspark is broken without this patch.

@asfgit asfgit closed this in 5603e4c Jun 25, 2014
@andrewor14 andrewor14 deleted the fix-python branch June 25, 2014 20:54
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
This reverts a change introduced in 3870248, which redirected all stderr to the OS pipe instead of directly to the `bin/pyspark` shell output. This causes a simple job to hang in two ways:

1. If the cluster is not configured correctly or does not have enough resources, the job hangs without producing any output, because the relevant warning messages are masked.
2. If the stderr volume is large, this could lead to a deadlock if we redirect everything to the OS pipe. From the [python docs](https://docs.python.org/2/library/subprocess.html):

```
Note Do not use stdout=PIPE or stderr=PIPE with this function as that can deadlock
based on the child process output volume. Use Popen with the communicate() method
when you need pipes.
```

Note that we cannot remove `stdout=PIPE` in a similar way, because we currently use it to communicate the py4j port. However, it should be fine (as it has been for a long time) because we do not produce a ton of traffic through `stdout`.

That commit was not merged in branch-1.0, so this fix is for master only.

Author: Andrew Or <[email protected]>

Closes apache#1178 from andrewor14/fix-python and squashes the following commits:

e68e870 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-python
20849a8 [Andrew Or] Tone down stdout interference message
a09805b [Andrew Or] Return more than 1 line of error message to user
6dfbd1e [Andrew Or] Don't swallow original exception
0d1861f [Andrew Or] Provide more helpful output if stdout is garbled
21c9d7c [Andrew Or] Do not mask stderr from output
wangyum pushed a commit that referenced this pull request May 26, 2023
* [CARMEL-6426] Adjust scan parallelism dynamically improvement

* Use  ExplodeBase instead of Explode
mapr-devops pushed a commit to mapr/spark that referenced this pull request May 8, 2025
…-36944 (…" (apache#1178)

This reverts commit 4272502245483e4f7680c1b3faa8b02bf0e466c6.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants