[ https://issues.apache.org/jira/browse/SPARK-28502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16894238#comment-16894238 ]
Nasir Ali commented on SPARK-28502: ----------------------------------- I tried to set timzone to UTC as suggested by [~icexelloss] but it didn't solve the problem. It is throwing following error (same error but with tz=UTC). {code:java} TypeError: Unsupported type in conversion from Arrow: struct<start: timestamp[us, tz=UTC], end: timestamp[us, tz=UTC]> {code} I think, this is a conversion error, maybe? Below is the complete trace: {code:java} df.groupby('id', F.window("ts", "15 days")).apply(some_udf).show() File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/sql/dataframe.py", line 378, in show File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 1257, in __call__ File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in deco File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 328, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o207.showString. : org.apache.spark.SparkException: Job aborted due to stage failure: Task 13 in stage 5.0 failed 1 times, most recent failure: Lost task 13.0 in stage 5.0 (TID 32, localhost, executor driver): org.apache.spark.api.python.PythonException: Traceback (most recent call last): File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/worker.py", line 372, in main process() File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/worker.py", line 367, in process serializer.dump_stream(func(split_index, iterator), outfile) File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/serializers.py", line 283, in dump_stream for series in iterator: File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/serializers.py", line 301, in load_stream yield [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch]).itercolumns()] File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/serializers.py", line 301, in <listcomp> yield [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch]).itercolumns()] File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/serializers.py", line 271, in arrow_to_pandas s = _check_series_convert_date(s, from_arrow_type(arrow_column.type)) File "/home/ali/spark/spark-2.4.0-bin-hadoop2.7/python/lib/pyspark.zip/pyspark/sql/types.py", line 1672, in from_arrow_type raise TypeError("Unsupported type in conversion from Arrow: " + str(at)) TypeError: Unsupported type in conversion from Arrow: struct<start: timestamp[us, tz=UTC], end: timestamp[us, tz=UTC]> {code} > Error with struct conversion while using pandas_udf > --------------------------------------------------- > > Key: SPARK-28502 > URL: https://issues.apache.org/jira/browse/SPARK-28502 > Project: Spark > Issue Type: Bug > Components: PySpark > Affects Versions: 2.4.3 > Environment: OS: Ubuntu > Python: 3.6 > Reporter: Nasir Ali > Priority: Minor > > What I am trying to do: Group data based on time intervals (e.g., 15 days > window) and perform some operations on dataframe using (pandas) UDFs. I don't > know if there is a better/cleaner way to do it. > Below is the sample code that I tried and error message I am getting. > > {code:java} > df = sparkSession.createDataFrame([(17.00, "2018-03-10T15:27:18+00:00"), > (13.00, "2018-03-11T12:27:18+00:00"), > (25.00, "2018-03-12T11:27:18+00:00"), > (20.00, "2018-03-13T15:27:18+00:00"), > (17.00, "2018-03-14T12:27:18+00:00"), > (99.00, "2018-03-15T11:27:18+00:00"), > (156.00, "2018-03-22T11:27:18+00:00"), > (17.00, "2018-03-31T11:27:18+00:00"), > (25.00, "2018-03-15T11:27:18+00:00"), > (25.00, "2018-03-16T11:27:18+00:00") > ], > ["id", "ts"]) > df = df.withColumn('ts', df.ts.cast('timestamp')) > schema = StructType([ > StructField("id", IntegerType()), > StructField("ts", TimestampType()) > ]) > @pandas_udf(schema, PandasUDFType.GROUPED_MAP) > def some_udf(df): > # some computation > return df > df.groupby('id', F.window("ts", "15 days")).apply(some_udf).show() > {code} > This throws following exception: > {code:java} > TypeError: Unsupported type in conversion from Arrow: struct<start: > timestamp[us, tz=America/Chicago], end: timestamp[us, tz=America/Chicago]> > {code} > > However, if I use builtin agg method then it works all fine. For example, > {code:java} > df.groupby('id', F.window("ts", "15 days")).mean().show(truncate=False) > {code} > Output > {code:java} > +-----+------------------------------------------+-------+ > |id |window |avg(id)| > +-----+------------------------------------------+-------+ > |13.0 |[2018-03-05 00:00:00, 2018-03-20 00:00:00]|13.0 | > |17.0 |[2018-03-20 00:00:00, 2018-04-04 00:00:00]|17.0 | > |156.0|[2018-03-20 00:00:00, 2018-04-04 00:00:00]|156.0 | > |99.0 |[2018-03-05 00:00:00, 2018-03-20 00:00:00]|99.0 | > |20.0 |[2018-03-05 00:00:00, 2018-03-20 00:00:00]|20.0 | > |17.0 |[2018-03-05 00:00:00, 2018-03-20 00:00:00]|17.0 | > |25.0 |[2018-03-05 00:00:00, 2018-03-20 00:00:00]|25.0 | > +-----+------------------------------------------+-------+ > {code} -- This message was sent by Atlassian JIRA (v7.6.14#76016) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org