This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 00cb2f9  [SPARK-28881][PYTHON][TESTS] Add a test to make sure toPandas 
with Arrow optimization throws an exception per maxResultSize
00cb2f9 is described below

commit 00cb2f99ccbd7c0fdba19ba63c4ec73ca97dea66
Author: HyukjinKwon <gurwls...@apache.org>
AuthorDate: Tue Aug 27 17:30:06 2019 +0900

    [SPARK-28881][PYTHON][TESTS] Add a test to make sure toPandas with Arrow 
optimization throws an exception per maxResultSize
    
    ### What changes were proposed in this pull request?
    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 
https://github.com/apache/spark/pull/25593#issuecomment-525153808). This 
regression was found between Spark 2.3 and Spark 2.4, and accidentally fixed.
    
    ### Why are the changes needed?
    To prevent the same regression in the future.
    
    ### Does this PR introduce any user-facing change?
    No.
    
    ### How was this patch tested?
    Test was added.
    
    Closes #25594 from HyukjinKwon/SPARK-28881.
    
    Authored-by: HyukjinKwon <gurwls...@apache.org>
    Signed-off-by: HyukjinKwon <gurwls...@apache.org>
---
 python/pyspark/sql/tests/test_arrow.py | 31 ++++++++++++++++++++++++++++++-
 1 file changed, 30 insertions(+), 1 deletion(-)

diff --git a/python/pyspark/sql/tests/test_arrow.py 
b/python/pyspark/sql/tests/test_arrow.py
index f533083..50c82b0 100644
--- a/python/pyspark/sql/tests/test_arrow.py
+++ b/python/pyspark/sql/tests/test_arrow.py
@@ -22,7 +22,7 @@ import time
 import unittest
 import warnings
 
-from pyspark.sql import Row
+from pyspark.sql import Row, SparkSession
 from pyspark.sql.functions import udf
 from pyspark.sql.types import *
 from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, 
have_pyarrow, \
@@ -421,6 +421,35 @@ class ArrowTests(ReusedSQLTestCase):
             run_test(*case)
 
 
+@unittest.skipIf(
+    not have_pandas or not have_pyarrow,
+    pandas_requirement_message or pyarrow_requirement_message)
+class MaxResultArrowTests(unittest.TestCase):
+    # These tests are separate as 'spark.driver.maxResultSize' configuration
+    # is a static configuration to Spark context.
+
+    @classmethod
+    def setUpClass(cls):
+        cls.spark = SparkSession.builder \
+            .master("local[4]") \
+            .appName(cls.__name__) \
+            .config("spark.driver.maxResultSize", "10k") \
+            .getOrCreate()
+
+        # Explicitly enable Arrow and disable fallback.
+        cls.spark.conf.set("spark.sql.execution.arrow.pyspark.enabled", "true")
+        
cls.spark.conf.set("spark.sql.execution.arrow.pyspark.fallback.enabled", 
"false")
+
+    @classmethod
+    def tearDownClass(cls):
+        if hasattr(cls, "spark"):
+            cls.spark.stop()
+
+    def test_exception_by_max_results(self):
+        with self.assertRaisesRegexp(Exception, "is bigger than"):
+            self.spark.range(0, 10000, 1, 100).toPandas()
+
+
 class EncryptionArrowTests(ArrowTests):
 
     @classmethod


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to