forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathworker.py
More file actions
1310 lines (1104 loc) · 51.6 KB
/
worker.py
File metadata and controls
1310 lines (1104 loc) · 51.6 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
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# 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.
#
"""
Worker that receives input from Piped RDD.
"""
import os
import sys
import time
from inspect import currentframe, getframeinfo, getfullargspec
import importlib
import json
from typing import Any, Iterable, Iterator
# 'resource' is a Unix specific module.
has_resource_module = True
try:
import resource
except ImportError:
has_resource_module = False
import traceback
import warnings
import faulthandler
from pyspark.accumulators import _accumulatorRegistry
from pyspark.broadcast import Broadcast, _broadcastRegistry
from pyspark.java_gateway import local_connect_and_auth
from pyspark.taskcontext import BarrierTaskContext, TaskContext
from pyspark.files import SparkFiles
from pyspark.resource import ResourceInformation
from pyspark.rdd import PythonEvalType
from pyspark.serializers import (
write_with_length,
write_int,
read_long,
read_bool,
write_long,
read_int,
SpecialLengths,
UTF8Deserializer,
CPickleSerializer,
BatchedSerializer,
)
from pyspark.sql.pandas.serializers import (
ArrowStreamPandasUDFSerializer,
ArrowStreamPandasUDTFSerializer,
CogroupUDFSerializer,
ArrowStreamUDFSerializer,
ApplyInPandasWithStateSerializer,
)
from pyspark.sql.pandas.types import to_arrow_type
from pyspark.sql.types import BinaryType, StringType, StructType, _parse_datatype_json_string
from pyspark.util import fail_on_stopiteration, try_simplify_traceback
from pyspark import shuffle
from pyspark.errors import PySparkRuntimeError, PySparkTypeError
pickleSer = CPickleSerializer()
utf8_deserializer = UTF8Deserializer()
def report_times(outfile, boot, init, finish):
write_int(SpecialLengths.TIMING_DATA, outfile)
write_long(int(1000 * boot), outfile)
write_long(int(1000 * init), outfile)
write_long(int(1000 * finish), outfile)
def add_path(path):
# worker can be used, so do not add path multiple times
if path not in sys.path:
# overwrite system packages
sys.path.insert(1, path)
def read_command(serializer, file):
command = serializer._read_with_length(file)
if isinstance(command, Broadcast):
command = serializer.loads(command.value)
return command
def chain(f, g):
"""chain two functions together"""
return lambda *a: g(f(*a))
def wrap_udf(f, return_type):
if return_type.needConversion():
toInternal = return_type.toInternal
return lambda *a: toInternal(f(*a))
else:
return lambda *a: f(*a)
def wrap_scalar_pandas_udf(f, return_type):
arrow_return_type = to_arrow_type(return_type)
def verify_result_type(result):
if not hasattr(result, "__len__"):
pd_type = "pandas.DataFrame" if type(return_type) == StructType else "pandas.Series"
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": pd_type,
"actual": type(result).__name__,
},
)
return result
def verify_result_length(result, length):
if len(result) != length:
raise PySparkRuntimeError(
error_class="SCHEMA_MISMATCH_FOR_PANDAS_UDF",
message_parameters={
"expected": str(length),
"actual": str(len(result)),
},
)
return result
return lambda *a: (
verify_result_length(verify_result_type(f(*a)), len(a[0])),
arrow_return_type,
)
def wrap_arrow_batch_udf(f, return_type):
import pandas as pd
arrow_return_type = to_arrow_type(return_type)
# "result_func" ensures the result of a Python UDF to be consistent with/without Arrow
# optimization.
# Otherwise, an Arrow-optimized Python UDF raises "pyarrow.lib.ArrowTypeError: Expected a
# string or bytes dtype, got ..." whereas a non-Arrow-optimized Python UDF returns
# successfully.
result_func = lambda pdf: pdf # noqa: E731
if type(return_type) == StringType:
result_func = lambda r: str(r) if r is not None else r # noqa: E731
elif type(return_type) == BinaryType:
result_func = lambda r: bytes(r) if r is not None else r # noqa: E731
@fail_on_stopiteration
def evaluate(*args: pd.Series) -> pd.Series:
return pd.Series([result_func(f(*a)) for a in zip(*args)])
def verify_result_length(result, length):
if len(result) != length:
raise PySparkRuntimeError(
error_class="SCHEMA_MISMATCH_FOR_PANDAS_UDF",
message_parameters={
"expected": str(length),
"actual": str(len(result)),
},
)
return result
return lambda *a: (
verify_result_length(evaluate(*a), len(a[0])),
arrow_return_type,
)
def wrap_pandas_batch_iter_udf(f, return_type):
arrow_return_type = to_arrow_type(return_type)
iter_type_label = "pandas.DataFrame" if type(return_type) == StructType else "pandas.Series"
def verify_result(result):
if not isinstance(result, Iterator) and not hasattr(result, "__iter__"):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterator of {}".format(iter_type_label),
"actual": type(result).__name__,
},
)
return result
def verify_element(elem):
import pandas as pd
if not isinstance(elem, pd.DataFrame if type(return_type) == StructType else pd.Series):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterator of {}".format(iter_type_label),
"actual": "iterator of {}".format(type(elem).__name__),
},
)
verify_pandas_result(
elem, return_type, assign_cols_by_name=True, truncate_return_schema=True
)
return elem
return lambda *iterator: map(
lambda res: (res, arrow_return_type), map(verify_element, verify_result(f(*iterator)))
)
def verify_pandas_result(result, return_type, assign_cols_by_name, truncate_return_schema):
import pandas as pd
if type(return_type) == StructType:
if not isinstance(result, pd.DataFrame):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "pandas.DataFrame",
"actual": type(result).__name__,
},
)
# check the schema of the result only if it is not empty or has columns
if not result.empty or len(result.columns) != 0:
# if any column name of the result is a string
# the column names of the result have to match the return type
# see create_array in pyspark.sql.pandas.serializers.ArrowStreamPandasSerializer
field_names = set([field.name for field in return_type.fields])
# only the first len(field_names) result columns are considered
# when truncating the return schema
result_columns = (
result.columns[: len(field_names)] if truncate_return_schema else result.columns
)
column_names = set(result_columns)
if (
assign_cols_by_name
and any(isinstance(name, str) for name in result.columns)
and column_names != field_names
):
missing = sorted(list(field_names.difference(column_names)))
missing = f" Missing: {', '.join(missing)}." if missing else ""
extra = sorted(list(column_names.difference(field_names)))
extra = f" Unexpected: {', '.join(extra)}." if extra else ""
raise PySparkRuntimeError(
error_class="RESULT_COLUMNS_MISMATCH_FOR_PANDAS_UDF",
message_parameters={
"missing": missing,
"extra": extra,
},
)
# otherwise the number of columns of result have to match the return type
elif len(result_columns) != len(return_type):
raise PySparkRuntimeError(
error_class="RESULT_LENGTH_MISMATCH_FOR_PANDAS_UDF",
message_parameters={
"expected": str(len(return_type)),
"actual": str(len(result.columns)),
},
)
else:
if not isinstance(result, pd.Series):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={"expected": "pandas.Series", "actual": type(result).__name__},
)
def wrap_arrow_batch_iter_udf(f, return_type):
arrow_return_type = to_arrow_type(return_type)
def verify_result(result):
if not isinstance(result, Iterator) and not hasattr(result, "__iter__"):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterator of pyarrow.RecordBatch",
"actual": type(result).__name__,
},
)
return result
def verify_element(elem):
import pyarrow as pa
if not isinstance(elem, pa.RecordBatch):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterator of pyarrow.RecordBatch",
"actual": "iterator of {}".format(type(elem).__name__),
},
)
return elem
return lambda *iterator: map(
lambda res: (res, arrow_return_type), map(verify_element, verify_result(f(*iterator)))
)
def wrap_cogrouped_map_pandas_udf(f, return_type, argspec, runner_conf):
_assign_cols_by_name = assign_cols_by_name(runner_conf)
def wrapped(left_key_series, left_value_series, right_key_series, right_value_series):
import pandas as pd
left_df = pd.concat(left_value_series, axis=1)
right_df = pd.concat(right_value_series, axis=1)
if len(argspec.args) == 2:
result = f(left_df, right_df)
elif len(argspec.args) == 3:
key_series = left_key_series if not left_df.empty else right_key_series
key = tuple(s[0] for s in key_series)
result = f(key, left_df, right_df)
verify_pandas_result(
result, return_type, _assign_cols_by_name, truncate_return_schema=False
)
return result
return lambda kl, vl, kr, vr: [(wrapped(kl, vl, kr, vr), to_arrow_type(return_type))]
def wrap_grouped_map_pandas_udf(f, return_type, argspec, runner_conf):
_assign_cols_by_name = assign_cols_by_name(runner_conf)
def wrapped(key_series, value_series):
import pandas as pd
if len(argspec.args) == 1:
result = f(pd.concat(value_series, axis=1))
elif len(argspec.args) == 2:
key = tuple(s[0] for s in key_series)
result = f(key, pd.concat(value_series, axis=1))
verify_pandas_result(
result, return_type, _assign_cols_by_name, truncate_return_schema=False
)
return result
return lambda k, v: [(wrapped(k, v), to_arrow_type(return_type))]
def wrap_grouped_map_pandas_udf_with_state(f, return_type):
"""
Provides a new lambda instance wrapping user function of applyInPandasWithState.
The lambda instance receives (key series, iterator of value series, state) and performs
some conversion to be adapted with the signature of user function.
See the function doc of inner function `wrapped` for more details on what adapter does.
See the function doc of `mapper` function for
`eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE` for more details on
the input parameters of lambda function.
Along with the returned iterator, the lambda instance will also produce the return_type as
converted to the arrow schema.
"""
def wrapped(key_series, value_series_gen, state):
"""
Provide an adapter of the user function performing below:
- Extract the first value of all columns in key series and produce as a tuple.
- If the state has timed out, call the user function with empty pandas DataFrame.
- If not, construct a new generator which converts each element of value series to
pandas DataFrame (lazy evaluation), and call the user function with the generator
- Verify each element of returned iterator to check the schema of pandas DataFrame.
"""
import pandas as pd
key = tuple(s[0] for s in key_series)
if state.hasTimedOut:
# Timeout processing pass empty iterator. Here we return an empty DataFrame instead.
values = [
pd.DataFrame(columns=pd.concat(next(value_series_gen), axis=1).columns),
]
else:
values = (pd.concat(x, axis=1) for x in value_series_gen)
result_iter = f(key, values, state)
def verify_element(result):
if not isinstance(result, pd.DataFrame):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterator of pandas.DataFrame",
"actual": "iterator of {}".format(type(result).__name__),
},
)
# the number of columns of result have to match the return type
# but it is fine for result to have no columns at all if it is empty
if not (
len(result.columns) == len(return_type)
or (len(result.columns) == 0 and result.empty)
):
raise PySparkRuntimeError(
error_class="RESULT_LENGTH_MISMATCH_FOR_PANDAS_UDF",
message_parameters={
"expected": str(len(return_type)),
"actual": str(len(result.columns)),
},
)
return result
if isinstance(result_iter, pd.DataFrame):
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={
"expected": "iterable of pandas.DataFrame",
"actual": type(result_iter).__name__,
},
)
try:
iter(result_iter)
except TypeError:
raise PySparkTypeError(
error_class="UDF_RETURN_TYPE",
message_parameters={"expected": "iterable", "actual": type(result_iter).__name__},
)
result_iter_with_validation = (verify_element(x) for x in result_iter)
return (
result_iter_with_validation,
state,
)
return lambda k, v, s: [(wrapped(k, v, s), to_arrow_type(return_type))]
def wrap_grouped_agg_pandas_udf(f, return_type):
arrow_return_type = to_arrow_type(return_type)
def wrapped(*series):
import pandas as pd
result = f(*series)
return pd.Series([result])
return lambda *a: (wrapped(*a), arrow_return_type)
def wrap_window_agg_pandas_udf(f, return_type, runner_conf, udf_index):
window_bound_types_str = runner_conf.get("pandas_window_bound_types")
window_bound_type = [t.strip().lower() for t in window_bound_types_str.split(",")][udf_index]
if window_bound_type == "bounded":
return wrap_bounded_window_agg_pandas_udf(f, return_type)
elif window_bound_type == "unbounded":
return wrap_unbounded_window_agg_pandas_udf(f, return_type)
else:
raise PySparkRuntimeError(
error_class="INVALID_WINDOW_BOUND_TYPE",
message_parameters={
"window_bound_type": window_bound_type,
},
)
def wrap_unbounded_window_agg_pandas_udf(f, return_type):
# This is similar to grouped_agg_pandas_udf, the only difference
# is that window_agg_pandas_udf needs to repeat the return value
# to match window length, where grouped_agg_pandas_udf just returns
# the scalar value.
arrow_return_type = to_arrow_type(return_type)
def wrapped(*series):
import pandas as pd
result = f(*series)
return pd.Series([result]).repeat(len(series[0]))
return lambda *a: (wrapped(*a), arrow_return_type)
def wrap_bounded_window_agg_pandas_udf(f, return_type):
arrow_return_type = to_arrow_type(return_type)
def wrapped(begin_index, end_index, *series):
import pandas as pd
result = []
# Index operation is faster on np.ndarray,
# So we turn the index series into np array
# here for performance
begin_array = begin_index.values
end_array = end_index.values
for i in range(len(begin_array)):
# Note: Create a slice from a series for each window is
# actually pretty expensive. However, there
# is no easy way to reduce cost here.
# Note: s.iloc[i : j] is about 30% faster than s[i: j], with
# the caveat that the created slices shares the same
# memory with s. Therefore, user are not allowed to
# change the value of input series inside the window
# function. It is rare that user needs to modify the
# input series in the window function, and therefore,
# it is be a reasonable restriction.
# Note: Calling reset_index on the slices will increase the cost
# of creating slices by about 100%. Therefore, for performance
# reasons we don't do it here.
series_slices = [s.iloc[begin_array[i] : end_array[i]] for s in series]
result.append(f(*series_slices))
return pd.Series(result)
return lambda *a: (wrapped(*a), arrow_return_type)
def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index):
num_arg = read_int(infile)
arg_offsets = [read_int(infile) for i in range(num_arg)]
chained_func = None
for i in range(read_int(infile)):
f, return_type = read_command(pickleSer, infile)
if chained_func is None:
chained_func = f
else:
chained_func = chain(chained_func, f)
if eval_type in (
PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF,
PythonEvalType.SQL_ARROW_BATCHED_UDF,
):
func = chained_func
else:
# make sure StopIteration's raised in the user code are not ignored
# when they are processed in a for loop, raise them as RuntimeError's instead
func = fail_on_stopiteration(chained_func)
# the last returnType will be the return type of UDF
if eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF:
return arg_offsets, wrap_scalar_pandas_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_ARROW_BATCHED_UDF:
return arg_offsets, wrap_arrow_batch_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF:
return arg_offsets, wrap_pandas_batch_iter_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF:
return arg_offsets, wrap_pandas_batch_iter_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_MAP_ARROW_ITER_UDF:
return arg_offsets, wrap_arrow_batch_iter_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF:
argspec = getfullargspec(chained_func) # signature was lost when wrapping it
return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec, runner_conf)
elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE:
return arg_offsets, wrap_grouped_map_pandas_udf_with_state(func, return_type)
elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF:
argspec = getfullargspec(chained_func) # signature was lost when wrapping it
return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec, runner_conf)
elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF:
return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type)
elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF:
return arg_offsets, wrap_window_agg_pandas_udf(func, return_type, runner_conf, udf_index)
elif eval_type == PythonEvalType.SQL_BATCHED_UDF:
return arg_offsets, wrap_udf(func, return_type)
else:
raise ValueError("Unknown eval type: {}".format(eval_type))
# Used by SQL_GROUPED_MAP_PANDAS_UDF and SQL_SCALAR_PANDAS_UDF and SQL_ARROW_BATCHED_UDF when
# returning StructType
def assign_cols_by_name(runner_conf):
return (
runner_conf.get(
"spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true"
).lower()
== "true"
)
# Read and process a serialized user-defined table function (UDTF) from a socket.
# It expects the UDTF to be in a specific format and performs various checks to
# ensure the UDTF is valid. This function also prepares a mapper function for applying
# the UDTF logic to input rows.
def read_udtf(pickleSer, infile, eval_type):
if eval_type == PythonEvalType.SQL_ARROW_TABLE_UDF:
runner_conf = {}
# Load conf used for arrow evaluation.
num_conf = read_int(infile)
for i in range(num_conf):
k = utf8_deserializer.loads(infile)
v = utf8_deserializer.loads(infile)
runner_conf[k] = v
# NOTE: if timezone is set here, that implies respectSessionTimeZone is True
timezone = runner_conf.get("spark.sql.session.timeZone", None)
safecheck = (
runner_conf.get("spark.sql.execution.pandas.convertToArrowArraySafely", "false").lower()
== "true"
)
ser = ArrowStreamPandasUDTFSerializer(timezone, safecheck)
else:
# Each row is a group so do not batch but send one by one.
ser = BatchedSerializer(CPickleSerializer(), 1)
# See `PythonUDTFRunner.PythonUDFWriterThread.writeCommand'
num_arg = read_int(infile)
arg_offsets = [read_int(infile) for _ in range(num_arg)]
handler = read_command(pickleSer, infile)
if not isinstance(handler, type):
raise PySparkRuntimeError(
f"Invalid UDTF handler type. Expected a class (type 'type'), but "
f"got an instance of {type(handler).__name__}."
)
return_type = _parse_datatype_json_string(utf8_deserializer.loads(infile))
if not type(return_type) == StructType:
raise PySparkRuntimeError(
f"The return type of a UDTF must be a struct type, but got {type(return_type)}."
)
# Instantiate the UDTF class.
try:
udtf = handler()
except Exception as e:
raise PySparkRuntimeError(
error_class="UDTF_EXEC_ERROR",
message_parameters={"method_name": "__init__", "error": str(e)},
)
# Validate the UDTF
if not hasattr(udtf, "eval"):
raise PySparkRuntimeError(
"Failed to execute the user defined table function because it has not "
"implemented the 'eval' method. Please add the 'eval' method and try "
"the query again."
)
if eval_type == PythonEvalType.SQL_ARROW_TABLE_UDF:
def wrap_arrow_udtf(f, return_type):
import pandas as pd
arrow_return_type = to_arrow_type(return_type)
return_type_size = len(return_type)
def verify_result(result):
if not isinstance(result, pd.DataFrame):
raise PySparkTypeError(
error_class="INVALID_ARROW_UDTF_RETURN_TYPE",
message_parameters={
"type_name": type(result).__name__,
"value": str(result),
"func": f.__name__,
},
)
# Validate the output schema when the result dataframe has either output
# rows or columns. Note that we avoid using `df.empty` here because the
# result dataframe may contain an empty row. For example, when a UDTF is
# defined as follows: def eval(self): yield tuple().
if len(result) > 0 or len(result.columns) > 0:
if len(result.columns) != return_type_size:
raise PySparkRuntimeError(
error_class="UDTF_RETURN_SCHEMA_MISMATCH",
message_parameters={
"expected": str(return_type_size),
"actual": str(len(result.columns)),
"func": f.__name__,
},
)
# Verify the type and the schema of the result.
verify_pandas_result(
result, return_type, assign_cols_by_name=False, truncate_return_schema=False
)
return result
# Wrap the exception thrown from the UDTF in a PySparkRuntimeError.
def func(*args: Any) -> Any:
try:
return f(*args)
except Exception as e:
raise PySparkRuntimeError(
error_class="UDTF_EXEC_ERROR",
message_parameters={"method_name": f.__name__, "error": str(e)},
)
def check_return_value(res):
# Check whether the result of an arrow UDTF is iterable before
# using it to construct a pandas DataFrame.
if res is not None and not isinstance(res, Iterable):
raise PySparkRuntimeError(
error_class="UDTF_RETURN_NOT_ITERABLE",
message_parameters={
"type": type(res).__name__,
"func": f.__name__,
},
)
def evaluate(*args: pd.Series):
if len(args) == 0:
res = func()
check_return_value(res)
yield verify_result(pd.DataFrame(res)), arrow_return_type
else:
# Create tuples from the input pandas Series, each tuple
# represents a row across all Series.
row_tuples = zip(*args)
for row in row_tuples:
res = func(*row)
check_return_value(res)
yield verify_result(pd.DataFrame(res)), arrow_return_type
return evaluate
eval = wrap_arrow_udtf(getattr(udtf, "eval"), return_type)
if hasattr(udtf, "terminate"):
terminate = wrap_arrow_udtf(getattr(udtf, "terminate"), return_type)
else:
terminate = None
def mapper(_, it):
try:
for a in it:
# The eval function yields an iterator. Each element produced by this
# iterator is a tuple in the form of (pandas.DataFrame, arrow_return_type).
yield from eval(*[a[o] for o in arg_offsets])
finally:
if terminate is not None:
yield from terminate()
return mapper, None, ser, ser
else:
def wrap_udtf(f, return_type):
assert return_type.needConversion()
toInternal = return_type.toInternal
return_type_size = len(return_type)
def verify_and_convert_result(result):
if result is not None:
if hasattr(result, "__len__") and len(result) != return_type_size:
raise PySparkRuntimeError(
error_class="UDTF_RETURN_SCHEMA_MISMATCH",
message_parameters={
"expected": str(return_type_size),
"actual": str(len(result)),
"func": f.__name__,
},
)
if not (isinstance(result, (list, dict, tuple)) or hasattr(result, "__dict__")):
raise PySparkRuntimeError(
error_class="UDTF_INVALID_OUTPUT_ROW_TYPE",
message_parameters={
"type": type(result).__name__,
"func": f.__name__,
},
)
return toInternal(result)
# Evaluate the function and return a tuple back to the executor.
def evaluate(*a) -> tuple:
try:
res = f(*a)
except Exception as e:
raise PySparkRuntimeError(
error_class="UDTF_EXEC_ERROR",
message_parameters={"method_name": f.__name__, "error": str(e)},
)
if res is None:
# If the function returns None or does not have an explicit return statement,
# an empty tuple is returned to the executor.
# This is because directly constructing tuple(None) results in an exception.
return tuple()
if not isinstance(res, Iterable):
raise PySparkRuntimeError(
error_class="UDTF_RETURN_NOT_ITERABLE",
message_parameters={
"type": type(res).__name__,
"func": f.__name__,
},
)
# If the function returns a result, we map it to the internal representation and
# returns the results as a tuple.
return tuple(map(verify_and_convert_result, res))
return evaluate
eval = wrap_udtf(getattr(udtf, "eval"), return_type)
if hasattr(udtf, "terminate"):
terminate = wrap_udtf(getattr(udtf, "terminate"), return_type)
else:
terminate = None
# Return an iterator of iterators.
def mapper(_, it):
try:
for a in it:
yield eval(*[a[o] for o in arg_offsets])
finally:
if terminate is not None:
yield terminate()
return mapper, None, ser, ser
def read_udfs(pickleSer, infile, eval_type):
runner_conf = {}
if eval_type in (
PythonEvalType.SQL_ARROW_BATCHED_UDF,
PythonEvalType.SQL_SCALAR_PANDAS_UDF,
PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF,
PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF,
PythonEvalType.SQL_MAP_PANDAS_ITER_UDF,
PythonEvalType.SQL_MAP_ARROW_ITER_UDF,
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF,
PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF,
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE,
):
# Load conf used for pandas_udf evaluation
num_conf = read_int(infile)
for i in range(num_conf):
k = utf8_deserializer.loads(infile)
v = utf8_deserializer.loads(infile)
runner_conf[k] = v
state_object_schema = None
if eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE:
state_object_schema = StructType.fromJson(json.loads(utf8_deserializer.loads(infile)))
# NOTE: if timezone is set here, that implies respectSessionTimeZone is True
timezone = runner_conf.get("spark.sql.session.timeZone", None)
safecheck = (
runner_conf.get("spark.sql.execution.pandas.convertToArrowArraySafely", "false").lower()
== "true"
)
if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF:
ser = CogroupUDFSerializer(timezone, safecheck, assign_cols_by_name(runner_conf))
elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE:
arrow_max_records_per_batch = runner_conf.get(
"spark.sql.execution.arrow.maxRecordsPerBatch", 10000
)
arrow_max_records_per_batch = int(arrow_max_records_per_batch)
ser = ApplyInPandasWithStateSerializer(
timezone,
safecheck,
assign_cols_by_name(runner_conf),
state_object_schema,
arrow_max_records_per_batch,
)
elif eval_type == PythonEvalType.SQL_MAP_ARROW_ITER_UDF:
ser = ArrowStreamUDFSerializer()
else:
# Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of
# pandas Series. See SPARK-27240.
df_for_struct = (
eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF
or eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF
or eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF
)
# Arrow-optimized Python UDF takes a struct type argument as a Row
struct_in_pandas = (
"row" if eval_type == PythonEvalType.SQL_ARROW_BATCHED_UDF else "dict"
)
ndarray_as_list = eval_type == PythonEvalType.SQL_ARROW_BATCHED_UDF
# Arrow-optimized Python UDF uses explicit Arrow cast for type coercion
arrow_cast = eval_type == PythonEvalType.SQL_ARROW_BATCHED_UDF
ser = ArrowStreamPandasUDFSerializer(
timezone,
safecheck,
assign_cols_by_name(runner_conf),
df_for_struct,
struct_in_pandas,
ndarray_as_list,
arrow_cast,
)
else:
ser = BatchedSerializer(CPickleSerializer(), 100)
num_udfs = read_int(infile)
is_scalar_iter = eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF
is_map_pandas_iter = eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF
is_map_arrow_iter = eval_type == PythonEvalType.SQL_MAP_ARROW_ITER_UDF
if is_scalar_iter or is_map_pandas_iter or is_map_arrow_iter:
if is_scalar_iter:
assert num_udfs == 1, "One SCALAR_ITER UDF expected here."
if is_map_pandas_iter:
assert num_udfs == 1, "One MAP_PANDAS_ITER UDF expected here."
if is_map_arrow_iter:
assert num_udfs == 1, "One MAP_ARROW_ITER UDF expected here."
arg_offsets, udf = read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index=0)
def func(_, iterator):
num_input_rows = 0
def map_batch(batch):
nonlocal num_input_rows
udf_args = [batch[offset] for offset in arg_offsets]
num_input_rows += len(udf_args[0])
if len(udf_args) == 1:
return udf_args[0]
else:
return tuple(udf_args)
iterator = map(map_batch, iterator)
result_iter = udf(iterator)
num_output_rows = 0
for result_batch, result_type in result_iter:
num_output_rows += len(result_batch)
# This assert is for Scalar Iterator UDF to fail fast.
# The length of the entire input can only be explicitly known
# by consuming the input iterator in user side. Therefore,
# it's very unlikely the output length is higher than
# input length.
assert (
is_map_pandas_iter or is_map_arrow_iter or num_output_rows <= num_input_rows
), "Pandas SCALAR_ITER UDF outputted more rows than input rows."
yield (result_batch, result_type)
if is_scalar_iter:
try:
next(iterator)
except StopIteration:
pass
else:
raise PySparkRuntimeError(
error_class="STOP_ITERATION_OCCURRED_FROM_SCALAR_ITER_PANDAS_UDF",
message_parameters={},
)
if num_output_rows != num_input_rows:
raise PySparkRuntimeError(
error_class="RESULT_LENGTH_MISMATCH_FOR_SCALAR_ITER_PANDAS_UDF",
message_parameters={
"output_length": str(num_output_rows),
"input_length": str(num_input_rows),
},
)
# profiling is not supported for UDF
return func, None, ser, ser
def extract_key_value_indexes(grouped_arg_offsets):
"""
Helper function to extract the key and value indexes from arg_offsets for the grouped and
cogrouped pandas udfs. See BasePandasGroupExec.resolveArgOffsets for equivalent scala code.
Parameters
----------
grouped_arg_offsets: list
List containing the key and value indexes of columns of the
DataFrames to be passed to the udf. It consists of n repeating groups where n is the
number of DataFrames. Each group has the following format:
group[0]: length of group
group[1]: length of key indexes
group[2.. group[1] +2]: key attributes
group[group[1] +3 group[0]]: value attributes
"""
parsed = []
idx = 0
while idx < len(grouped_arg_offsets):
offsets_len = grouped_arg_offsets[idx]
idx += 1
offsets = grouped_arg_offsets[idx : idx + offsets_len]
split_index = offsets[0] + 1
offset_keys = offsets[1:split_index]
offset_values = offsets[split_index:]
parsed.append([offset_keys, offset_values])
idx += offsets_len
return parsed
if eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF:
# We assume there is only one UDF here because grouped map doesn't
# support combining multiple UDFs.
assert num_udfs == 1
# See FlatMapGroupsInPandasExec for how arg_offsets are used to