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
1 change: 1 addition & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -1048,6 +1048,7 @@ def __hash__(self):
"pyspark.sql.tests.connect.test_parity_arrow_grouped_map",
"pyspark.sql.tests.connect.test_parity_arrow_cogrouped_map",
"pyspark.sql.tests.connect.test_parity_python_datasource",
"pyspark.sql.tests.connect.test_parity_python_streaming_datasource",
"pyspark.sql.tests.connect.test_utils",
"pyspark.sql.tests.connect.client.test_artifact",
"pyspark.sql.tests.connect.client.test_client",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
from typing import IO, Iterator, Tuple

from pyspark.accumulators import _accumulatorRegistry
from pyspark.java_gateway import local_connect_and_auth
from pyspark.errors import IllegalArgumentException, PySparkAssertionError, PySparkRuntimeError
from pyspark.serializers import (
read_int,
Expand All @@ -37,7 +36,7 @@
StructType,
)
from pyspark.sql.worker.plan_data_source_read import records_to_arrow_batches
from pyspark.util import handle_worker_exception
from pyspark.util import handle_worker_exception, local_connect_and_auth
from pyspark.worker_util import (
check_python_version,
read_command,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
#
# 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.
#
from pyspark.sql.tests.test_python_streaming_datasource import (
BasePythonStreamingDataSourceTestsMixin,
)
from pyspark.testing.connectutils import ReusedConnectTestCase


class PythonStreamingDataSourceParityTests(
BasePythonStreamingDataSourceTestsMixin, ReusedConnectTestCase
):
pass


if __name__ == "__main__":
import unittest
from pyspark.sql.tests.connect.test_parity_python_streaming_datasource import * # noqa: F401

try:
import xmlrunner # type: ignore[import]

testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
except ImportError:
testRunner = None
unittest.main(testRunner=testRunner, verbosity=2)
6 changes: 1 addition & 5 deletions python/pyspark/sql/tests/test_python_streaming_datasource.py
Original file line number Diff line number Diff line change
Expand Up @@ -142,15 +142,11 @@ def test_stream_reader(self):
self.spark.dataSource.register(self._get_test_data_source())
df = self.spark.readStream.format("TestDataSource").load()

current_batch_id = -1

def check_batch(df, batch_id):
nonlocal current_batch_id
current_batch_id = batch_id
assertDataFrameEqual(df, [Row(batch_id * 2), Row(batch_id * 2 + 1)])

q = df.writeStream.foreachBatch(check_batch).start()
while current_batch_id < 10:
while len(q.recentProgress) < 10:
time.sleep(0.2)
q.stop()
q.awaitTermination()
Expand Down