Skip to content
Closed
Show file tree
Hide file tree
Changes from 13 commits
Commits
Show all changes
55 commits
Select commit Hold shift + click to select a range
b1175e4
[WIP] Poc.
sahnib Jan 22, 2024
0a98ed8
Introduce Protobuf.
sahnib Feb 6, 2024
8e2b193
Fixing things.
sahnib Feb 6, 2024
16e4c17
support timeMode for python state v2 API
bogao007 Jun 20, 2024
92ef716
Add protobuf for serde
sahnib Feb 13, 2024
c3eaf38
protobuf change
bogao007 Jun 20, 2024
609d94e
Initial commit
bogao007 Jun 27, 2024
a27f9d9
better error handling, support value state with different types
bogao007 Jun 28, 2024
684939b
addressed comments
bogao007 Jul 3, 2024
7f65fbd
fix
bogao007 Jul 3, 2024
c25d7da
Added support for unix domain socket
bogao007 Jul 11, 2024
9c8c616
removed unrelated log lines, addressed part of the comments
bogao007 Jul 17, 2024
c641192
fix
bogao007 Jul 17, 2024
8d3da4e
Addressed comments
bogao007 Jul 19, 2024
cc9bf95
removed unnecessary print
bogao007 Jul 19, 2024
f7df2dc
rename
bogao007 Jul 19, 2024
27cd169
fix
bogao007 Jul 19, 2024
3b5b3e5
removed duplicate proto file
bogao007 Jul 20, 2024
5d910d8
revert unrelated changes
bogao007 Jul 20, 2024
df859ab
fix
bogao007 Jul 20, 2024
654f2f6
Added unit tests for transformWithStateInPandas
bogao007 Jul 24, 2024
38832a6
Merge branch 'master' into state-v2-initial
bogao007 Jul 24, 2024
0585ac0
fix and rename
bogao007 Jul 24, 2024
0ee5029
update test
bogao007 Jul 24, 2024
6232c81
Added lisences
bogao007 Jul 25, 2024
41f8234
fixed format issues
bogao007 Jul 25, 2024
d57633f
fix
bogao007 Jul 25, 2024
df9ea9e
fix format
bogao007 Jul 25, 2024
68f7a7e
doc
bogao007 Jul 25, 2024
ca5216b
addressed comments
bogao007 Jul 26, 2024
c9e3a7c
structured log
bogao007 Jul 26, 2024
2320805
suppress auto generated proto file
bogao007 Jul 29, 2024
6e5de2e
fix linter
bogao007 Jul 29, 2024
200ec5e
fixed dependency issue
bogao007 Jul 29, 2024
dd3e46b
make protobuf as local dependency
bogao007 Jul 30, 2024
e8360d4
fix dependency issue
bogao007 Jul 30, 2024
82983af
fix
bogao007 Jul 30, 2024
49dbc16
fix lint
bogao007 Jul 30, 2024
d4e04ea
fix
bogao007 Jul 30, 2024
e108f60
updated fix
bogao007 Jul 30, 2024
bae26c2
reformat
bogao007 Jul 30, 2024
d96fa9e
addressed comments
bogao007 Jul 31, 2024
92531db
fix linter
bogao007 Jul 31, 2024
d507793
linter
bogao007 Jul 31, 2024
5dcb4c8
addressed comments
bogao007 Aug 2, 2024
37be02a
address comment
bogao007 Aug 2, 2024
f63687f
addressed comments
bogao007 Aug 9, 2024
263c087
Merge branch 'master' into state-v2-initial
bogao007 Aug 10, 2024
c7b0a4f
address comments
bogao007 Aug 12, 2024
c80b292
address comments
bogao007 Aug 12, 2024
81276f3
address comments
bogao007 Aug 14, 2024
5886b5c
fix lint
bogao007 Aug 14, 2024
23e54b4
fix lint
bogao007 Aug 14, 2024
2ba4fd0
address comments
bogao007 Aug 14, 2024
2a9c20b
fix test
bogao007 Aug 14, 2024
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -3698,6 +3698,12 @@
],
"sqlState" : "42802"
},
"STATEFUL_PROCESSOR_UNKNOWN_TIME_MODE" : {
"message" : [
"Unknown time mode <timeMode>. Accepted timeMode modes are 'none', 'processingTime', 'eventTime'"
],
"sqlState" : "42802"
},
"STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS" : {
"message" : [
"Failed to create column family with unsupported starting character and name=<colFamilyName>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ private[spark] object PythonEvalType {
val SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE = 208
val SQL_GROUPED_MAP_ARROW_UDF = 209
val SQL_COGROUPED_MAP_ARROW_UDF = 210
val SQL_TRANSFORM_WITH_STATE_PANDAS_UDF = 211

val SQL_TABLE_UDF = 300
val SQL_ARROW_TABLE_UDF = 301
Expand All @@ -82,6 +83,7 @@ private[spark] object PythonEvalType {
case SQL_COGROUPED_MAP_ARROW_UDF => "SQL_COGROUPED_MAP_ARROW_UDF"
case SQL_TABLE_UDF => "SQL_TABLE_UDF"
case SQL_ARROW_TABLE_UDF => "SQL_ARROW_TABLE_UDF"
case SQL_TRANSFORM_WITH_STATE_PANDAS_UDF => "SQL_TRANSFORM_WITH_STATE_PANDAS_UDF"
}
}

Expand Down
1 change: 1 addition & 0 deletions python/pyspark/sql/pandas/_typing/__init__.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ ArrowMapIterUDFType = Literal[207]
PandasGroupedMapUDFWithStateType = Literal[208]
ArrowGroupedMapUDFType = Literal[209]
ArrowCogroupedMapUDFType = Literal[210]
PandasGroupedMapUDFTransformWithStateType = Literal[211]

class PandasVariadicScalarToScalarFunction(Protocol):
def __call__(self, *_: DataFrameOrSeriesLike_) -> DataFrameOrSeriesLike_: ...
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/sql/pandas/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,7 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]:
PythonEvalType.SQL_MAP_ARROW_ITER_UDF,
PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF,
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE,
PythonEvalType.SQL_TRANSFORM_WITH_STATE,
PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF,
PythonEvalType.SQL_COGROUPED_MAP_ARROW_UDF,
None,
Expand Down Expand Up @@ -453,6 +454,7 @@ def _validate_pandas_udf(f, evalType) -> int:
PythonEvalType.SQL_MAP_ARROW_ITER_UDF,
PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF,
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE,
PythonEvalType.SQL_TRANSFORM_WITH_STATE,
PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF,
PythonEvalType.SQL_COGROUPED_MAP_ARROW_UDF,
PythonEvalType.SQL_ARROW_BATCHED_UDF,
Expand Down
120 changes: 119 additions & 1 deletion python/pyspark/sql/pandas/group_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,17 @@
# limitations under the License.
#
import sys
from typing import List, Union, TYPE_CHECKING, cast
from typing import Any, Iterator, List, Union, TYPE_CHECKING, cast
import warnings

from pyspark.errors import PySparkTypeError
from pyspark.util import PythonEvalType
from pyspark.sql.column import Column
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.functions.builtin import udf
from pyspark.sql.streaming.state import GroupStateTimeout
from pyspark.sql.streaming.state_api_client import StateApiClient, StatefulProcessorHandleState
from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle
from pyspark.sql.types import StructType, _parse_datatype_string

if TYPE_CHECKING:
Expand All @@ -33,6 +36,7 @@
PandasCogroupedMapFunction,
ArrowGroupedMapFunction,
ArrowCogroupedMapFunction,
DataFrameLike as PandasDataFrameLike
)
from pyspark.sql.group import GroupedData

Expand Down Expand Up @@ -358,6 +362,120 @@ def applyInPandasWithState(
)
return DataFrame(jdf, self.session)


def transformWithStateInPandas(self,
stateful_processor: StatefulProcessor,
outputStructType: Union[StructType, str],
outputMode: str,
timeMode: str) -> DataFrame:
"""
Invokes methods defined in the stateful processor used in arbitrary state API v2.
We allow the user to act on per-group set of input rows along with keyed state and the
user can choose to output/return 0 or more rows.

For a streaming dataframe, we will repeatedly invoke the interface methods for new rows
in each trigger and the user's state/state variables will be stored persistently across
invocations.

The `stateful_processor` should be a Python class that implements the interface defined in
pyspark.sql.streaming.stateful_processor.StatefulProcessor.

The `outputStructType` should be a :class:`StructType` describing the schema of all
elements in the returned value, `pandas.DataFrame`. The column labels of all elements in
returned `pandas.DataFrame` must either match the field names in the defined schema if
specified as strings, or match the field data types by position if not strings,
e.g. integer indices.

The size of each `pandas.DataFrame` in both the input and output can be arbitrary. The
number of `pandas.DataFrame` in both the input and output can also be arbitrary.

.. versionadded:: 4.0.0

Parameters
----------
stateful_processor : :class:`pyspark.sql.streaming.stateful_processor.StatefulProcessor`
Instance of StatefulProcessor whose functions will be invoked by the operator.
outputStructType : :class:`pyspark.sql.types.DataType` or str
The type of the output records. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
outputMode : str
The output mode of the stateful processor.
timeMode : str
The time mode semantics of the stateful processor for timers and TTL.

Examples
--------
>>> import pandas as pd
>>> from pyspark.sql.streaming import StatefulProcessor, StatefulProcessorHandle
>>> from pyspark.sql.types import StructType, StructField, LongType, StringType
>>> from typing import Iterator
>>> output_schema = StructType([
... StructField("value", LongType(), True)
... ])
>>> state_schema = StructType([
... StructField("value", StringType(), True)
... ])
>>> class SimpleStatefulProcessor(StatefulProcessor):
... def init(self, handle: StatefulProcessorHandle) -> None:
... self.value_state = handle.getValueState("testValueState", state_schema)
... def handleInputRows(self, key, rows) -> Iterator[pd.DataFrame]:
... self.value_state.update("test_value")
... exists = self.value_state.exists()
... value = self.value_state.get()
... self.value_state.clear()
... return rows
... def close(self) -> None:
... pass
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[nit] It might be more useful to provide a running count example, where we store values above a specified threshold in the state (to keep track of violations). [something like processing temperature sensor values in a stream]

...
>>> df.groupBy("value").transformWithStateInPandas(stateful_processor =
... SimpleStatefulProcessor(), outputStructType=output_schema, outputMode="Update",
... timeMode="None") # doctest: +SKIP

Notes
-----
This function requires a full shuffle.

This API is experimental.
"""

from pyspark.sql import GroupedData
from pyspark.sql.functions import pandas_udf
assert isinstance(self, GroupedData)

def transformWithStateUDF(state_api_client: StateApiClient, key: Any,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: maybe move each arg to new line ?

inputRows: Iterator["PandasDataFrameLike"]) -> Iterator["PandasDataFrameLike"]:
handle = StatefulProcessorHandle(state_api_client)

if (state_api_client.handle_state == StatefulProcessorHandleState.CREATED):
stateful_processor.init(handle)
state_api_client.set_handle_state(StatefulProcessorHandleState.INITIALIZED)

state_api_client.set_implicit_key(str(key[0]))
result = stateful_processor.handleInputRows(key, inputRows)
state_api_client.remove_implicit_key()

return result

if isinstance(outputStructType, str):
outputStructType = cast(StructType, _parse_datatype_string(outputStructType))

udf = pandas_udf(
transformWithStateUDF, # type: ignore[call-overload]
returnType=outputStructType,
functionType=PythonEvalType.SQL_TRANSFORM_WITH_STATE,
)
df = self._df
udf_column = udf(*[df[col] for col in df.columns])

jdf = self._jgd.transformWithStateInPandas(
udf_column._jc.expr(),
self.session._jsparkSession.parseDataType(outputStructType.json()),
outputMode,
timeMode,
)
return DataFrame(jdf, self.session)


def applyInArrow(
self, func: "ArrowGroupedMapFunction", schema: Union[StructType, str]
) -> "DataFrame":
Expand Down
75 changes: 74 additions & 1 deletion python/pyspark/sql/pandas/serializers.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,14 @@
Serializers for PyArrow and pandas conversions. See `pyspark.serializers` for more details.
"""

from enum import Enum
from itertools import groupby
import os
import socket
from typing import Any
from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError
from pyspark.loose_version import LooseVersion
from pyspark.serializers import Serializer, read_int, write_int, UTF8Deserializer, CPickleSerializer
from pyspark.serializers import Serializer, read_int, write_int, UTF8Deserializer, CPickleSerializer, write_with_length
from pyspark.sql.pandas.types import (
from_arrow_type,
to_arrow_type,
Expand Down Expand Up @@ -1116,3 +1121,71 @@ 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.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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems like this documentation is referring to the ApplyInPandasWithState serializer which transfers both state and data.

"""
import pyarrow as pa

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()]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

key_series = [data_pandas[o] for o in self.key_offsets]
batch_key = tuple(s[0] for s in key_series)
yield (batch_key, data_pandas)

_batches = super(ArrowStreamPandasSerializer, self).load_stream(stream)
data_batches = generate_data_batches(_batches)

for k, g in groupby(data_batches, key=lambda x: x[0]):
yield (k, g)


def dump_stream(self, iterator, stream):
"""
Read through an iterator of (iterator of pandas DataFrame, state), serialize them to Arrow
RecordBatches, and write batches to stream.
"""
result = [(b, t) for x in iterator for y, t in x for b in y]
super().dump_stream(result, stream)
86 changes: 86 additions & 0 deletions python/pyspark/sql/streaming/StateMessage.proto
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
syntax = "proto3";

package pyspark.sql.streaming;

message StateRequest {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it possible to add some high level comments here or in some other Python file ?

int32 version = 1;
oneof method {
StatefulProcessorCall statefulProcessorCall = 2;
StateVariableRequest stateVariableRequest = 3;
ImplicitGroupingKeyRequest implicitGroupingKeyRequest = 4;
}
}

message StateResponse {
int32 statusCode = 1;
string errorMessage = 2;
}

message StatefulProcessorCall {
oneof method {
SetHandleState setHandleState = 1;
StateCallCommand getValueState = 2;
StateCallCommand getListState = 3;
StateCallCommand getMapState = 4;
}
}

message StateVariableRequest {
oneof method {
ValueStateCall valueStateCall = 1;
}
}

message ImplicitGroupingKeyRequest {
oneof method {
SetImplicitKey setImplicitKey = 1;
RemoveImplicitKey removeImplicitKey = 2;
}
}

message StateCallCommand {
string stateName = 1;
string schema = 2;
}

message ValueStateCall {
string stateName = 1;
oneof method {
Exists exists = 2;
Get get = 3;
ValueStateUpdate valueStateUpdate = 4;
Clear clear = 5;
}
}

message SetImplicitKey {
string key = 1;
}

message RemoveImplicitKey {
}

message Exists {
}

message Get {
}

message ValueStateUpdate {
string schema = 1;
bytes value = 2;
}

message Clear {
}

enum HandleState {
CREATED = 0;
INITIALIZED = 1;
DATA_PROCESSED = 2;
CLOSED = 3;
}

message SetHandleState {
HandleState state = 1;
}
Loading