-
Notifications
You must be signed in to change notification settings - Fork 29.2k
Expand file tree
/
Copy pathstateful_processor_api_client.py
More file actions
648 lines (539 loc) · 28.4 KB
/
stateful_processor_api_client.py
File metadata and controls
648 lines (539 loc) · 28.4 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
#
# 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 json
import os
import socket
from typing import Any, Dict, List, Union, Optional, Tuple, Iterator
from pyspark.serializers import write_int, read_int, UTF8Deserializer
from pyspark.sql.pandas.serializers import ArrowStreamSerializer
from pyspark.sql.types import (
StructType,
Row,
)
from pyspark.sql.pandas.types import convert_pandas_using_numpy_type
from pyspark.sql.utils import has_numpy
from pyspark.serializers import CPickleSerializer
from pyspark.errors import PySparkRuntimeError
import uuid
__all__ = ["StatefulProcessorApiClient", "StatefulProcessorHandleState"]
if has_numpy:
import numpy as np
def _normalize_value(v: Any) -> Any:
# Convert NumPy types to Python primitive types.
if isinstance(v, np.generic):
return v.tolist()
# Named tuples (collections.namedtuple or typing.NamedTuple) and Row both
# require positional arguments and cannot be instantiated
# with a generator expression.
if isinstance(v, Row) or (isinstance(v, tuple) and hasattr(v, "_fields")):
return type(v)(*[_normalize_value(e) for e in v])
# List / tuple: recursively normalize each element
if isinstance(v, (list, tuple)):
return type(v)(_normalize_value(e) for e in v)
# Dict: normalize both keys and values
if isinstance(v, dict):
return {_normalize_value(k): _normalize_value(val) for k, val in v.items()}
# Address a couple of pandas dtypes too.
elif hasattr(v, "to_pytimedelta"):
return v.to_pytimedelta()
elif hasattr(v, "to_pydatetime"):
return v.to_pydatetime()
return v
def _normalize_tuple(data: Tuple) -> Tuple:
return tuple(_normalize_value(v) for v in data)
else:
def _normalize_tuple(data: Tuple) -> Tuple:
return data # toInternal handles tuples natively
class StatefulProcessorHandleState(Enum):
PRE_INIT = 0
CREATED = 1
INITIALIZED = 2
DATA_PROCESSED = 3
TIMER_PROCESSED = 4
CLOSED = 5
class StatefulProcessorApiClient:
def __init__(
self, state_server_port: Union[int, str], key_schema: StructType, is_driver: bool = False
) -> None:
self.key_schema = key_schema
if isinstance(state_server_port, str):
self._client_socket = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM)
self._client_socket.connect(state_server_port)
else:
self._client_socket = socket.socket()
self._client_socket.connect(("localhost", state_server_port))
# SPARK-51667: We have a pattern of sending messages continuously from one side
# (Python -> JVM, and vice versa) before getting response from other side. Since most
# messages we are sending are small, this triggers the bad combination of Nagle's
# algorithm and delayed ACKs, which can cause a significant delay on the latency.
# See SPARK-51667 for more details on how this can be a problem.
#
# Disabling either would work, but it's more common to disable Nagle's algorithm; there
# is lot less reference to disabling delayed ACKs, while there are lots of resources to
# disable Nagle's algorithm.
self._client_socket.setsockopt(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)
self.sockfile = self._client_socket.makefile(
"rwb", int(os.environ.get("SPARK_BUFFER_SIZE", 65536))
)
if is_driver:
self.handle_state = StatefulProcessorHandleState.PRE_INIT
else:
self.handle_state = StatefulProcessorHandleState.CREATED
self.utf8_deserializer = UTF8Deserializer()
self.pickleSer = CPickleSerializer()
self.serializer = ArrowStreamSerializer()
# Dictionaries to store the mapping between iterator id and a tuple of data batch
# and the index of the last row that was read.
self.list_timer_iterator_cursors: Dict[str, Tuple[Any, int, bool]] = {}
self.expiry_timer_iterator_cursors: Dict[str, Tuple[Any, int, bool]] = {}
# Cache of schema-id -> fast-serialize callable, so we avoid
# rebuilding field_names / Row / closure on every _serialize_to_bytes call.
self._serializer_cache: Dict[int, Any] = {}
# statefulProcessorApiClient is initialized per batch per partition,
# so we will have new timestamps for a new batch
self._batch_timestamp = -1
self._watermark_timestamp = -1
def set_handle_state(self, state: StatefulProcessorHandleState) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if state == StatefulProcessorHandleState.PRE_INIT:
proto_state = stateMessage.PRE_INIT
elif state == StatefulProcessorHandleState.CREATED:
proto_state = stateMessage.CREATED
elif state == StatefulProcessorHandleState.INITIALIZED:
proto_state = stateMessage.INITIALIZED
elif state == StatefulProcessorHandleState.DATA_PROCESSED:
proto_state = stateMessage.DATA_PROCESSED
elif state == StatefulProcessorHandleState.TIMER_PROCESSED:
proto_state = stateMessage.TIMER_PROCESSED
else:
proto_state = stateMessage.CLOSED
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.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status == 0:
self.handle_state = state
else:
# TODO(SPARK-49233): Classify errors thrown by internal methods.
raise PySparkRuntimeError(f"Error setting handle state: {response_message[1]}")
def set_implicit_key(self, key: Tuple) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
key_bytes = self._serialize_to_bytes(self.key_schema, key)
set_implicit_key = stateMessage.SetImplicitKey(key=key_bytes)
request = stateMessage.ImplicitGroupingKeyRequest(setImplicitKey=set_implicit_key)
message = stateMessage.StateRequest(implicitGroupingKeyRequest=request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify errors thrown by internal methods.
raise PySparkRuntimeError(f"Error setting implicit key: {response_message[1]}")
def remove_implicit_key(self) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
remove_implicit_key = stateMessage.RemoveImplicitKey()
request = stateMessage.ImplicitGroupingKeyRequest(removeImplicitKey=remove_implicit_key)
message = stateMessage.StateRequest(implicitGroupingKeyRequest=request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify errors thrown by internal methods.
raise PySparkRuntimeError(f"Error removing implicit key: {response_message[1]}")
def get_value_state(
self, state_name: str, schema: Union[StructType, str], ttl_duration_ms: Optional[int]
) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if isinstance(schema, str):
schema = self._parse_string_schema(schema)
state_call_command = stateMessage.StateCallCommand()
state_call_command.stateName = state_name
state_call_command.schema = schema.json()
if ttl_duration_ms is not None:
state_call_command.ttl.durationMs = ttl_duration_ms
call = stateMessage.StatefulProcessorCall(getValueState=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error initializing value state: {response_message[1]}")
def get_list_state(
self, state_name: str, schema: Union[StructType, str], ttl_duration_ms: Optional[int]
) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if isinstance(schema, str):
schema = self._parse_string_schema(schema)
state_call_command = stateMessage.StateCallCommand()
state_call_command.stateName = state_name
state_call_command.schema = schema.json()
if ttl_duration_ms is not None:
state_call_command.ttl.durationMs = ttl_duration_ms
call = stateMessage.StatefulProcessorCall(getListState=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error initializing list state: {response_message[1]}")
def register_timer(self, expiry_time_stamp_ms: int) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
register_call = stateMessage.RegisterTimer(expiryTimestampMs=expiry_time_stamp_ms)
state_call_command = stateMessage.TimerStateCallCommand(register=register_call)
call = stateMessage.StatefulProcessorCall(timerStateCall=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error register timer: {response_message[1]}")
def delete_timer(self, expiry_time_stamp_ms: int) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
delete_call = stateMessage.DeleteTimer(expiryTimestampMs=expiry_time_stamp_ms)
state_call_command = stateMessage.TimerStateCallCommand(delete=delete_call)
call = stateMessage.StatefulProcessorCall(timerStateCall=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error deleting timer: {response_message[1]}")
def get_list_timer_row(self, iterator_id: str) -> Tuple[int, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if iterator_id in self.list_timer_iterator_cursors:
# if the iterator is already in the dictionary, return the next row
data_batch, index, require_next_fetch = self.list_timer_iterator_cursors[iterator_id]
else:
list_call = stateMessage.ListTimers(iteratorId=iterator_id)
state_call_command = stateMessage.TimerStateCallCommand(list=list_call)
call = stateMessage.StatefulProcessorCall(timerStateCall=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message_with_timers()
status = response_message[0]
if status == 0:
data_batch = list(map(lambda x: x.timestampMs, response_message[2]))
require_next_fetch = response_message[3]
index = 0
else:
raise StopIteration()
is_last_row = False
new_index = index + 1
if new_index < len(data_batch):
# Update the index in the dictionary.
self.list_timer_iterator_cursors[iterator_id] = (
data_batch,
new_index,
require_next_fetch,
)
else:
# If the index is at the end of the data batch, remove the state from the dictionary.
self.list_timer_iterator_cursors.pop(iterator_id, None)
is_last_row = True
is_last_row_from_iterator = is_last_row and not require_next_fetch
timestamp = data_batch[index]
return (timestamp, is_last_row_from_iterator)
def get_expiry_timers_iterator(
self, iterator_id: str, expiry_timestamp: int
) -> Tuple[Tuple, int, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if iterator_id in self.expiry_timer_iterator_cursors:
# If the state is already in the dictionary, return the next row.
data_batch, index, require_next_fetch = self.expiry_timer_iterator_cursors[iterator_id]
else:
expiry_timer_call = stateMessage.ExpiryTimerRequest(
expiryTimestampMs=expiry_timestamp, iteratorId=iterator_id
)
timer_request = stateMessage.TimerRequest(expiryTimerRequest=expiry_timer_call)
message = stateMessage.StateRequest(timerRequest=timer_request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message_with_timers()
status = response_message[0]
if status == 0:
data_batch = list(
map(
lambda x: (self._deserialize_from_bytes(x.key), x.timestampMs),
response_message[2],
)
)
require_next_fetch = response_message[3]
index = 0
else:
raise StopIteration()
is_last_row = False
new_index = index + 1
if new_index < len(data_batch):
# Update the index in the dictionary.
self.expiry_timer_iterator_cursors[iterator_id] = (
data_batch,
new_index,
require_next_fetch,
)
else:
# If the index is at the end of the data batch, remove the state from the dictionary.
self.expiry_timer_iterator_cursors.pop(iterator_id, None)
is_last_row = True
is_last_row_from_iterator = is_last_row and not require_next_fetch
key, timestamp = data_batch[index]
return (key, timestamp, is_last_row_from_iterator)
def get_timestamps(self, time_mode: str) -> Tuple[int, int]:
if time_mode.lower() == "none":
return -1, -1
else:
if self._batch_timestamp == -1:
self._batch_timestamp = self._get_batch_timestamp()
if self._watermark_timestamp == -1:
self._watermark_timestamp = self._get_watermark_timestamp()
return self._batch_timestamp, self._watermark_timestamp
def get_map_state(
self,
state_name: str,
user_key_schema: Union[StructType, str],
value_schema: Union[StructType, str],
ttl_duration_ms: Optional[int],
) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
if isinstance(user_key_schema, str):
user_key_schema = self._parse_string_schema(user_key_schema)
if isinstance(value_schema, str):
value_schema = self._parse_string_schema(value_schema)
state_call_command = stateMessage.StateCallCommand()
state_call_command.stateName = state_name
state_call_command.schema = user_key_schema.json()
state_call_command.mapStateValueSchema = value_schema.json()
if ttl_duration_ms is not None:
state_call_command.ttl.durationMs = ttl_duration_ms
call = stateMessage.StatefulProcessorCall(getMapState=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error initializing map state: {response_message[1]}")
def delete_if_exists(self, state_name: str) -> None:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
state_call_command = stateMessage.StateCallCommand()
state_call_command.stateName = state_name
call = stateMessage.StatefulProcessorCall(deleteIfExists=state_call_command)
message = stateMessage.StateRequest(statefulProcessorCall=call)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error deleting state: {response_message[1]}")
def _get_batch_timestamp(self) -> int:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
get_processing_time_call = stateMessage.GetProcessingTime()
timer_value_call = stateMessage.TimerValueRequest(
getProcessingTimer=get_processing_time_call
)
timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call)
message = stateMessage.StateRequest(timerRequest=timer_request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message_with_long_value()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error getting processing timestamp: {response_message[1]}")
else:
timestamp = response_message[2]
return timestamp
def _get_watermark_timestamp(self) -> int:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
get_watermark_call = stateMessage.GetWatermark()
timer_value_call = stateMessage.TimerValueRequest(getWatermark=get_watermark_call)
timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call)
message = stateMessage.StateRequest(timerRequest=timer_request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message_with_long_value()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error getting eventtime timestamp: {response_message[1]}")
else:
timestamp = response_message[2]
return timestamp
def _send_proto_message(self, message: bytes) -> None:
# 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(message), self.sockfile)
self.sockfile.write(message)
self.sockfile.flush()
def _receive_proto_message(self) -> Tuple[int, str, bytes]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponse()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.value
def _receive_proto_message_with_long_value(self) -> Tuple[int, str, int]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithLongTypeVal()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.value
def _receive_proto_message_with_string_value(self) -> Tuple[int, str, str]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithStringTypeVal()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.value
# The third return type is RepeatedScalarFieldContainer[bytes], which is protobuf's container
# type. We simplify it to Any here to avoid unnecessary complexity.
def _receive_proto_message_with_list_get(self) -> Tuple[int, str, Any, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithListGet()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.value, message.requireNextFetch
# The third return type is RepeatedScalarFieldContainer[bytes], which is protobuf's container
# type. We simplify it to Any here to avoid unnecessary complexity.
def _receive_proto_message_with_map_keys_values(self) -> Tuple[int, str, Any, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithMapKeysOrValues()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.value, message.requireNextFetch
# The third return type is RepeatedScalarFieldContainer[KeyAndValuePair], which is protobuf's
# container type. We simplify it to Any here to avoid unnecessary complexity.
def _receive_proto_message_with_map_pairs(self) -> Tuple[int, str, Any, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithMapIterator()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.kvPair, message.requireNextFetch
# The third return type is RepeatedScalarFieldContainer[TimerInfo], which is protobuf's
# container type. We simplify it to Any here to avoid unnecessary complexity.
def _receive_proto_message_with_timers(self) -> Tuple[int, str, Any, bool]:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
length = read_int(self.sockfile)
bytes = self.sockfile.read(length)
message = stateMessage.StateResponseWithTimer()
message.ParseFromString(bytes)
return message.statusCode, message.errorMessage, message.timer, message.requireNextFetch
def _receive_str(self) -> str:
return self.utf8_deserializer.loads(self.sockfile)
def _get_serializer(self, schema: StructType) -> Any:
schema_id = id(schema)
serializer = self._serializer_cache.get(schema_id)
if serializer is not None:
return serializer
to_internal = schema.toInternal
dumps = self.pickleSer.dumps
def _fast_serialize(data: Tuple) -> bytes:
return dumps(to_internal(_normalize_tuple(data)))
self._serializer_cache[schema_id] = _fast_serialize
return _fast_serialize
def _serialize_to_bytes(self, schema: StructType, data: Tuple) -> bytes:
return self._get_serializer(schema)(data)
def _deserialize_from_bytes(self, value: bytes) -> Any:
return self.pickleSer.loads(value)
def _send_arrow_state(self, schema: StructType, state: List[Tuple]) -> None:
import pyarrow as pa
import pandas as pd
column_names = [field.name for field in schema.fields]
pandas_df = convert_pandas_using_numpy_type(
pd.DataFrame(state, columns=column_names), schema
)
batch = pa.RecordBatch.from_pandas(pandas_df)
self.serializer.dump_stream(iter([batch]), self.sockfile)
self.sockfile.flush()
def _read_arrow_state(self) -> Any:
return self.serializer.load_stream(self.sockfile)
def _send_list_state(self, schema: StructType, state: List[Tuple]) -> None:
for value in state:
bytes = self._serialize_to_bytes(schema, value)
length = len(bytes)
write_int(length, self.sockfile)
self.sockfile.write(bytes)
write_int(-1, self.sockfile)
self.sockfile.flush()
def _read_list_state(self) -> List[Any]:
data_array = []
while True:
length = read_int(self.sockfile)
if length < 0:
break
bytes = self.sockfile.read(length)
data_array.append(self._deserialize_from_bytes(bytes))
return data_array
# Parse a string schema into a StructType schema. This method will perform an API call to
# JVM side to parse the schema string.
def _parse_string_schema(self, schema: str) -> StructType:
import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage
parse_string_schema_call = stateMessage.ParseStringSchema(schema=schema)
utils_request = stateMessage.UtilsRequest(parseStringSchema=parse_string_schema_call)
message = stateMessage.StateRequest(utilsRequest=utils_request)
self._send_proto_message(message.SerializeToString())
response_message = self._receive_proto_message_with_string_value()
status = response_message[0]
if status != 0:
# TODO(SPARK-49233): Classify user facing errors.
raise PySparkRuntimeError(f"Error parsing string schema: {response_message[1]}")
else:
return StructType.fromJson(json.loads(response_message[2]))
class ListTimerIterator:
def __init__(self, stateful_processor_api_client: StatefulProcessorApiClient):
# Generate a unique identifier for the iterator to make sure iterators on the
# same partition won't interfere with each other
self.iterator_id = str(uuid.uuid4())
self.stateful_processor_api_client = stateful_processor_api_client
self.iterator_fully_consumed = False
def __iter__(self) -> Iterator[int]:
return self
def __next__(self) -> int:
if self.iterator_fully_consumed:
raise StopIteration()
ts, is_last_row = self.stateful_processor_api_client.get_list_timer_row(self.iterator_id)
if is_last_row:
self.iterator_fully_consumed = True
return ts
class ExpiredTimerIterator:
def __init__(
self, stateful_processor_api_client: StatefulProcessorApiClient, expiry_timestamp: int
):
# Generate a unique identifier for the iterator to make sure iterators on the
# same partition won't interfere with each other
self.iterator_id = str(uuid.uuid4())
self.stateful_processor_api_client = stateful_processor_api_client
self.expiry_timestamp = expiry_timestamp
self.iterator_fully_consumed = False
def __iter__(self) -> Iterator[Tuple[Tuple, int]]:
return self
def __next__(self) -> Tuple[Tuple, int]:
if self.iterator_fully_consumed:
raise StopIteration()
key, ts, is_last_row = self.stateful_processor_api_client.get_expiry_timers_iterator(
self.iterator_id, self.expiry_timestamp
)
if is_last_row:
self.iterator_fully_consumed = True
return (key, ts)