anishshri-db commented on code in PR #47133: URL: https://github.com/apache/spark/pull/47133#discussion_r1680013451
########## python/pyspark/worker.py: ########## @@ -487,6 +489,20 @@ def wrapped(key_series, value_series): return lambda k, v: [(wrapped(k, v), to_arrow_type(return_type))] +def wrap_grouped_transform_with_state_pandas_udf(f, return_type, runner_conf): + _assign_cols_by_name = assign_cols_by_name(runner_conf) + + def wrapped(state_api_client, key, value_series_gen): + import pandas as pd + values = (pd.concat(x, axis=1) for x in value_series_gen) + result_iter = f(state_api_client, key, values) + + # TODO: add verification that elements in result_iter are Review Comment: Do other operators do this too ? ########## python/pyspark/util.py: ########## @@ -585,6 +586,7 @@ class PythonEvalType: SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE: "PandasGroupedMapUDFWithStateType" = 208 SQL_GROUPED_MAP_ARROW_UDF: "ArrowGroupedMapUDFType" = 209 SQL_COGROUPED_MAP_ARROW_UDF: "ArrowCogroupedMapUDFType" = 210 + SQL_TRANSFORM_WITH_STATE: "PandasGroupedMapUDFTransformWithStateType" = 211 Review Comment: nit: `SQL_TRANSFORM_WITH_STATE_UDF` ? ########## python/pyspark/sql/streaming/state_api_client.py: ########## @@ -0,0 +1,142 @@ +# +# 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. +# + +from enum import Enum +import os +import socket +from typing import Any, Union, cast + +import pyspark.sql.streaming.StateMessage_pb2 as stateMessage +from pyspark.serializers import write_int, read_int, UTF8Deserializer +from pyspark.sql.types import StructType, _parse_datatype_string + + +class StatefulProcessorHandleState(Enum): + CREATED = 1 + INITIALIZED = 2 + DATA_PROCESSED = 3 + CLOSED = 4 + + +class StateApiClient: + def __init__( + self, + state_server_id: int) -> None: + self._client_socket = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + server_address = f'./uds_{state_server_id}.sock' + self._client_socket.connect(server_address) + self.sockfile = self._client_socket.makefile("rwb", + int(os.environ.get("SPARK_BUFFER_SIZE",65536))) + print(f"client is ready - connection established") + self.handle_state = StatefulProcessorHandleState.CREATED + self.utf8_deserializer = UTF8Deserializer() + + def set_handle_state(self, state: StatefulProcessorHandleState) -> None: + print(f"setting handle state to: {state}") + proto_state = self._get_proto_state(state) + set_handle_state = stateMessage.SetHandleState(state=proto_state) + handle_call = stateMessage.StatefulProcessorCall(setHandleState=set_handle_state) + message = stateMessage.StateRequest(statefulProcessorCall=handle_call) + + self._send_proto_message(message) + + response_message = self._receive_proto_message() + status = response_message.statusCode + if (status == 0): + self.handle_state = state + else: + raise Exception(f"Error setting handle state: {response_message.errorMessage}") + print(f"setHandleState status= {status}") + + def set_implicit_key(self, key: str) -> None: + print(f"setting implicit key: {key}") + set_implicit_key = stateMessage.SetImplicitKey(key=key) + request = stateMessage.ImplicitGroupingKeyRequest(setImplicitKey=set_implicit_key) + message = stateMessage.StateRequest(implicitGroupingKeyRequest=request) + + self._send_proto_message(message) + response_message = self._receive_proto_message() + status = response_message.statusCode + print(f"setImplicitKey status= {status}") + if (status != 0): + raise Exception(f"Error setting implicit key: {response_message.errorMessage}") + + def remove_implicit_key(self) -> None: + print(f"removing implicit key") + remove_implicit_key = stateMessage.RemoveImplicitKey() + request = stateMessage.ImplicitGroupingKeyRequest(removeImplicitKey=remove_implicit_key) + message = stateMessage.StateRequest(implicitGroupingKeyRequest=request) + + self._send_proto_message(message) + response_message = self._receive_proto_message() + status = response_message.statusCode + print(f"removeImplicitKey status= {status}") + if (status != 0): + raise Exception(f"Error removing implicit key: {response_message.errorMessage}") + + def get_value_state(self, state_name: str, schema: Union[StructType, str]) -> None: + if isinstance(schema, str): + schema = cast(StructType, _parse_datatype_string(schema)) + + print(f"initializing value state: {state_name}") + + state_call_command = stateMessage.StateCallCommand() + state_call_command.stateName = state_name + state_call_command.schema = schema.json() + call = stateMessage.StatefulProcessorCall(getValueState=state_call_command) + message = stateMessage.StateRequest(statefulProcessorCall=call) + + self._send_proto_message(message) + response_message = self._receive_proto_message() + status = response_message.statusCode + if (status != 0): + raise Exception(f"Error initializing value state: {response_message.errorMessage}") + print(f"getValueState status= {status}") + + def _get_proto_state(self, Review Comment: Is double underscore for starting chars the recommended practice here for private methods ? cc - @HyukjinKwon to confirm -- 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