You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2018/12/02 09:41:13 UTC
spark git commit: [SPARK-26080][PYTHON] Skips Python resource limit
on Windows in Python worker
Repository: spark
Updated Branches:
refs/heads/master c7d95cced -> 9cda9a892
[SPARK-26080][PYTHON] Skips Python resource limit on Windows in Python 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 #23055 from HyukjinKwon/SPARK-26080.
Lead-authored-by: hyukjinkwon <gu...@apache.org>
Co-authored-by: Hyukjin Kwon <gu...@apache.org>
Signed-off-by: Hyukjin Kwon <gu...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9cda9a89
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9cda9a89
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9cda9a89
Branch: refs/heads/master
Commit: 9cda9a892d03f60a76cd5d9b4546e72c50962c85
Parents: c7d95cc
Author: hyukjinkwon <gu...@apache.org>
Authored: Sun Dec 2 17:41:08 2018 +0800
Committer: Hyukjin Kwon <gu...@apache.org>
Committed: Sun Dec 2 17:41:08 2018 +0800
----------------------------------------------------------------------
docs/configuration.md | 2 ++
python/pyspark/worker.py | 19 ++++++++++++-------
2 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/9cda9a89/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 8914bd0..9abbb3f 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -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>
http://git-wip-us.apache.org/repos/asf/spark/blob/9cda9a89/python/pyspark/worker.py
----------------------------------------------------------------------
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index 8c59f1f..953b468 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -22,7 +22,12 @@ from __future__ import print_function
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
@@ -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)
@@ -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
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org