Skip to content

Commit 6ab47c6

Browse files
committed
[SPARK-28881][PYTHON][TESTS] Add a test to make sure toPandas with Arrow optimization throws an exception per maxResultSize
This PR proposes to add a test case for: ```bash ./bin/pyspark --conf spark.driver.maxResultSize=1m spark.conf.set("spark.sql.execution.arrow.enabled",True) ``` ```python spark.range(10000000).toPandas() ``` ``` Empty DataFrame Columns: [id] Index: [] ``` which can result in partial results (see #25593 (comment)). This regression was found between Spark 2.3 and Spark 2.4, and accidentally fixed. To prevent the same regression in the future. No. Test was added. Closes #25594 from HyukjinKwon/SPARK-28881. Authored-by: HyukjinKwon <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
1 parent 5758b6c commit 6ab47c6

File tree

1 file changed

+29
-0
lines changed

1 file changed

+29
-0
lines changed

python/pyspark/sql/tests.py

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4550,6 +4550,35 @@ def test_timestamp_dst(self):
45504550
self.assertPandasEqual(pdf, df_from_pandas.toPandas())
45514551

45524552

4553+
@unittest.skipIf(
4554+
not _have_pandas or not _have_pyarrow,
4555+
_pandas_requirement_message or _pyarrow_requirement_message)
4556+
class MaxResultArrowTests(unittest.TestCase):
4557+
# These tests are separate as 'spark.driver.maxResultSize' configuration
4558+
# is a static configuration to Spark context.
4559+
4560+
@classmethod
4561+
def setUpClass(cls):
4562+
cls.spark = SparkSession.builder \
4563+
.master("local[4]") \
4564+
.appName(cls.__name__) \
4565+
.config("spark.driver.maxResultSize", "10k") \
4566+
.getOrCreate()
4567+
4568+
# Explicitly enable Arrow and disable fallback.
4569+
cls.spark.conf.set("spark.sql.execution.arrow.enabled", "true")
4570+
cls.spark.conf.set("spark.sql.execution.arrow.fallback.enabled", "false")
4571+
4572+
@classmethod
4573+
def tearDownClass(cls):
4574+
if hasattr(cls, "spark"):
4575+
cls.spark.stop()
4576+
4577+
def test_exception_by_max_results(self):
4578+
with self.assertRaisesRegexp(Exception, "is bigger than"):
4579+
self.spark.range(0, 10000, 1, 100).toPandas()
4580+
4581+
45534582
class EncryptionArrowTests(ArrowTests):
45544583

45554584
@classmethod

0 commit comments

Comments
 (0)