Commit graph

23 commits

Author SHA1 Message Date
WeichenXu 3b14088541 [SPARK-26175][PYTHON] Redirect the standard input of the forked child to devnull in daemon
## What changes were proposed in this pull request?

PySpark worker daemon reads from stdin the worker PIDs to kill. 1bb60ab839/python/pyspark/daemon.py (L127)

However, the worker process is a forked process from the worker daemon process and we didn't close stdin on the child after fork. This means the child and user program can read stdin as well, which blocks daemon from receiving the PID to kill. This can cause issues because the task reaper might detect the task was not terminated and eventually kill the JVM.

This PR fix this by redirecting the standard input of the forked child to devnull.

## How was this patch tested?

Manually test.

In `pyspark`, run:
```
import subprocess
def task(_):
  subprocess.check_output(["cat"])

sc.parallelize(range(1), 1).mapPartitions(task).count()
```

Before:
The job will get stuck and press Ctrl+C to exit the job but the python worker process do not exit.
After:
The job finish correctly. The "cat" print nothing (because the dummay stdin is "/dev/null").
The python worker process exit normally.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25138 from WeichenXu123/SPARK-26175.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-31 09:10:24 +09:00
HyukjinKwon 26998b86c1 [SPARK-27870][SQL][PYTHON] Add a runtime buffer size configuration for Pandas UDFs
## What changes were proposed in this pull request?

This PR is an alternative approach for #24734.

This PR fixes two things:

1. Respects `spark.buffer.size` in Python workers.
2. Adds a runtime buffer size configuration for Pandas UDFs, `spark.sql.pandas.udf.buffer.size` (which falls back to `spark.buffer.size`.

## How was this patch tested?

Manually tested:

```python
import time
from pyspark.sql.functions import *

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```

```
consume time: 62.68265891075134
```

```python
import time
from pyspark.sql.functions import *

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```

```
consume time: 34.00594782829285
```

Closes #24826 from HyukjinKwon/SPARK-27870.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:56:22 +09:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
Benjamin Peterson 7013eea11c [SPARK-23522][PYTHON] always use sys.exit over builtin exit
The exit() builtin is only for interactive use. applications should use sys.exit().

## What changes were proposed in this pull request?

All usage of the builtin `exit()` function is replaced by `sys.exit()`.

## How was this patch tested?

I ran `python/run-tests`.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Benjamin Peterson <benjamin@python.org>

Closes #20682 from benjaminp/sys-exit.
2018-03-08 20:38:34 +09:00
Davies Liu 04e44b37cc [SPARK-4897] [PySpark] Python 3 support
This PR update PySpark to support Python 3 (tested with 3.4).

Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped.

TODO: ec2/spark-ec2.py is not fully tested with python3.

Author: Davies Liu <davies@databricks.com>
Author: twneale <twneale@gmail.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #5173 from davies/python3 and squashes the following commits:

d7d6323 [Davies Liu] fix tests
6c52a98 [Davies Liu] fix mllib test
99e334f [Davies Liu] update timeout
b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
cafd5ec [Davies Liu] adddress comments from @mengxr
bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
179fc8d [Davies Liu] tuning flaky tests
8c8b957 [Davies Liu] fix ResourceWarning in Python 3
5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
4006829 [Davies Liu] fix test
2fc0066 [Davies Liu] add python3 path
71535e9 [Davies Liu] fix xrange and divide
5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ed498c8 [Davies Liu] fix compatibility with python 3
820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ad7c374 [Davies Liu] fix mllib test and warning
ef1fc2f [Davies Liu] fix tests
4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
59bb492 [Davies Liu] fix tests
1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ca0fdd3 [Davies Liu] fix code style
9563a15 [Davies Liu] add imap back for python 2
0b1ec04 [Davies Liu] make python examples work with Python 3
d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
a716d34 [Davies Liu] test with python 3.4
f1700e8 [Davies Liu] fix test in python3
671b1db [Davies Liu] fix test in python3
692ff47 [Davies Liu] fix flaky test
7b9699f [Davies Liu] invalidate import cache for Python 3.3+
9c58497 [Davies Liu] fix kill worker
309bfbf [Davies Liu] keep compatibility
5707476 [Davies Liu] cleanup, fix hash of string in 3.3+
8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
f53e1f0 [Davies Liu] fix tests
70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3
a39167e [Davies Liu] support customize class in __main__
814c77b [Davies Liu] run unittests with python 3
7f4476e [Davies Liu] mllib tests passed
d737924 [Davies Liu] pass ml tests
375ea17 [Davies Liu] SQL tests pass
6cc42a9 [Davies Liu] rename
431a8de [Davies Liu] streaming tests pass
78901a7 [Davies Liu] fix hash of serializer in Python 3
24b2f2e [Davies Liu] pass all RDD tests
35f48fe [Davies Liu] run future again
1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py
6e3c21d [Davies Liu] make cloudpickle work with Python3
2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run
1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out
7354371 [twneale] buffer --> memoryview  I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work.
b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?).
f40d925 [twneale] xrange --> range
e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206
79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper
2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3
854be27 [Josh Rosen] Run `futurize` on Python code:
7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
2015-04-16 16:20:57 -07:00
Davies Liu 712679a7b4 [SPARK-6294] fix hang when call take() in JVM on PythonRDD
The Thread.interrupt() can not terminate the thread in some cases, so we should not wait for the writerThread of PythonRDD.

This PR also ignore some exception during clean up.

cc JoshRosen mengxr

Author: Davies Liu <davies@databricks.com>

Closes #4987 from davies/fix_take and squashes the following commits:

4488f1a [Davies Liu] fix hang when call take() in JVM on PythonRDD
2015-03-12 01:34:38 -07:00
Davies Liu e41786c774 [SPARK-4088] [PySpark] Python worker should exit after socket is closed by JVM
In case of take() or exception in Python, python worker may exit before JVM read() all the response, then the write thread may raise "Connection reset" exception.

Python should always wait JVM to close the socket first.

cc JoshRosen This is a warm fix, or the tests will be flaky, sorry for that.

Author: Davies Liu <davies@databricks.com>

Closes #2941 from davies/fix_exit and squashes the following commits:

9d4d21e [Davies Liu] fix race
2014-10-25 01:20:39 -07:00
Davies Liu e595c8d08a [SPARK-3993] [PySpark] fix bug while reuse worker after take()
After take(), maybe there are some garbage left in the socket, then next task assigned to this worker will hang because of corrupted data.

We should make sure the socket is clean before reuse it, write END_OF_STREAM at the end, and check it after read out all result from python.

Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #2838 from davies/fix_reuse and squashes the following commits:

8872914 [Davies Liu] fix tests
660875b [Davies Liu] fix bug while reuse worker after take()
2014-10-23 17:20:00 -07:00
Davies Liu 2aea0da84c [SPARK-3030] [PySpark] Reuse Python worker
Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts.

This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming.

For a job with broadcast (43M after compress):
```
    b = sc.broadcast(set(range(30000000)))
    print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count()
```
It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks.

It's enabled by default, could be disabled by `spark.python.worker.reuse = false`.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2259 from davies/reuse-worker and squashes the following commits:

f11f617 [Davies Liu] Merge branch 'master' into reuse-worker
3939f20 [Davies Liu] fix bug in serializer in mllib
cf1c55e [Davies Liu] address comments
3133a60 [Davies Liu] fix accumulator with reused worker
760ab1f [Davies Liu] do not reuse worker if there are any exceptions
7abb224 [Davies Liu] refactor: sychronized with itself
ac3206e [Davies Liu] renaming
8911f44 [Davies Liu] synchronized getWorkerBroadcasts()
6325fc1 [Davies Liu] bugfix: bid >= 0
e0131a2 [Davies Liu] fix name of config
583716e [Davies Liu] only reuse completed and not interrupted worker
ace2917 [Davies Liu] kill python worker after timeout
6123d0f [Davies Liu] track broadcasts for each worker
8d2f08c [Davies Liu] reuse python worker
2014-09-13 16:22:04 -07:00
Davies Liu 71af030b46 [SPARK-3094] [PySpark] compatitable with PyPy
After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example:

```
PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py
```

The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks:

 Job | CPython 2.7 | PyPy 2.3.1  | Speed up
 ------- | ------------ | ------------- | -------
 Word Count | 41s   | 15s  | 2.7x
 Sort | 46s |  44s | 1.05x
 Stats | 174s | 3.6s | 48x

Here is the code used for benchmark:

```python
rdd = sc.textFile("text")
def wordcount():
    rdd.flatMap(lambda x:x.split('/'))\
        .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap()
def sort():
    rdd.sortBy(lambda x:x, 1).count()
def stats():
    sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats()
```

Author: Davies Liu <davies.liu@gmail.com>

Closes #2144 from davies/pypy and squashes the following commits:

9aed6c5 [Davies Liu] use protocol 2 in CloudPickle
4bc1f04 [Davies Liu] refactor
b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way
3ca2351 [Davies Liu] Merge branch 'master' into pypy
fae8b19 [Davies Liu] improve attrgetter, add tests
591f830 [Davies Liu] try to run tests with PyPy in run-tests
c8d62ba [Davies Liu] cleanup
f651fd0 [Davies Liu] fix tests using array with PyPy
1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways
3c1dbfe [Davies Liu] Merge branch 'master' into pypy
42fb5fa [Davies Liu] Merge branch 'master' into pypy
cb2d724 [Davies Liu] fix tests
9986692 [Davies Liu] Merge branch 'master' into pypy
25b4ca7 [Davies Liu] support PyPy
2014-09-12 18:42:50 -07:00
Davies Liu 28dcbb531a [SPARK-2898] [PySpark] fix bugs in deamon.py
1. do not use signal handler for SIGCHILD, it's easy to cause deadlock
2. handle EINTR during accept()
3. pass errno into JVM
4. handle EAGAIN during fork()

Now, it can pass 50k tasks tests in 180 seconds.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1842 from davies/qa and squashes the following commits:

f0ea451 [Davies Liu] fix lint
03a2e8c [Davies Liu] cleanup dead children every seconds
32cb829 [Davies Liu] fix lint
0cd0817 [Davies Liu] fix bugs in deamon.py
2014-08-10 13:00:38 -07:00
Nicholas Chammas d614967b0b [SPARK-2627] [PySpark] have the build enforce PEP 8 automatically
As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that.

Notes:
* We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server.
* I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request.
* I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete.
* Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo.

Author: Nicholas Chammas <nicholas.chammas@gmail.com>
Author: nchammas <nicholas.chammas@gmail.com>

Closes #1744 from nchammas/master and squashes the following commits:

274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes
983d963 [nchammas] Merge pull request #5 from apache/master
1db5314 [nchammas] Merge pull request #4 from apache/master
0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes
bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing
6db9a44 [nchammas] Merge pull request #3 from apache/master
7b4750e [Nicholas Chammas] merge upstream changes
91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks
44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files
b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily
bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes
9da347f [nchammas] Merge pull request #2 from apache/master
aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks
d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine
dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime
a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections
21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8
6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes
fe57ed0 [Nicholas Chammas] removing merge conflict backups
9c01d4c [nchammas] Merge pull request #1 from apache/master
9a66cb0 [Nicholas Chammas] resolving merge conflicts
a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes
beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status
723ed39 [Nicholas Chammas] always delete the report file
0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests
12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter
61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter
75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 12:58:24 -07:00
Davies Liu 55349f9fe8 [SPARK-1740] [PySpark] kill the python worker
Kill only the python worker related to cancelled tasks.

The daemon will start a background thread to monitor all the opened sockets for all workers. If the socket is closed by JVM, this thread will kill the worker.

When an task is cancelled, the socket to worker will be closed, then the worker will be killed by deamon.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1643 from davies/kill and squashes the following commits:

8ffe9f3 [Davies Liu] kill worker by deamon, because runtime.exec() is too heavy
46ca150 [Davies Liu] address comment
acd751c [Davies Liu] kill the worker when task is canceled
2014-08-03 15:52:00 -07:00
Josh Rosen e8e0fd691a [SPARK-2764] Simplify daemon.py process structure
Curently, daemon.py forks a pool of numProcessors subprocesses, and those processes fork themselves again to create the actual Python worker processes that handle data.

I think that this extra layer of indirection is unnecessary and adds a lot of complexity.  This commit attempts to remove this middle layer of subprocesses by launching the workers directly from daemon.py.

See https://github.com/mesos/spark/pull/563 for the original PR that added daemon.py, where I raise some issues with the current design.

Author: Josh Rosen <joshrosen@apache.org>

Closes #1680 from JoshRosen/pyspark-daemon and squashes the following commits:

5abbcb9 [Josh Rosen] Replace magic number: 4 -> EINTR
5495dff [Josh Rosen] Throw IllegalStateException if worker launch fails.
b79254d [Josh Rosen] Detect failed fork() calls; improve error logging.
282c2c4 [Josh Rosen] Remove daemon.py exit logging, since it caused problems:
8554536 [Josh Rosen] Fix daemon’s shutdown(); log shutdown reason.
4e0fab8 [Josh Rosen] Remove shared-memory exit_flag; don't die on worker death.
e9892b4 [Josh Rosen] [WIP] [SPARK-2764] Simplify daemon.py process structure.
2014-08-01 19:38:21 -07:00
Nicholas Chammas 5d16d5bbfd [SPARK-2470] PEP8 fixes to PySpark
This pull request aims to resolve all outstanding PEP8 violations in PySpark.

Author: Nicholas Chammas <nicholas.chammas@gmail.com>
Author: nchammas <nicholas.chammas@gmail.com>

Closes #1505 from nchammas/master and squashes the following commits:

98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle
cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses
e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break
9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses
22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses
24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest
7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py
8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py
b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py
d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py
aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py
1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py
95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py
a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib
c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py
d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py
81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py
1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py
7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py
ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py
f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py
a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py
f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py
4dd148f [nchammas] Merge pull request #5 from apache/master
f7e4581 [Nicholas Chammas] unrelated pep8 fix
a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently
de7292a [nchammas] Merge pull request #4 from apache/master
2e4fe00 [nchammas] Merge pull request #3 from apache/master
89fde08 [nchammas] Merge pull request #2 from apache/master
69f6e22 [Nicholas Chammas] PEP8 fixes
2627247 [Nicholas Chammas] broke up lines before they hit 100 chars
6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names
69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-21 22:30:53 -07:00
Matthew Farrellee 3c104c79d2 [SPARK-1394] Remove SIGCHLD handler in worker subprocess
It should not be the responsibility of the worker subprocess, which
does not intentionally fork, to try and cleanup child processes. Doing
so is complex and interferes with operations such as
platform.system().

If it is desirable to have tighter control over subprocesses, then
namespaces should be used and it should be the manager's resposibility
to handle cleanup.

Author: Matthew Farrellee <matt@redhat.com>

Closes #1247 from mattf/SPARK-1394 and squashes the following commits:

c36f308 [Matthew Farrellee] [SPARK-1394] Remove SIGCHLD handler in worker subprocess
2014-06-28 18:39:27 -07:00
Aaron Davidson 3308722ca0 SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions
This patch includes several cleanups to PythonRDD, focused around fixing [SPARK-1579](https://issues.apache.org/jira/browse/SPARK-1579) cleanly. Listed in order of approximate importance:

- The Python daemon waits for Spark to close the socket before exiting,
  in order to avoid causing spurious IOExceptions in Spark's
  `PythonRDD::WriterThread`.
- Removes the Python Monitor Thread, which polled for task cancellations
  in order to kill the Python worker. Instead, we do this in the
  onCompleteCallback, since this is guaranteed to be called during
  cancellation.
- Adds a "completed" variable to TaskContext to avoid the issue noted in
  [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), where onCompleteCallbacks may be execution-order dependent.
  Along with this, I removed the "context.interrupted = true" flag in
  the onCompleteCallback.
- Extracts PythonRDD::WriterThread to its own class.

Since this patch provides an alternative solution to [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), I did test it with

```
sc.textFile("latlon.tsv").take(5)
```

many times without error.

Additionally, in order to test the unswallowed exceptions, I performed

```
sc.textFile("s3n://<big file>").count()
```

and cut my internet during execution. Prior to this patch, we got the "stdin writer exited early" message, which was unhelpful. Now, we get the SocketExceptions propagated through Spark to the user and get proper (though unsuccessful) task retries.

Author: Aaron Davidson <aaron@databricks.com>

Closes #640 from aarondav/pyspark-io and squashes the following commits:

b391ff8 [Aaron Davidson] Detect "clean socket shutdowns" and stop waiting on the socket
c0c49da [Aaron Davidson] SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions
2014-05-07 09:48:31 -07:00
Matei Zaharia af3c9d5042 Add Apache license headers and LICENSE and NOTICE files 2013-07-16 17:21:33 -07:00
root ec31e68d5d Fixed PySpark perf regression by not using socket.makefile(), and improved
debuggability by letting "print" statements show up in the executor's stderr

Conflicts:
	core/src/main/scala/spark/api/python/PythonRDD.scala
2013-07-01 06:26:31 +00:00
Jey Kottalam c75bed0eeb Fix reporting of PySpark exceptions 2013-06-21 12:14:16 -04:00
Jey Kottalam 7c5ff733ee PySpark daemon: fix deadlock, improve error handling 2013-06-21 12:14:16 -04:00
Jey Kottalam 62c4781400 Add tests and fixes for Python daemon shutdown 2013-06-21 12:14:16 -04:00
Jey Kottalam c79a6078c3 Prefork Python worker processes 2013-06-21 12:14:16 -04:00