[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 <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
This commit is contained in:
HyukjinKwon 2021-06-18 12:02:29 +09:00 committed by Hyukjin Kwon
parent 4373b67a6b
commit 41af409b7b
14 changed files with 60 additions and 38 deletions

View file

@ -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)

View file

@ -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.

View file

@ -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``.

View file

@ -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

View file

@ -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__",
]

View file

@ -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

View file

@ -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)

View file

@ -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,

View file

@ -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"]

View file

@ -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:

View file

@ -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()

View file

@ -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.

29
python/pyspark/util.pyi Normal file
View file

@ -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

View file

@ -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()