You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/03/27 12:29:10 UTC

[GitHub] [spark] HyukjinKwon opened a new pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

HyukjinKwon opened a new pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052
 
 
   ### What changes were proposed in this pull request?
   
   This PR proposes to make pandas function APIs (`groupby.(cogroup.)applyInPandas` and `mapInPandas`) to ignore Python type hints.
   
   ### Why are the changes needed?
   
   Python type hints are optional. It shouldn't affect where pandas UDFs are not used.
   This is also a future work for them to support other type hints. We shouldn't at least throw an exception at this moment.
   
   ### Does this PR introduce any user-facing change?
   
   No, it's master-only change.
   
   ```python
   import pandas as pd
   
   def pandas_plus_one(pdf: pd.DataFrame) -> pd.DataFrame:
       return pdf + 1
   
   spark.range(10).groupby('id').applyInPandas(pandas_plus_one, schema="id long").show()
   ```
   ```python
   import pandas as pd
   from pyspark.sql.functions import pandas_udf, PandasUDFType
   
   def pandas_plus_one(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame:
       return left + 1
   
   spark.range(10).groupby('id').cogroup(spark.range(10).groupby("id")).applyInPandas(pandas_plus_one, schema="id long").show()
   ```
   
   ```python
   from typing import Iterator
   import pandas as pd
   
   def pandas_plus_one(iter: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]:
       return map(lambda v: v + 1, iter)
   
   spark.range(10).mapInPandas(pandas_plus_one, schema="id long").show()
   ```
   
   
   **Before:**
   
   Exception
   
   **After:**
   
   ```
   +---+
   | id|
   +---+
   |  1|
   |  2|
   |  3|
   |  4|
   |  5|
   |  6|
   |  7|
   |  8|
   |  9|
   | 10|
   +---+
   ```
   
   ### How was this patch tested?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605040832
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977465
 
 
   **[Test build #120489 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120489/testReport)** for PR 28052 at commit [`6b702f4`](https://github.com/apache/spark/commit/6b702f4f90c13cdb5f9cce6e62704326f52510b3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605021097
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25200/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977465
 
 
   **[Test build #120489 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120489/testReport)** for PR 28052 at commit [`6b702f4`](https://github.com/apache/spark/commit/6b702f4f90c13cdb5f9cce6e62704326f52510b3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604993233
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605040841
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120494/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605021097
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25200/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977917
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605559338
 
 
   Thanks, @viirya. Let me merge this.
   
   Merged to master and branch-3.0.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605040832
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605020545
 
 
   **[Test build #120494 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120494/testReport)** for PR 28052 at commit [`371b8eb`](https://github.com/apache/spark/commit/371b8ebfc0d354539a96e3c0f4f6a776dbb212ce).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#discussion_r399745568
 
 

 ##########
 File path: python/pyspark/sql/pandas/functions.py
 ##########
 @@ -384,6 +384,14 @@ def _create_pandas_udf(f, returnType, evalType):
                 "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for "
                 "pandas UDF instead of specifying pandas UDF type which will be deprecated "
                 "in the future releases. See SPARK-28264 for more details.", UserWarning)
+        elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
+                          PythonEvalType.SQL_MAP_PANDAS_ITER_UDF,
+                          PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]:
+            # In case of 'SQL_GROUPED_MAP_PANDAS_UDF',  deprecation warning is being triggered
+            # at `apply` instead.
+            # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the
+            # evaluation type will always be set.
+            pass
         elif len(argspec.annotations) > 0:
 
 Review comment:
   Let me don't add the warning for now .. I am not yet sure if we should add the support of type hints in these pandas Function APIs .. Hopefully the documentation in the site could clarify that type hints are only supported in pandas UDFs.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605021082
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605020545
 
 
   **[Test build #120494 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120494/testReport)** for PR 28052 at commit [`371b8eb`](https://github.com/apache/spark/commit/371b8ebfc0d354539a96e3c0f4f6a776dbb212ce).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977927
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25194/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604992632
 
 
   **[Test build #120489 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120489/testReport)** for PR 28052 at commit [`6b702f4`](https://github.com/apache/spark/commit/6b702f4f90c13cdb5f9cce6e62704326f52510b3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#discussion_r399745691
 
 

 ##########
 File path: python/pyspark/sql/pandas/functions.py
 ##########
 @@ -384,6 +384,14 @@ def _create_pandas_udf(f, returnType, evalType):
                 "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for "
                 "pandas UDF instead of specifying pandas UDF type which will be deprecated "
                 "in the future releases. See SPARK-28264 for more details.", UserWarning)
+        elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
+                          PythonEvalType.SQL_MAP_PANDAS_ITER_UDF,
+                          PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]:
+            # In case of 'SQL_GROUPED_MAP_PANDAS_UDF',  deprecation warning is being triggered
+            # at `apply` instead.
+            # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the
+            # evaluation type will always be set.
+            pass
         elif len(argspec.annotations) > 0:
 
 Review comment:
   It implies a good point - there's a mismatch about how pandas UDF uses Python type hints because we force to set the Python type hints but the type hints are supposed to be completely optional ...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604993246
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120489/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604993233
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605040115
 
 
   **[Test build #120494 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120494/testReport)** for PR 28052 at commit [`371b8eb`](https://github.com/apache/spark/commit/371b8ebfc0d354539a96e3c0f4f6a776dbb212ce).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604993246
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120489/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon closed pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
HyukjinKwon closed pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605040841
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120494/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977917
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#discussion_r399414499
 
 

 ##########
 File path: python/pyspark/sql/pandas/functions.py
 ##########
 @@ -384,6 +384,14 @@ def _create_pandas_udf(f, returnType, evalType):
                 "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for "
                 "pandas UDF instead of specifying pandas UDF type which will be deprecated "
                 "in the future releases. See SPARK-28264 for more details.", UserWarning)
+        elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
+                          PythonEvalType.SQL_MAP_PANDAS_ITER_UDF,
+                          PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]:
+            # In case of 'SQL_GROUPED_MAP_PANDAS_UDF',  deprecation warning is being triggered
+            # at `apply` instead.
+            # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the
+            # evaluation type will always be set.
+            pass
         elif len(argspec.annotations) > 0:
 
 Review comment:
   Shall we show warn messaging in these cases that type hints are not supported yet?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-605021082
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28052: [SPARK-31287][PYTHON][SQL] Ignore type hints in groupby.(cogroup.)applyInPandas and mapInPandas
URL: https://github.com/apache/spark/pull/28052#issuecomment-604977927
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25194/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org