-
Notifications
You must be signed in to change notification settings - Fork 29.2k
Expand file tree
/
Copy pathcore.py
More file actions
2515 lines (2245 loc) · 99.4 KB
/
core.py
File metadata and controls
2515 lines (2245 loc) · 99.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
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.
#
__all__ = [
"ChannelBuilder",
"DefaultChannelBuilder",
"SparkConnectClient",
]
import atexit
import pyspark
from pyspark.sql.connect.proto.base_pb2 import FetchErrorDetailsResponse
import concurrent.futures
import logging
import threading
import os
import copy
import platform
import urllib.parse
import uuid
import sys
import time
import traceback
import weakref
from typing import (
Iterable,
Iterator,
Optional,
Any,
Union,
List,
Tuple,
Dict,
Set,
NoReturn,
Mapping,
cast,
TYPE_CHECKING,
Type,
)
import pandas as pd
import pyarrow as pa
import google.protobuf.message
from grpc_status import rpc_status
import grpc
from google.protobuf import text_format, any_pb2
from google.rpc import error_details_pb2
from pyspark.util import is_remote_only, disable_gc
from pyspark.accumulators import SpecialAccumulatorIds, pickleSer
from pyspark.version import __version__
from pyspark.traceback_utils import CallSite
from pyspark.resource.information import ResourceInformation
from pyspark.sql.metrics import MetricValue, PlanMetrics, ExecutionInfo, ObservedMetrics
from pyspark.sql.connect.client.artifact import ArtifactManager
from pyspark.sql.connect.logging import logger
from pyspark.sql.connect.profiler import ConnectProfilerCollector
from pyspark.sql.connect.client.reattach import ExecutePlanResponseReattachableIterator
from pyspark.sql.connect.client.retries import RetryPolicy, Retrying, DefaultPolicy
from pyspark.sql.connect.conversion import (
storage_level_to_proto,
proto_to_storage_level,
proto_to_remote_cached_dataframe,
)
import pyspark.sql.connect.proto as pb2
import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib
import pyspark.sql.connect.types as types
from pyspark.errors.exceptions.connect import (
convert_exception,
convert_observation_errors,
SparkConnectException,
SparkConnectGrpcException,
)
from pyspark.sql.connect.expressions import (
LiteralExpression,
PythonUDF,
CommonInlineUserDefinedFunction,
JavaUDF,
)
from pyspark.sql.connect.plan import (
CommonInlineUserDefinedTableFunction,
CommonInlineUserDefinedDataSource,
PythonUDTF,
PythonDataSource,
)
from pyspark.sql.connect.observation import Observation
from pyspark.sql.connect.utils import get_python_ver
from pyspark.sql.pandas.types import from_arrow_schema
from pyspark.sql.pandas.conversion import _convert_arrow_table_to_pandas
from pyspark.sql.types import DataType, StructType
from pyspark.util import PythonEvalType
from pyspark.storagelevel import StorageLevel
from pyspark.errors import PySparkValueError, PySparkAssertionError, PySparkNotImplementedError
from pyspark.sql.connect.shell.progress import Progress, ProgressHandler, from_proto
if TYPE_CHECKING:
from google.rpc.error_details_pb2 import ErrorInfo
from google.rpc.status_pb2 import Status
from pyspark.sql.connect._typing import DataTypeOrString
from pyspark.sql.connect.session import SparkSession
from pyspark.sql.datasource import DataSource
PYSPARK_ROOT = os.path.dirname(pyspark.__file__)
def _import_zstandard_if_available() -> Optional[Any]:
"""
Import zstandard if available, otherwise return None.
This is used to handle the case when zstandard is not installed.
"""
try:
import zstandard
return zstandard
except ImportError:
return None
class ChannelBuilder:
"""
This is a helper class that is used to create a GRPC channel based on the given
connection string per the documentation of Spark Connect.
The standard implementation is in :class:`DefaultChannelBuilder`.
"""
PARAM_USE_SSL = "use_ssl"
PARAM_TOKEN = "token"
PARAM_USER_ID = "user_id"
PARAM_USER_AGENT = "user_agent"
PARAM_SESSION_ID = "session_id"
GRPC_MAX_MESSAGE_LENGTH_DEFAULT = 128 * 1024 * 1024
GRPC_DEFAULT_OPTIONS = [
("grpc.max_send_message_length", GRPC_MAX_MESSAGE_LENGTH_DEFAULT),
("grpc.max_receive_message_length", GRPC_MAX_MESSAGE_LENGTH_DEFAULT),
]
def __init__(
self,
channelOptions: Optional[List[Tuple[str, Any]]] = None,
params: Optional[Dict[str, str]] = None,
):
self._interceptors: List[grpc.UnaryStreamClientInterceptor] = []
self._params: Dict[str, str] = params or dict()
self._channel_options: List[Tuple[str, Any]] = ChannelBuilder.GRPC_DEFAULT_OPTIONS.copy()
if channelOptions is not None:
for key, value in channelOptions:
self.setChannelOption(key, value)
def get(self, key: str) -> Any:
"""
Parameters
----------
key : str
Parameter key name.
Returns
-------
The parameter value if present, raises exception otherwise.
"""
return self._params[key]
def getDefault(self, key: str, default: Any) -> Any:
return self._params.get(key, default)
def set(self, key: str, value: Any) -> None:
self._params[key] = value
def setChannelOption(self, key: str, value: Any) -> None:
# overwrite option if it exists already else append it
for i, option in enumerate(self._channel_options):
if option[0] == key:
self._channel_options[i] = (key, value)
return
self._channel_options.append((key, value))
def add_interceptor(self, interceptor: grpc.UnaryStreamClientInterceptor) -> None:
self._interceptors.append(interceptor)
def toChannel(self) -> grpc.Channel:
"""
The actual channel builder implementations should implement this function
to return grpc Channel.
This function should generally use self._insecure_channel or
self._secure_channel so that configuration options are applied
appropriately.
"""
raise PySparkNotImplementedError
@property
def host(self) -> str:
"""
The hostname where this client intends to connect.
This is used for end-user display purpose in REPL
"""
raise PySparkNotImplementedError
def _insecure_channel(self, target: Any, **kwargs: Any) -> grpc.Channel:
channel = grpc.insecure_channel(target, options=self._channel_options, **kwargs)
if len(self._interceptors) > 0:
logger.debug(f"Applying interceptors ({self._interceptors})")
channel = grpc.intercept_channel(channel, *self._interceptors)
return channel
def _secure_channel(self, target: Any, credentials: Any, **kwargs: Any) -> grpc.Channel:
channel = grpc.secure_channel(target, credentials, options=self._channel_options, **kwargs)
if len(self._interceptors) > 0:
logger.debug(f"Applying interceptors ({self._interceptors})")
channel = grpc.intercept_channel(channel, *self._interceptors)
return channel
@property
def userId(self) -> Optional[str]:
"""
Returns
-------
The user_id (extracted from connection string or configured by other means).
"""
return self._params.get(ChannelBuilder.PARAM_USER_ID, None)
@property
def token(self) -> Optional[str]:
return self._params.get(
ChannelBuilder.PARAM_TOKEN, os.environ.get("SPARK_CONNECT_AUTHENTICATE_TOKEN")
)
def metadata(self) -> Iterable[Tuple[str, str]]:
"""
Builds the GRPC specific metadata list to be injected into the request. All
parameters will be converted to metadata except ones that are explicitly used
by the channel.
Returns
-------
A list of tuples (key, value)
"""
return [
(k, self._params[k])
for k in self._params
if k
not in [
ChannelBuilder.PARAM_TOKEN,
ChannelBuilder.PARAM_USE_SSL,
ChannelBuilder.PARAM_USER_ID,
ChannelBuilder.PARAM_USER_AGENT,
ChannelBuilder.PARAM_SESSION_ID,
]
]
@property
def session_id(self) -> Optional[str]:
"""
Returns
-------
The session_id extracted from the parameters of the connection string or `None` if not
specified.
"""
session_id = self._params.get(ChannelBuilder.PARAM_SESSION_ID, None)
if session_id is not None:
try:
uuid.UUID(session_id, version=4)
except ValueError as ve:
raise PySparkValueError(
errorClass="INVALID_SESSION_UUID_ID",
messageParameters={"arg_name": "session_id", "origin": str(ve)},
)
return session_id
@property
def userAgent(self) -> str:
"""
Returns
-------
user_agent : str
The user_agent parameter specified in the connection string,
or "_SPARK_CONNECT_PYTHON" when not specified.
The returned value will be percent encoded.
"""
user_agent = self._params.get(
ChannelBuilder.PARAM_USER_AGENT,
os.getenv("SPARK_CONNECT_USER_AGENT", "_SPARK_CONNECT_PYTHON"),
)
ua_len = len(urllib.parse.quote(user_agent))
if ua_len > 2048:
raise SparkConnectException(
f"'user_agent' parameter should not exceed 2048 characters after URL "
f"escaping, found {ua_len} characters."
)
return " ".join(
[
user_agent,
f"spark/{__version__}",
f"os/{platform.uname().system.lower()}",
f"python/{platform.python_version()}",
]
)
class DefaultChannelBuilder(ChannelBuilder):
"""
This is a helper class that is used to create a GRPC channel based on the given
connection string per the documentation of Spark Connect.
.. versionadded:: 3.4.0
Examples
--------
>>> cb = DefaultChannelBuilder("sc://localhost")
... cb.endpoint
"localhost:15002"
>>> cb = DefaultChannelBuilder("sc://localhost/;use_ssl=true;token=aaa")
... cb.secure
True
"""
@staticmethod
def default_port() -> int:
if "SPARK_TESTING" in os.environ and not is_remote_only():
from pyspark.sql.session import SparkSession as PySparkSession
# In the case when Spark Connect uses the local mode, it starts the regular Spark
# session that starts Spark Connect server that sets `SparkSession._instantiatedSession`
# via SparkSession.__init__.
#
# We are getting the actual server port from the Spark session via Py4J to address
# the case when the server port is set to 0 (in which allocates an ephemeral port).
#
# This is only used in the test/development mode.
session = PySparkSession._instantiatedSession
if session is not None:
jvm = PySparkSession._instantiatedSession._jvm # type: ignore[union-attr]
return getattr(
getattr(
jvm,
"org.apache.spark.sql.connect.service.SparkConnectService$",
),
"MODULE$",
).localPort()
return 15002
def __init__(self, url: str, channelOptions: Optional[List[Tuple[str, Any]]] = None) -> None:
"""
Constructs a new channel builder. This is used to create the proper GRPC channel from
the connection string.
Parameters
----------
url : str
Spark Connect connection string
channelOptions: list of tuple, optional
Additional options that can be passed to the GRPC channel construction.
"""
super().__init__(channelOptions=channelOptions)
# Explicitly check the scheme of the URL.
if url[:5] != "sc://":
raise PySparkValueError(
errorClass="INVALID_CONNECT_URL",
messageParameters={
"detail": "The URL must start with 'sc://'. Please update the URL to "
"follow the correct format, e.g., 'sc://hostname:port'.",
},
)
# Rewrite the URL to use http as the scheme so that we can leverage
# Python's built-in parser.
tmp_url = "http" + url[2:]
self.url = urllib.parse.urlparse(tmp_url)
if len(self.url.path) > 0 and self.url.path != "/":
raise PySparkValueError(
errorClass="INVALID_CONNECT_URL",
messageParameters={
"detail": f"The path component '{self.url.path}' must be empty. Please update "
f"the URL to follow the correct format, e.g., 'sc://hostname:port'.",
},
)
self._extract_attributes()
def _extract_attributes(self) -> None:
if len(self.url.params) > 0:
parts = self.url.params.split(";")
for p in parts:
kv = p.split("=")
if len(kv) != 2:
raise PySparkValueError(
errorClass="INVALID_CONNECT_URL",
messageParameters={
"detail": f"Parameter '{p}' should be provided as a "
f"key-value pair separated by an equal sign (=). Please update "
f"the parameter to follow the correct format, e.g., 'key=value'.",
},
)
self.set(kv[0], urllib.parse.unquote(kv[1]))
if not self.url.hostname:
raise PySparkValueError(
errorClass="INVALID_CONNECT_URL",
messageParameters={
"detail": f"Hostname is missing in the URL: '{self.url.geturl()}'. "
"Please update the URL to follow the correct format, "
"e.g., 'sc://hostname:port'.",
},
)
self._host = f"[{self.url.hostname}]" if ":" in self.url.hostname else self.url.hostname
self._port = (
self.url.port if self.url.port is not None else DefaultChannelBuilder.default_port()
)
@property
def secure(self) -> bool:
return self.use_ssl or self.token is not None
@property
def use_ssl(self) -> bool:
return self.getDefault(ChannelBuilder.PARAM_USE_SSL, "").lower() == "true"
@property
def host(self) -> str:
"""
The hostname where this client intends to connect.
"""
return self._host
@property
def endpoint(self) -> str:
return f"{self._host}:{self._port}"
def toChannel(self) -> grpc.Channel:
"""
Applies the parameters of the connection string and creates a new
GRPC channel according to the configuration. Passes optional channel options to
construct the channel.
Returns
-------
GRPC Channel instance.
"""
if not self.secure:
return self._insecure_channel(self.endpoint)
elif not self.use_ssl and self._host == "localhost":
creds = grpc.local_channel_credentials()
if self.token is not None:
creds = grpc.composite_channel_credentials(
creds, grpc.access_token_call_credentials(self.token)
)
return self._secure_channel(self.endpoint, creds)
else:
creds = grpc.ssl_channel_credentials()
if self.token is not None:
creds = grpc.composite_channel_credentials(
creds, grpc.access_token_call_credentials(self.token)
)
return self._secure_channel(self.endpoint, creds)
class PlanObservedMetrics(ObservedMetrics):
def __init__(self, name: str, metrics: List[pb2.Expression.Literal], keys: List[str]):
self._name = name
self._metrics = metrics
self._keys = keys if keys else [f"observed_metric_{i}" for i in range(len(self.metrics))]
def __repr__(self) -> str:
return f"Plan observed({self._name}={self._metrics})"
@property
def name(self) -> str:
return self._name
@property
def metrics(self) -> List[pb2.Expression.Literal]:
return self._metrics
@property
def pairs(self) -> dict[str, Any]:
result = {}
for x in range(len(self._metrics)):
result[self.keys[x]] = LiteralExpression._to_value(self.metrics[x])
return result
@property
def keys(self) -> List[str]:
return self._keys
def to_dict(self) -> dict[str, Any]:
"""Return a JSON-serializable dictionary representation of this observed metrics.
Returns
-------
dict
A dictionary with keys 'name', 'keys', and 'pairs'.
"""
return {
"name": self._name,
"keys": self._keys,
"pairs": self.pairs,
}
class AnalyzeResult:
def __init__(
self,
schema: Optional[DataType],
explain_string: Optional[str],
tree_string: Optional[str],
is_local: Optional[bool],
is_streaming: Optional[bool],
input_files: Optional[List[str]],
spark_version: Optional[str],
parsed: Optional[DataType],
is_same_semantics: Optional[bool],
semantic_hash: Optional[int],
storage_level: Optional[StorageLevel],
ddl_string: Optional[str],
):
self.schema = schema
self.explain_string = explain_string
self.tree_string = tree_string
self.is_local = is_local
self.is_streaming = is_streaming
self.input_files = input_files
self.spark_version = spark_version
self.parsed = parsed
self.is_same_semantics = is_same_semantics
self.semantic_hash = semantic_hash
self.storage_level = storage_level
self.ddl_string = ddl_string
@classmethod
def fromProto(cls, pb: Any) -> "AnalyzeResult":
schema: Optional[DataType] = None
explain_string: Optional[str] = None
tree_string: Optional[str] = None
is_local: Optional[bool] = None
is_streaming: Optional[bool] = None
input_files: Optional[List[str]] = None
spark_version: Optional[str] = None
parsed: Optional[DataType] = None
is_same_semantics: Optional[bool] = None
semantic_hash: Optional[int] = None
storage_level: Optional[StorageLevel] = None
ddl_string: Optional[str] = None
if pb.HasField("schema"):
schema = types.proto_schema_to_pyspark_data_type(pb.schema.schema)
elif pb.HasField("explain"):
explain_string = pb.explain.explain_string
elif pb.HasField("tree_string"):
tree_string = pb.tree_string.tree_string
elif pb.HasField("is_local"):
is_local = pb.is_local.is_local
elif pb.HasField("is_streaming"):
is_streaming = pb.is_streaming.is_streaming
elif pb.HasField("input_files"):
input_files = pb.input_files.files
elif pb.HasField("spark_version"):
spark_version = pb.spark_version.version
elif pb.HasField("ddl_parse"):
parsed = types.proto_schema_to_pyspark_data_type(pb.ddl_parse.parsed)
elif pb.HasField("same_semantics"):
is_same_semantics = pb.same_semantics.result
elif pb.HasField("semantic_hash"):
semantic_hash = pb.semantic_hash.result
elif pb.HasField("persist"):
pass
elif pb.HasField("unpersist"):
pass
elif pb.HasField("get_storage_level"):
storage_level = proto_to_storage_level(pb.get_storage_level.storage_level)
elif pb.HasField("json_to_ddl"):
ddl_string = pb.json_to_ddl.ddl_string
else:
raise SparkConnectException("No analyze result found!")
return AnalyzeResult(
schema,
explain_string,
tree_string,
is_local,
is_streaming,
input_files,
spark_version,
parsed,
is_same_semantics,
semantic_hash,
storage_level,
ddl_string,
)
class ConfigResult:
def __init__(self, pairs: List[Tuple[str, Optional[str]]], warnings: List[str]):
self.pairs = pairs
self.warnings = warnings
@classmethod
def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult":
return ConfigResult(
pairs=[(pair.key, pair.value if pair.HasField("value") else None) for pair in pb.pairs],
warnings=list(pb.warnings),
)
def _is_pyspark_source(filename: str) -> bool:
"""Check if the given filename is from the pyspark package."""
return filename.startswith(PYSPARK_ROOT)
class SparkConnectClient(object):
"""
Conceptually the remote spark session that communicates with the server
"""
def __init__(
self,
connection: Union[str, ChannelBuilder],
user_id: Optional[str] = None,
channel_options: Optional[List[Tuple[str, Any]]] = None,
retry_policy: Optional[Dict[str, Any]] = None,
use_reattachable_execute: bool = True,
session_hooks: Optional[list["SparkSession.Hook"]] = None,
allow_arrow_batch_chunking: bool = True,
preferred_arrow_chunk_size: Optional[int] = None,
):
"""
Creates a new SparkSession for the Spark Connect interface.
Parameters
----------
connection : str or :class:`ChannelBuilder`
Connection string that is used to extract the connection parameters and configure
the GRPC connection. Or instance of ChannelBuilder that creates GRPC connection.
Defaults to `sc://localhost`.
user_id : str, optional
Optional unique user ID that is used to differentiate multiple users and
isolate their Spark Sessions. If the `user_id` is not set, will default to
the $USER environment. Defining the user ID as part of the connection string
takes precedence.
channel_options: list of tuple, optional
Additional options that can be passed to the GRPC channel construction.
retry_policy: dict of str and any, optional
Additional configuration for retrying. There are four configurations as below
* ``max_retries``
Maximum number of tries default 15
* ``backoff_multiplier``
Backoff multiplier for the policy. Default: 4(ms)
* ``initial_backoff``
Backoff to wait before the first retry. Default: 50(ms)
* ``max_backoff``
Maximum backoff controls the maximum amount of time to wait before retrying
a failed request. Default: 60000(ms).
use_reattachable_execute: bool
Enable reattachable execution.
session_hooks: list[SparkSession.Hook], optional
List of session hooks to call.
allow_arrow_batch_chunking: bool
Whether to allow the server to split large Arrow batches into smaller chunks.
Although Arrow results are split into batches with a size limit according to estimation,
the size of the batches is not guaranteed to be less than the limit, especially when a
single row is larger than the limit, in which case the server will fail to split it
further into smaller batches. As a result, the client may encounter a gRPC error stating
"Received message larger than max" when a batch is too large.
If true, the server will split large Arrow batches into smaller chunks, and the client
is expected to handle the chunked Arrow batches.
If false, the server will not chunk large Arrow batches.
preferred_arrow_chunk_size: Optional[int]
Optional preferred Arrow batch size in bytes for the server to use when sending Arrow
results.
The server will attempt to use this size if it is set and within the valid range
([1KB, max batch size on server]). Otherwise, the server's maximum batch size is used.
"""
self.thread_local = threading.local()
# Parse the connection string.
self._builder = (
connection
if isinstance(connection, ChannelBuilder)
else DefaultChannelBuilder(connection, channel_options)
)
self._user_id = None
self._retry_policies: List[RetryPolicy] = []
retry_policy_args = retry_policy or dict()
default_policy = DefaultPolicy(**retry_policy_args)
self.set_retry_policies([default_policy])
if self._builder.session_id is None:
# Generate a unique session ID for this client. This UUID must be unique to allow
# concurrent Spark sessions of the same user. If the channel is closed, creating
# a new client will create a new session ID.
self._session_id = str(uuid.uuid4())
else:
# Use the pre-defined session ID.
self._session_id = str(self._builder.session_id)
if self._builder.userId is not None:
self._user_id = self._builder.userId
elif user_id is not None:
self._user_id = user_id
else:
self._user_id = os.getenv("SPARK_USER", os.getenv("USER", None))
self._channel = self._builder.toChannel()
self._closed = False
self._internal_stub = grpc_lib.SparkConnectServiceStub(self._channel)
self._artifact_manager = ArtifactManager(
self._user_id, self._session_id, self._channel, self._builder.metadata()
)
self._use_reattachable_execute = use_reattachable_execute
self._allow_arrow_batch_chunking = allow_arrow_batch_chunking
self._preferred_arrow_chunk_size = preferred_arrow_chunk_size
self._session_hooks = session_hooks or []
# Configure logging for the SparkConnect client.
# Capture the server-side session ID and set it to None initially. It will
# be updated on the first response received.
self._server_session_id: Optional[str] = None
self._profiler_collector = ConnectProfilerCollector()
self._progress_handlers: List[ProgressHandler] = []
self._execution_info_callbacks: "List[Callable[[str, ExecutionInfo], None]]" = []
self._zstd_module = _import_zstandard_if_available()
self._plan_compression_threshold: Optional[int] = None # Will be fetched lazily
self._plan_compression_algorithm: Optional[str] = None # Will be fetched lazily
self._release_futures: weakref.WeakSet[concurrent.futures.Future] = weakref.WeakSet()
self._release_session_on_exit = os.getenv(
"SPARK_CONNECT_RELEASE_SESSION_ON_EXIT", "false"
).lower() in ("true", "1")
# cleanup if possible
atexit.register(self._on_exit)
self.global_user_context_extensions: List[Tuple[str, any_pb2.Any]] = []
self.global_user_context_extensions_lock = threading.Lock()
@property
def _stub(self) -> grpc_lib.SparkConnectServiceStub:
if self.is_closed:
raise SparkConnectException(
errorClass="NO_ACTIVE_SESSION", messageParameters=dict()
) from None
return self._internal_stub
# For testing only.
@_stub.setter
def _stub(self, value: grpc_lib.SparkConnectServiceStub) -> None:
self._internal_stub = value
def register_progress_handler(self, handler: ProgressHandler) -> None:
"""
Register a progress handler to be called when a progress message is received.
Parameters
----------
handler : ProgressHandler
The callable that will be called with the progress information.
"""
if handler in self._progress_handlers:
return
self._progress_handlers.append(handler)
def clear_progress_handlers(self) -> None:
self._progress_handlers.clear()
def register_execution_info_callback(
self, cb: "Callable[[str, ExecutionInfo], None]"
) -> None:
if cb not in self._execution_info_callbacks:
self._execution_info_callbacks.append(cb)
def remove_execution_info_callback(
self, cb: "Callable[[str, ExecutionInfo], None]"
) -> None:
if cb in self._execution_info_callbacks:
self._execution_info_callbacks.remove(cb)
def _fire_execution_info(self, operation_id: str, ei: ExecutionInfo) -> None:
for cb in self._execution_info_callbacks:
try:
cb(operation_id, ei)
except Exception:
pass
def remove_progress_handler(self, handler: ProgressHandler) -> None:
"""
Remove a progress handler from the list of registered handlers.
Parameters
----------
handler : ProgressHandler
The callable to remove from the list of progress handlers.
"""
self._progress_handlers.remove(handler)
def _retrying(self) -> "Retrying":
return Retrying(self._retry_policies)
def disable_reattachable_execute(self) -> "SparkConnectClient":
self._use_reattachable_execute = False
return self
def enable_reattachable_execute(self) -> "SparkConnectClient":
self._use_reattachable_execute = True
return self
def set_retry_policies(self, policies: Iterable[RetryPolicy]) -> None:
"""
Sets list of policies to be used for retries.
I.e. set_retry_policies([DefaultPolicy(), CustomPolicy()]).
"""
self._retry_policies = list(policies)
def get_retry_policies(self) -> List[RetryPolicy]:
"""
Return list of currently used policies
"""
return list(self._retry_policies)
@classmethod
def _retrieve_stack_frames(cls) -> List[CallSite]:
"""
Return a list of CallSites representing the relevant stack frames in the callstack.
"""
frames = traceback.extract_stack()
filtered_stack_frames = []
for i, frame in enumerate(frames):
filename, lineno, func, _ = frame
if _is_pyspark_source(filename):
# Do not include PySpark internal frames as they are not user application code
break
if i + 1 < len(frames):
_, _, func, _ = frames[i + 1]
filtered_stack_frames.append(CallSite(function=func, file=filename, linenum=lineno))
return filtered_stack_frames
@classmethod
def _build_call_stack_trace(cls) -> Optional[any_pb2.Any]:
"""
Build a call stack trace for the current Spark Connect action
Returns
-------
FetchErrorDetailsResponse.Error: An Error object containing list of stack frames
of the user code packed as Any protobuf
"""
if os.getenv("SPARK_CONNECT_DEBUG_CLIENT_CALL_STACK", "false").lower() in ("true", "1"):
stack_frames = cls._retrieve_stack_frames()
call_stack = FetchErrorDetailsResponse.Error()
for call_site in stack_frames:
stack_trace_element = pb2.FetchErrorDetailsResponse.StackTraceElement()
stack_trace_element.declaring_class = "" # unknown information
stack_trace_element.method_name = call_site.function
stack_trace_element.file_name = call_site.file
stack_trace_element.line_number = call_site.linenum
call_stack.stack_trace.append(stack_trace_element)
if len(call_stack.stack_trace) > 0:
call_stack_details = any_pb2.Any()
call_stack_details.Pack(call_stack)
return call_stack_details
return None
def register_udf(
self,
function: Any,
return_type: "DataTypeOrString",
name: Optional[str] = None,
eval_type: int = PythonEvalType.SQL_BATCHED_UDF,
deterministic: bool = True,
) -> str:
"""
Create a temporary UDF in the session catalog on the other side. We generate a
temporary name for it.
"""
if name is None:
name = f"fun_{uuid.uuid4().hex}"
# construct a PythonUDF
py_udf = PythonUDF(
output_type=return_type,
eval_type=eval_type,
func=function,
python_ver="%d.%d" % sys.version_info[:2],
)
# construct a CommonInlineUserDefinedFunction
fun = CommonInlineUserDefinedFunction(
function_name=name,
arguments=[],
function=py_udf,
deterministic=deterministic,
).to_plan_udf(self)
# construct the request
req = self._execute_plan_request_with_metadata()
req.plan.command.register_function.CopyFrom(fun)
self._execute(req)
return name
def register_udtf(
self,
function: Any,
return_type: Optional["DataTypeOrString"],
name: str,
eval_type: int = PythonEvalType.SQL_TABLE_UDF,
deterministic: bool = True,
) -> str:
"""
Register a user-defined table function (UDTF) in the session catalog
as a temporary function. The return type, if specified, must be a
struct type and it's validated when building the proto message
for the PythonUDTF.
"""
udtf = PythonUDTF(
func=function,
return_type=return_type,
eval_type=eval_type,
python_ver=get_python_ver(),
)
func = CommonInlineUserDefinedTableFunction(
function_name=name,
function=udtf,
deterministic=deterministic,
arguments=[],
).udtf_plan(self)
req = self._execute_plan_request_with_metadata()
req.plan.command.register_table_function.CopyFrom(func)
self._execute(req)
return name
def register_data_source(self, dataSource: Type["DataSource"]) -> None:
"""
Register a data source in the session catalog.
"""
data_source = PythonDataSource(
data_source=dataSource,
python_ver=get_python_ver(),
)
proto = CommonInlineUserDefinedDataSource(
name=dataSource.name(),
data_source=data_source,
).to_data_source_proto(self)
req = self._execute_plan_request_with_metadata()
req.plan.command.register_data_source.CopyFrom(proto)
self._execute(req)
def register_java(
self,
name: str,
javaClassName: str,
return_type: Optional["DataTypeOrString"] = None,
aggregate: bool = False,
) -> None:
# construct a JavaUDF
if return_type is None:
java_udf = JavaUDF(class_name=javaClassName, aggregate=aggregate)
else:
java_udf = JavaUDF(class_name=javaClassName, output_type=return_type)
fun = CommonInlineUserDefinedFunction(
function_name=name,
function=java_udf,