Skip to content

Commit

Permalink
[SPARK-26080][PYTHON] Skips Python resource limit on Windows in Pytho…
Browse files Browse the repository at this point in the history
…n worker

## What changes were proposed in this pull request?

`resource` package is a Unix specific package. See https://docs.python.org/2/library/resource.html and https://docs.python.org/3/library/resource.html.

Note that we document Windows support:

> Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS).

This should be backported into branch-2.4 to restore Windows support in Spark 2.4.1.

## How was this patch tested?

Manually mocking the changed logics.

Closes apache#23055 from HyukjinKwon/SPARK-26080.

Lead-authored-by: hyukjinkwon <[email protected]>
Co-authored-by: Hyukjin Kwon <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
  • Loading branch information
HyukjinKwon committed Dec 2, 2018
1 parent c7d95cc commit 9cda9a8
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 7 deletions.
2 changes: 2 additions & 0 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,8 @@ of the most common options to set are:
and it is up to the application to avoid exceeding the overhead memory space
shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory
is added to executor resource requests.

NOTE: Python memory usage may not be limited on platforms that do not support resource limiting, such as Windows.
</td>
</tr>
<tr>
Expand Down
19 changes: 12 additions & 7 deletions python/pyspark/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,12 @@
import os
import sys
import time
import resource
# 'resource' is a Unix specific module.
has_resource_module = True
try:
import resource
except ImportError:
has_resource_module = False
import socket
import traceback

Expand Down Expand Up @@ -268,9 +273,9 @@ def main(infile, outfile):

# set up memory limits
memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1"))
total_memory = resource.RLIMIT_AS
try:
if memory_limit_mb > 0:
if memory_limit_mb > 0 and has_resource_module:
total_memory = resource.RLIMIT_AS
try:
(soft_limit, hard_limit) = resource.getrlimit(total_memory)
msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit)
print(msg, file=sys.stderr)
Expand All @@ -283,9 +288,9 @@ def main(infile, outfile):
print(msg, file=sys.stderr)
resource.setrlimit(total_memory, (new_limit, new_limit))

except (resource.error, OSError, ValueError) as e:
# not all systems support resource limits, so warn instead of failing
print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr)
except (resource.error, OSError, ValueError) as e:
# not all systems support resource limits, so warn instead of failing
print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr)

# initialize global state
taskContext = None
Expand Down

0 comments on commit 9cda9a8

Please sign in to comment.