Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
fix
  • Loading branch information
bogao007 committed Jul 19, 2024
commit 27cd169db281025dcff55ebac9770a1442d3cba7
2 changes: 2 additions & 0 deletions python/pyspark/sql/streaming/state_api_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,8 @@ def _get_proto_state(self,

def _send_proto_message(self, message: stateMessage.StateRequest) -> None:
serialized_msg = message.SerializeToString()
# Writing zero here to indicate message version. This allows us to evolve the message
# format or even changing the message protocol in the future.
write_int(0, self.sockfile)
write_int(len(serialized_msg), self.sockfile)
self.sockfile.write(serialized_msg)
Expand Down
4 changes: 2 additions & 2 deletions python/pyspark/util.py
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@
PandasGroupedMapUDFWithStateType,
ArrowGroupedMapUDFType,
ArrowCogroupedMapUDFType,
PandasGroupedMapUDFTransformWithStateType,
PandasTransformWithStateUDFType,
)
from pyspark.sql._typing import (
SQLArrowBatchedUDFType,
Expand Down Expand Up @@ -586,7 +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
SQL_TRANSFORM_WITH_STATE_PANDAS_UDF: "PandasTransformWithStateUDFType" = 211

SQL_TABLE_UDF: "SQLTableUDFType" = 300
SQL_ARROW_TABLE_UDF: "SQLArrowTableUDFType" = 301
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -733,7 +733,6 @@ class RelationalGroupedDataset protected[sql](
* along with corresponding state, and the results for all groups are combined into a new
* [[DataFrame]].
*
*
* This function uses Apache Arrow as serialization format between Java executors and Python
* workers.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,17 +71,17 @@ private[python] trait PythonArrowInput[IN] { self: BasePythonRunner[IN, _] =>
protected val root = VectorSchemaRoot.create(arrowSchema, allocator)
protected var writer: ArrowStreamWriter = _

protected def close(): Unit = {
Utils.tryWithSafeFinally {
// end writes footer to the output stream and doesn't clean any resources.
// It could throw exception if the output stream is closed, so it should be
// in the try block.
writer.end()
} {
root.close()
allocator.close()
}
protected def close(): Unit = {
Utils.tryWithSafeFinally {
// end writes footer to the output stream and doesn't clean any resources.
// It could throw exception if the output stream is closed, so it should be
// in the try block.
writer.end()
} {
root.close()
allocator.close()
}
}

protected override def newWriter(
env: SparkEnv,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,6 @@ case class TransformWithStateInPandasExec(
private val groupingKeyExprEncoder = ExpressionEncoder(groupingKeySchema)
.resolveAndBind().asInstanceOf[ExpressionEncoder[Any]]


/** The keys that may have a watermark attribute. */
override def keyExpressions: Seq[Attribute] = groupingAttributes

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,24 +34,23 @@ import org.apache.spark.sql.streaming.ValueState
import org.apache.spark.sql.types.StructType

/**
* This class is used to handle the state requests from the Python side.
* This class is used to handle the state requests from the Python side. It processes following
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we also info about which port this runs on and when its spawned/torn down ?

* state requests and return responses to the Python side:
* - Requests for managing explicit grouping key.
* - Stateful processor requests.
* - Requests for managing state variables (e.g. valueState).
*/
class TransformWithStateInPandasStateServer(
private val stateServerSocket: ServerSocket,
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: in many cases, having private val in constructor param in class is redundant.

private val statefulProcessorHandle: StatefulProcessorHandleImpl,
private val groupingKeySchema: StructType)
extends Runnable
with Logging{

extends Runnable with Logging {
private var inputStream: DataInputStream = _
private var outputStream: DataOutputStream = _

private val valueStates = mutable.HashMap[String, ValueState[Row]]()

def run(): Unit = {
val listeningSocket = stateServerSocket.accept()
logWarning(s"listening on socket - ${listeningSocket.getLocalAddress}")

inputStream = new DataInputStream(
new BufferedInputStream(listeningSocket.getInputStream))
outputStream = new DataOutputStream(
Expand All @@ -60,21 +59,15 @@ class TransformWithStateInPandasStateServer(

while (listeningSocket.isConnected &&
statefulProcessorHandle.getHandleState != StatefulProcessorHandleState.CLOSED) {

try {
val version = inputStream.readInt()

if (version != -1) {
assert(version == 0)
val messageLen = inputStream.readInt()

val messageBytes = new Array[Byte](messageLen)
inputStream.read(messageBytes)

val message = StateRequest.parseFrom(ByteString.copyFrom(messageBytes))

handleRequest(message)

outputStream.flush()
}
} catch {
Expand Down