ericm-db commented on code in PR #47133:
URL: https://github.com/apache/spark/pull/47133#discussion_r1679749407


##########
python/pyspark/sql/pandas/serializers.py:
##########
@@ -1116,3 +1121,88 @@ def init_stream_yield_batches(batches):
         batches_to_write = init_stream_yield_batches(serialize_batches())
 
         return ArrowStreamSerializer.dump_stream(self, batches_to_write, 
stream)
+
+
+class TransformWithStateInPandasSerializer(ArrowStreamPandasUDFSerializer):
+    """
+    Serializer used by Python worker to evaluate UDF for 
transformWithStateInPandasSerializer.
+
+    Parameters
+    ----------
+    timezone : str
+        A timezone to respect when handling timestamp values
+    safecheck : bool
+        If True, conversion from Arrow to Pandas checks for overflow/truncation
+    assign_cols_by_name : bool
+        If True, then Pandas DataFrames will get columns by name
+    arrow_max_records_per_batch : int
+        Limit of the number of records that can be written to a single 
ArrowRecordBatch in memory.
+    """
+
+    def __init__(
+            self,
+            timezone,
+            safecheck,
+            assign_cols_by_name,
+            arrow_max_records_per_batch):
+        super(
+            TransformWithStateInPandasSerializer,
+            self
+        ).__init__(timezone, safecheck, assign_cols_by_name)
+
+        # self.state_server_port = state_server_port
+
+        # # open client connection to state server socket
+        self.arrow_max_records_per_batch = arrow_max_records_per_batch
+        self.key_offsets = None
+
+    # Nothing special here, we need to create the handle and read
+    # data in groups.
+    def load_stream(self, stream):
+        """
+        Read ArrowRecordBatches from stream, deserialize them to populate a 
list of pair
+        (data chunk, state), and convert the data into a list of pandas.Series.
+
+        Please refer the doc of inner function `gen_data_and_state` for more 
details how
+        this function works in overall.
+
+        In addition, this function further groups the return of 
`gen_data_and_state` by the state
+        instance (same semantic as grouping by grouping key) and produces an 
iterator of data
+        chunks for each group, so that the caller can lazily materialize the 
data chunk.
+        """
+        import pyarrow as pa
+        from itertools import tee
+
+        def generate_data_batches(batches):
+            for batch in batches:
+                data_pandas = [self.arrow_to_pandas(c) for c in 
pa.Table.from_batches([batch]).itercolumns()]

Review Comment:
   Not sure if this is a common pattern in Python, but this line is a little 
hard to read



##########
python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py:
##########
@@ -0,0 +1,152 @@
+#
+# 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.
+#
+
+import random
+import shutil
+import string
+import sys
+import tempfile
+import pandas as pd
+from pyspark.sql.streaming import StatefulProcessor, StatefulProcessorHandle
+from typing import Iterator
+
+import unittest
+from typing import cast
+
+from pyspark import SparkConf
+from pyspark.sql.streaming.state import GroupStateTimeout, GroupState
+from pyspark.sql.types import (
+    LongType,
+    StringType,
+    StructType,
+    StructField,
+    Row,
+)
+from pyspark.testing.sqlutils import (
+    ReusedSQLTestCase,
+    have_pandas,
+    have_pyarrow,
+    pandas_requirement_message,
+    pyarrow_requirement_message,
+)
+from pyspark.testing.utils import eventually
+
+
+@unittest.skipIf(
+    not have_pandas or not have_pyarrow,
+    cast(str, pandas_requirement_message or pyarrow_requirement_message),
+)
+class TransformWithStateInPandasTestsMixin:
+    @classmethod
+    def conf(cls):
+        cfg = SparkConf()
+        cfg.set("spark.sql.shuffle.partitions", "5")
+        cfg.set("spark.sql.streaming.stateStore.providerClass",
+                
"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider")
+        return cfg
+
+    def _test_apply_in_pandas_with_state_basic(self, func, check_results):
+        input_path = tempfile.mkdtemp()
+
+        def prepare_test_resource():
+            with open(input_path + "/text-test.txt", "w") as fw:
+                fw.write("hello\n")
+                fw.write("this\n")
+
+        prepare_test_resource()
+
+        df = self.spark.readStream.format("text").load(input_path)
+
+        for q in self.spark.streams.active:
+            q.stop()
+        self.assertTrue(df.isStreaming)
+
+        output_type = StructType(
+            [StructField("key", StringType()), StructField("countAsString", 
StringType())]
+        )
+        state_type = StructType([StructField("c", LongType())])
+
+        q = (
+            df.groupBy(df["value"])
+            .transformWithStateInPandas(stateful_processor = 
SimpleStatefulProcessor(),
+                                        outputStructType=output_type,
+                                        outputMode="Update",
+                                        timeMode="None")
+            .writeStream.queryName("this_query")
+            .foreachBatch(check_results)
+            .outputMode("update")
+            .start()
+        )
+
+        self.assertEqual(q.name, "this_query")
+        self.assertTrue(q.isActive)
+        q.processAllAvailable()

Review Comment:
   Should we include `q.awaitTermination()`?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to