Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion .github/workflows/build_python_connect.yml
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,9 @@ jobs:
cp conf/log4j2.properties.template conf/log4j2.properties
sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties
# Start a Spark Connect server
./sbin/start-connect-server.sh --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" --jars `find connector/connect/server/target -name spark-connect*SNAPSHOT.jar`
PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" --jars `find connector/connect/server/target -name spark-connect*SNAPSHOT.jar`
# Make sure running Python workers that contains pyspark.core once. They will be reused.
python -c "from pyspark.sql import SparkSession; _ = SparkSession.builder.remote('sc://localhost').getOrCreate().range(100).repartition(100).mapInPandas(lambda x: x, 'id INT').collect()"
# Remove Py4J and PySpark zipped library to make sure there is no JVM connection
rm python/lib/*
rm -r python/pyspark
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,10 @@
import os
import unittest

from pyspark.util import is_remote_only
from pyspark.tests.test_memory_profiler import MemoryProfiler2TestsMixin, _do_computation
from pyspark.testing.connectutils import ReusedConnectTestCase


# TODO(SPARK-47830): Reeanble MemoryProfilerParityTests for pyspark-connect
@unittest.skipIf(is_remote_only(), "Skipped for now")
class MemoryProfilerParityTests(MemoryProfiler2TestsMixin, ReusedConnectTestCase):
def setUp(self) -> None:
super().setUp()
Expand Down
3 changes: 0 additions & 3 deletions python/pyspark/sql/tests/connect/test_parity_udf_profiler.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,10 @@
import os
import unittest

from pyspark.util import is_remote_only
from pyspark.sql.tests.test_udf_profiler import UDFProfiler2TestsMixin, _do_computation
from pyspark.testing.connectutils import ReusedConnectTestCase


# TODO(SPARK-47756): Reeanble UDFProfilerParityTests for pyspark-connect
@unittest.skipIf(is_remote_only(), "Skipped for now")
class UDFProfilerParityTests(UDFProfiler2TestsMixin, ReusedConnectTestCase):
def setUp(self) -> None:
super().setUp()
Expand Down
12 changes: 4 additions & 8 deletions python/pyspark/worker_util.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
except ImportError:
has_resource_module = False

from pyspark.accumulators import _accumulatorRegistry
from pyspark.util import is_remote_only
from pyspark.errors import PySparkRuntimeError
from pyspark.util import local_connect_and_auth
Expand Down Expand Up @@ -183,11 +184,6 @@ def send_accumulator_updates(outfile: IO) -> None:
"""
Send the accumulator updates back to JVM.
"""
if not is_remote_only():
from pyspark.accumulators import _accumulatorRegistry

write_int(len(_accumulatorRegistry), outfile)
for aid, accum in _accumulatorRegistry.items():
pickleSer._write_with_length((aid, accum._value), outfile)
else:
write_int(0, outfile)
write_int(len(_accumulatorRegistry), outfile)
for aid, accum in _accumulatorRegistry.items():
pickleSer._write_with_length((aid, accum._value), outfile)