From 41af409b7bcfe1b3960274c0b3085bcc1f9d1c98 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 18 Jun 2021 12:02:29 +0900 Subject: [PATCH] [SPARK-35303][PYTHON] Enable pinned thread mode by default ### What changes were proposed in this pull request? PySpark added pinned thread mode at https://github.com/apache/spark/pull/24898 to sync Python thread to JVM thread. Previously, one JVM thread could be reused which ends up with messed inheritance hierarchy such as thread local especially when multiple jobs run in parallel. To completely fix this, we should enable this mode by default. ### Why are the changes needed? To correctly support parallel job submission and management. ### Does this PR introduce _any_ user-facing change? Yes, now Python thread is mapped to JVM thread one to one. ### How was this patch tested? Existing tests should cover it. Closes #32429 from HyukjinKwon/SPARK-35303. Authored-by: HyukjinKwon Signed-off-by: Hyukjin Kwon --- .../apache/spark/api/python/Py4JServer.scala | 2 +- docs/job-scheduling.md | 6 +--- .../migration_guide/pyspark_3.1_to_3.2.rst | 6 ++++ python/docs/source/reference/pyspark.rst | 5 +++- python/pyspark/__init__.py | 4 +-- python/pyspark/__init__.pyi | 5 +++- python/pyspark/context.py | 27 ++++------------- python/pyspark/java_gateway.py | 2 +- python/pyspark/serializers.py | 2 +- python/pyspark/shuffle.py | 2 +- python/pyspark/tests/test_context.py | 2 +- python/pyspark/util.py | 4 +-- python/pyspark/util.pyi | 29 +++++++++++++++++++ python/pyspark/worker.py | 2 +- 14 files changed, 60 insertions(+), 38 deletions(-) create mode 100644 python/pyspark/util.pyi diff --git a/core/src/main/scala/org/apache/spark/api/python/Py4JServer.scala b/core/src/main/scala/org/apache/spark/api/python/Py4JServer.scala index db440b1178..2edc492418 100644 --- a/core/src/main/scala/org/apache/spark/api/python/Py4JServer.scala +++ b/core/src/main/scala/org/apache/spark/api/python/Py4JServer.scala @@ -34,7 +34,7 @@ private[spark] class Py4JServer(sparkConf: SparkConf) extends Logging { // Java system properties and such private val localhost = InetAddress.getLoopbackAddress() private[spark] val server = if (sys.env.getOrElse( - "PYSPARK_PIN_THREAD", "false").toLowerCase(Locale.ROOT) == "true") { + "PYSPARK_PIN_THREAD", "true").toLowerCase(Locale.ROOT) == "true") { new py4j.ClientServer.ClientServerBuilder() .authToken(secret) .javaPort(0) diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 4181999876..de72463aa8 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -297,10 +297,6 @@ in each corresponding JVM thread. Due to this limitation, it is unable to set a via `sc.setJobGroup` in a separate PVM thread, which also disallows to cancel the job via `sc.cancelJobGroup` later. -In order to synchronize PVM threads with JVM threads, you should set `PYSPARK_PIN_THREAD` environment variable -to `true`. This pinned thread mode allows one PVM thread has one corresponding JVM thread. With this mode, `pyspark.InheritableThread` is recommended to use together for a PVM thread to inherit the inheritable attributes - such as local properties in a JVM thread. - -Note that `PYSPARK_PIN_THREAD` is currently experimental and not recommended for use in production. + such as local properties in a JVM thread, and to avoid resource leak. diff --git a/python/docs/source/migration_guide/pyspark_3.1_to_3.2.rst b/python/docs/source/migration_guide/pyspark_3.1_to_3.2.rst index 0d74a4f796..908d4d34d6 100644 --- a/python/docs/source/migration_guide/pyspark_3.1_to_3.2.rst +++ b/python/docs/source/migration_guide/pyspark_3.1_to_3.2.rst @@ -23,3 +23,9 @@ Upgrading from PySpark 3.1 to 3.2 * In Spark 3.2, the PySpark methods from sql, ml, spark_on_pandas modules raise the ``TypeError`` instead of ``ValueError`` when are applied to an param of inappropriate type. * In Spark 3.2, the traceback from Python UDFs, pandas UDFs and pandas function APIs are simplified by default without the traceback from the internal Python workers. In Spark 3.1 or earlier, the traceback from Python workers was printed out. To restore the behavior before Spark 3.2, you can set ``spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled`` to ``false``. + +* In Spark 3.2, pinned thread mode is enabled by default to map each Python thread to the corresponding JVM thread. Previously, + one JVM thread could be reused for multiple Python threads, which resulted in one JVM thread local being shared to multiple Python threads. + Also, note that now ``pyspark.InheritableThread`` or ``pyspark.inheritable_thread_target`` is recommended to use together for a Python thread + to properly inherit the inheritable attributes such as local properties in a JVM thread, and to avoid a potential resource leak issue. + To restore the behavior before Spark 3.2, you can set ``PYSPARK_PIN_THREAD`` environment variable to ``false``. diff --git a/python/docs/source/reference/pyspark.rst b/python/docs/source/reference/pyspark.rst index 49a8ff59ac..6d4d0b5547 100644 --- a/python/docs/source/reference/pyspark.rst +++ b/python/docs/source/reference/pyspark.rst @@ -41,7 +41,7 @@ Public Classes BarrierTaskContext BarrierTaskInfo InheritableThread - + util.VersionUtils Spark Context APIs ------------------ @@ -239,6 +239,7 @@ Management .. autosummary:: :toctree: api/ + inheritable_thread_target SparkConf.contains SparkConf.get SparkConf.getAll @@ -279,3 +280,5 @@ Management BarrierTaskContext.resources BarrierTaskContext.stageId BarrierTaskContext.taskAttemptId + util.VersionUtils.majorMinorVersion + diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 8e36d9fc94..9651b5308d 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -53,7 +53,7 @@ from pyspark.conf import SparkConf from pyspark.rdd import RDD, RDDBarrier from pyspark.files import SparkFiles from pyspark.status import StatusTracker, SparkJobInfo, SparkStageInfo -from pyspark.util import InheritableThread +from pyspark.util import InheritableThread, inheritable_thread_target from pyspark.storagelevel import StorageLevel from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast @@ -125,5 +125,5 @@ __all__ = [ "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "InheritableThread", - "__version__", + "inheritable_thread_target", "__version__", ] diff --git a/python/pyspark/__init__.pyi b/python/pyspark/__init__.pyi index 852de35496..07cbccbd52 100644 --- a/python/pyspark/__init__.pyi +++ b/python/pyspark/__init__.pyi @@ -51,7 +51,10 @@ from pyspark.taskcontext import ( # noqa: F401 BarrierTaskInfo as BarrierTaskInfo, TaskContext as TaskContext, ) -from pyspark.util import InheritableThread as InheritableThread # noqa: F401 +from pyspark.util import ( + InheritableThread as InheritableThread, # noqa: F401 + inheritable_thread_target as inheritable_thread_target, # noqa: F401 +) from pyspark.version import __version__ as __version__ # Compatibility imports diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c685e54931..c96156b210 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1104,13 +1104,8 @@ class SparkContext(object): ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead. - Currently, setting a group ID (set to local properties) with multiple threads - does not properly work. Internally threads on PVM and JVM are not synced, and JVM - thread can be reused for multiple threads on PVM, which fails to isolate local - properties for each thread on PVM. - - To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` - environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. + If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread + local inheritance, and preventing resource leak. Examples -------- @@ -1148,13 +1143,8 @@ class SparkContext(object): Notes ----- - Currently, setting a local property with multiple threads does not properly work. - Internally threads on PVM and JVM are not synced, and JVM thread - can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. - - To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` - environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. + If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread + local inheritance, and preventing resource leak. """ self._jsc.setLocalProperty(key, value) @@ -1171,13 +1161,8 @@ class SparkContext(object): Notes ----- - Currently, setting a job description (set to local properties) with multiple - threads does not properly work. Internally threads on PVM and JVM are not synced, - and JVM thread can be reused for multiple threads on PVM, which fails to isolate - local properties for each thread on PVM. - - To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` - environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. + If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread + local inheritance, and preventing resource leak. """ self._jsc.setJobDescription(value) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index afc3ea740c..bffdc0b7c8 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -130,7 +130,7 @@ def launch_gateway(conf=None, popen_kwargs=None): atexit.register(killChild) # Connect to the gateway (or client server to pin the thread between JVM and Python) - if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": + if os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true": gateway = ClientServer( java_parameters=JavaParameters( port=gateway_port, diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 1b434d3931..0a4527d4e9 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -65,7 +65,7 @@ import pickle pickle_protocol = pickle.HIGHEST_PROTOCOL from pyspark import cloudpickle -from pyspark.util import print_exec +from pyspark.util import print_exec # type: ignore __all__ = ["PickleSerializer", "MarshalSerializer", "UTF8Deserializer"] diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 4ba8462271..c19679a93a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -28,7 +28,7 @@ import sys import heapq from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ CompressedSerializer, AutoBatchedSerializer -from pyspark.util import fail_on_stopiteration +from pyspark.util import fail_on_stopiteration # type: ignore try: diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 4785b4b9f4..50155921e7 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -220,7 +220,7 @@ class ContextTests(unittest.TestCase): def run(): # When thread is pinned, job group should be set for each thread for now. # Local properties seem not being inherited like Scala side does. - if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": + if os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true": sc.setJobGroup('test_progress_api', '', True) try: rdd.count() diff --git a/python/pyspark/util.py b/python/pyspark/util.py index bc3faf0e7d..af32ba1ab0 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -276,13 +276,13 @@ def inheritable_thread_target(f): When the pinned thread mode is off, it return the original ``f``. + .. versionadded:: 3.2.0 + Parameters ---------- f : function the original thread target. - .. versionadded:: 3.2.0 - Notes ----- This API is experimental. diff --git a/python/pyspark/util.pyi b/python/pyspark/util.pyi new file mode 100644 index 0000000000..06940bd7ac --- /dev/null +++ b/python/pyspark/util.pyi @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import threading +from typing import Callable + +class VersionUtils(object): + @staticmethod + def majorMinorVersion(sparkVersion: str): ... + +def inheritable_thread_target(f: Callable) -> Callable: ... + +class InheritableThread(threading.Thread): + pass diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 9d3b619198..023a6553c8 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -45,7 +45,7 @@ from pyspark.serializers import write_with_length, write_int, read_long, read_bo from pyspark.sql.pandas.serializers import ArrowStreamPandasUDFSerializer, CogroupUDFSerializer from pyspark.sql.pandas.types import to_arrow_type from pyspark.sql.types import StructType -from pyspark.util import fail_on_stopiteration, try_simplify_traceback +from pyspark.util import fail_on_stopiteration, try_simplify_traceback # type: ignore from pyspark import shuffle pickleSer = PickleSerializer()