-
Notifications
You must be signed in to change notification settings - Fork 28.5k
/
Copy pathbuiltin.py
22209 lines (18085 loc) · 684 KB
/
builtin.py
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.
#
"""
A collections of builtin functions
"""
import inspect
import decimal
import sys
import functools
import warnings
from typing import (
Any,
cast,
Callable,
Mapping,
Sequence,
Iterable,
overload,
Optional,
Tuple,
Type,
TYPE_CHECKING,
Union,
ValuesView,
)
from pyspark.errors import PySparkTypeError, PySparkValueError
from pyspark.sql.column import Column
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.types import ArrayType, DataType, StringType, StructType, _from_numpy_type
# Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409
from pyspark.sql.udf import UserDefinedFunction, _create_py_udf # noqa: F401
from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult # noqa: F401
from pyspark.sql.udtf import OrderingColumn, PartitioningColumn, SelectedColumn # noqa: F401
from pyspark.sql.udtf import SkipRestOfInputTableException # noqa: F401
from pyspark.sql.udtf import UserDefinedTableFunction, _create_py_udtf
# Keep pandas_udf and PandasUDFType import for backwards compatible import; moved in SPARK-28264
from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType # noqa: F401
from pyspark.sql.utils import (
to_str as _to_str,
has_numpy as _has_numpy,
try_remote_functions as _try_remote_functions,
get_active_spark_context as _get_active_spark_context,
enum_to_value as _enum_to_value,
)
if TYPE_CHECKING:
from pyspark import SparkContext
from pyspark.sql._typing import (
ColumnOrName,
DataTypeOrString,
UserDefinedFunctionLike,
)
if _has_numpy:
import numpy as np
# Note to developers: all of PySpark functions here take string as column names whenever possible.
# Namely, if columns are referred as arguments, they can always be both Column or string,
# even though there might be few exceptions for legacy or inevitable reasons.
# If you are fixing other language APIs together, also please note that Scala side is not the case
# since it requires making every single overridden definition.
def _get_jvm_function(name: str, sc: "SparkContext") -> Callable:
"""
Retrieves JVM function identified by name from
Java gateway associated with sc.
"""
assert sc._jvm is not None
return getattr(getattr(sc._jvm, "org.apache.spark.sql.functions"), name)
def _invoke_function(name: str, *args: Any) -> Column:
"""
Invokes JVM function identified by name with args
and wraps the result with :class:`~pyspark.sql.Column`.
"""
from pyspark import SparkContext
assert SparkContext._active_spark_context is not None
jf = _get_jvm_function(name, SparkContext._active_spark_context)
return Column(jf(*args))
def _invoke_function_over_columns(name: str, *cols: "ColumnOrName") -> Column:
"""
Invokes n-ary JVM function identified by name
and wraps the result with :class:`~pyspark.sql.Column`.
"""
from pyspark.sql.classic.column import _to_java_column
return _invoke_function(name, *(_to_java_column(col) for col in cols))
def _invoke_function_over_seq_of_columns(name: str, cols: "Iterable[ColumnOrName]") -> Column:
"""
Invokes unary JVM function identified by name with
and wraps the result with :class:`~pyspark.sql.Column`.
"""
from pyspark.sql.classic.column import _to_java_column, _to_seq
sc = _get_active_spark_context()
return _invoke_function(name, _to_seq(sc, cols, _to_java_column))
def _invoke_binary_math_function(name: str, col1: Any, col2: Any) -> Column:
"""
Invokes binary JVM math function identified by name
and wraps the result with :class:`~pyspark.sql.Column`.
"""
from pyspark.sql.classic.column import _to_java_column, _create_column_from_literal
# For legacy reasons, the arguments here can be implicitly converted into column
cols = [
_to_java_column(c) if isinstance(c, (str, Column)) else _create_column_from_literal(c)
for c in (col1, col2)
]
return _invoke_function(name, *cols)
def _options_to_str(options: Optional[Mapping[str, Any]] = None) -> Mapping[str, Optional[str]]:
if options:
return {key: _to_str(value) for (key, value) in options.items()}
return {}
@_try_remote_functions
def lit(col: Any) -> Column:
"""
Creates a :class:`~pyspark.sql.Column` of literal value.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : :class:`~pyspark.sql.Column`, str, int, float, bool or list, NumPy literals or ndarray.
the value to make it as a PySpark literal. If a column is passed,
it returns the column as is.
.. versionchanged:: 3.4.0
Since 3.4.0, it supports the list type.
Returns
-------
:class:`~pyspark.sql.Column`
the literal instance.
Examples
--------
Example 1: Creating a literal column with an integer value.
>>> import pyspark.sql.functions as sf
>>> df = spark.range(1)
>>> df.select(sf.lit(5).alias('height'), df.id).show()
+------+---+
|height| id|
+------+---+
| 5| 0|
+------+---+
Example 2: Creating a literal column from a list.
>>> import pyspark.sql.functions as sf
>>> spark.range(1).select(sf.lit([1, 2, 3])).show()
+--------------+
|array(1, 2, 3)|
+--------------+
| [1, 2, 3]|
+--------------+
Example 3: Creating a literal column from a string.
>>> import pyspark.sql.functions as sf
>>> df = spark.range(1)
>>> df.select(sf.lit("PySpark").alias('framework'), df.id).show()
+---------+---+
|framework| id|
+---------+---+
| PySpark| 0|
+---------+---+
Example 4: Creating a literal column from a boolean value.
>>> import pyspark.sql.functions as sf
>>> df = spark.createDataFrame([(True, "Yes"), (False, "No")], ["flag", "response"])
>>> df.select(sf.lit(False).alias('is_approved'), df.response).show()
+-----------+--------+
|is_approved|response|
+-----------+--------+
| false| Yes|
| false| No|
+-----------+--------+
"""
if isinstance(col, Column):
return col
elif isinstance(col, list):
if any(isinstance(c, Column) for c in col):
raise PySparkValueError(
errorClass="COLUMN_IN_LIST", messageParameters={"func_name": "lit"}
)
return array(*[lit(item) for item in col])
else:
if _has_numpy and isinstance(col, np.generic):
dt = _from_numpy_type(col.dtype)
if dt is not None:
return _invoke_function("lit", _enum_to_value(col)).astype(dt).alias(str(col))
return _invoke_function("lit", _enum_to_value(col))
@_try_remote_functions
def col(col: str) -> Column:
"""
Returns a :class:`~pyspark.sql.Column` based on the given column name.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : str
the name for the column
Returns
-------
:class:`~pyspark.sql.Column`
the corresponding column instance.
Examples
--------
>>> col('x')
Column<'x'>
>>> column('x')
Column<'x'>
"""
return _invoke_function("col", col)
column = col
@_try_remote_functions
def asc(col: "ColumnOrName") -> Column:
"""
Returns a sort expression for the target column in ascending order.
This function is used in `sort` and `orderBy` functions.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Target column to sort by in the ascending order.
Returns
-------
:class:`~pyspark.sql.Column`
The column specifying the sort order.
Examples
--------
Example 1: Sort DataFrame by 'id' column in ascending order.
>>> from pyspark.sql.functions import asc
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.sort(asc("id")).show()
+---+-----+
| id|value|
+---+-----+
| 2| C|
| 3| A|
| 4| B|
+---+-----+
Example 2: Use `asc` in `orderBy` function to sort the DataFrame.
>>> from pyspark.sql.functions import asc
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.orderBy(asc("value")).show()
+---+-----+
| id|value|
+---+-----+
| 3| A|
| 4| B|
| 2| C|
+---+-----+
Example 3: Combine `asc` with `desc` to sort by multiple columns.
>>> from pyspark.sql.functions import asc, desc
>>> df = spark.createDataFrame([(2, 'A', 4),
... (1, 'B', 3),
... (3, 'A', 2)], ['id', 'group', 'value'])
>>> df.sort(asc("group"), desc("value")).show()
+---+-----+-----+
| id|group|value|
+---+-----+-----+
| 2| A| 4|
| 3| A| 2|
| 1| B| 3|
+---+-----+-----+
Example 4: Implement `asc` from column expression.
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.sort(df.id.asc()).show()
+---+-----+
| id|value|
+---+-----+
| 2| C|
| 3| A|
| 4| B|
+---+-----+
"""
return col.asc() if isinstance(col, Column) else _invoke_function("asc", col)
@_try_remote_functions
def desc(col: "ColumnOrName") -> Column:
"""
Returns a sort expression for the target column in descending order.
This function is used in `sort` and `orderBy` functions.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Target column to sort by in the descending order.
Returns
-------
:class:`~pyspark.sql.Column`
The column specifying the sort order.
Examples
--------
Example 1: Sort DataFrame by 'id' column in descending order.
>>> from pyspark.sql.functions import desc
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.sort(desc("id")).show()
+---+-----+
| id|value|
+---+-----+
| 4| B|
| 3| A|
| 2| C|
+---+-----+
Example 2: Use `desc` in `orderBy` function to sort the DataFrame.
>>> from pyspark.sql.functions import desc
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.orderBy(desc("value")).show()
+---+-----+
| id|value|
+---+-----+
| 2| C|
| 4| B|
| 3| A|
+---+-----+
Example 3: Combine `asc` with `desc` to sort by multiple columns.
>>> from pyspark.sql.functions import asc, desc
>>> df = spark.createDataFrame([(2, 'A', 4),
... (1, 'B', 3),
... (3, 'A', 2)], ['id', 'group', 'value'])
>>> df.sort(desc("group"), asc("value")).show()
+---+-----+-----+
| id|group|value|
+---+-----+-----+
| 1| B| 3|
| 3| A| 2|
| 2| A| 4|
+---+-----+-----+
Example 4: Implement `desc` from column expression.
>>> df = spark.createDataFrame([(4, 'B'), (3, 'A'), (2, 'C')], ['id', 'value'])
>>> df.sort(df.id.desc()).show()
+---+-----+
| id|value|
+---+-----+
| 4| B|
| 3| A|
| 2| C|
+---+-----+
"""
return col.desc() if isinstance(col, Column) else _invoke_function("desc", col)
@_try_remote_functions
def sqrt(col: "ColumnOrName") -> Column:
"""
Computes the square root of the specified float value.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
target column to compute on.
Returns
-------
:class:`~pyspark.sql.Column`
column for computed results.
Examples
--------
>>> df = spark.range(1)
>>> df.select(sqrt(lit(4))).show()
+-------+
|SQRT(4)|
+-------+
| 2.0|
+-------+
"""
return _invoke_function_over_columns("sqrt", col)
@_try_remote_functions
def try_add(left: "ColumnOrName", right: "ColumnOrName") -> Column:
"""
Returns the sum of `left`and `right` and the result is null on overflow.
The acceptable input types are the same with the `+` operator.
.. versionadded:: 3.5.0
Parameters
----------
left : :class:`~pyspark.sql.Column` or str
right : :class:`~pyspark.sql.Column` or str
Examples
--------
Example 1: Integer plus Integer.
>>> import pyspark.sql.functions as sf
>>> spark.createDataFrame(
... [(1982, 15), (1990, 2)], ["birth", "age"]
... ).select(sf.try_add("birth", "age")).show()
+-------------------+
|try_add(birth, age)|
+-------------------+
| 1997|
| 1992|
+-------------------+
Example 2: Date plus Integer.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (DATE('2015-09-30')) AS TAB(date)"
... ).select(sf.try_add("date", sf.lit(1))).show()
+----------------+
|try_add(date, 1)|
+----------------+
| 2015-10-01|
+----------------+
Example 3: Date plus Interval.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (DATE('2015-09-30'), INTERVAL 1 YEAR) AS TAB(date, i)"
... ).select(sf.try_add("date", "i")).show()
+----------------+
|try_add(date, i)|
+----------------+
| 2016-09-30|
+----------------+
Example 4: Interval plus Interval.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (INTERVAL 1 YEAR, INTERVAL 2 YEAR) AS TAB(i, j)"
... ).select(sf.try_add("i", "j")).show()
+-----------------+
| try_add(i, j)|
+-----------------+
|INTERVAL '3' YEAR|
+-----------------+
Example 5: Overflow results in NULL when ANSI mode is on
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.range(1)
... df.select(sf.try_add(sf.lit(sys.maxsize), sf.lit(sys.maxsize))).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+-------------------------------------------------+
|try_add(9223372036854775807, 9223372036854775807)|
+-------------------------------------------------+
| NULL|
+-------------------------------------------------+
"""
return _invoke_function_over_columns("try_add", left, right)
@_try_remote_functions
def try_avg(col: "ColumnOrName") -> Column:
"""
Returns the mean calculated from values of a group and the result is null on overflow.
.. versionadded:: 3.5.0
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Examples
--------
Example 1: Calculating the average age
>>> import pyspark.sql.functions as sf
>>> df = spark.createDataFrame([(1982, 15), (1990, 2)], ["birth", "age"])
>>> df.select(sf.try_avg("age")).show()
+------------+
|try_avg(age)|
+------------+
| 8.5|
+------------+
Example 2: Calculating the average age with None
>>> import pyspark.sql.functions as sf
>>> df = spark.createDataFrame([(1982, None), (1990, 2), (2000, 4)], ["birth", "age"])
>>> df.select(sf.try_avg("age")).show()
+------------+
|try_avg(age)|
+------------+
| 3.0|
+------------+
Example 3: Overflow results in NULL when ANSI mode is on
>>> from decimal import Decimal
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.createDataFrame(
... [(Decimal("1" * 38),), (Decimal(0),)], "number DECIMAL(38, 0)")
... df.select(sf.try_avg(df.number)).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+---------------+
|try_avg(number)|
+---------------+
| NULL|
+---------------+
"""
return _invoke_function_over_columns("try_avg", col)
@_try_remote_functions
def try_divide(left: "ColumnOrName", right: "ColumnOrName") -> Column:
"""
Returns `dividend`/`divisor`. It always performs floating point division. Its result is
always null if `divisor` is 0.
.. versionadded:: 3.5.0
Parameters
----------
left : :class:`~pyspark.sql.Column` or str
dividend
right : :class:`~pyspark.sql.Column` or str
divisor
Examples
--------
Example 1: Integer divided by Integer.
>>> import pyspark.sql.functions as sf
>>> spark.createDataFrame(
... [(6000, 15), (1990, 2), (1234, 0)], ["a", "b"]
... ).select(sf.try_divide("a", "b")).show()
+----------------+
|try_divide(a, b)|
+----------------+
| 400.0|
| 995.0|
| NULL|
+----------------+
Example 2: Interval divided by Integer.
>>> import pyspark.sql.functions as sf
>>> spark.range(4).select(
... sf.try_divide(sf.make_interval(sf.lit(1)), "id")
... ).show()
+--------------------------------------------------+
|try_divide(make_interval(1, 0, 0, 0, 0, 0, 0), id)|
+--------------------------------------------------+
| NULL|
| 1 years|
| 6 months|
| 4 months|
+--------------------------------------------------+
Example 3: Exception during division, resulting in NULL when ANSI mode is on
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.range(1)
... df.select(sf.try_divide(df.id, sf.lit(0))).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+-----------------+
|try_divide(id, 0)|
+-----------------+
| NULL|
+-----------------+
"""
return _invoke_function_over_columns("try_divide", left, right)
@_try_remote_functions
def try_mod(left: "ColumnOrName", right: "ColumnOrName") -> Column:
"""
Returns the remainder after `dividend`/`divisor`. Its result is
always null if `divisor` is 0.
.. versionadded:: 4.0.0
Parameters
----------
left : :class:`~pyspark.sql.Column` or str
dividend
right : :class:`~pyspark.sql.Column` or str
divisor
Examples
--------
Example 1: Integer divided by Integer.
>>> import pyspark.sql.functions as sf
>>> spark.createDataFrame(
... [(6000, 15), (3, 2), (1234, 0)], ["a", "b"]
... ).select(sf.try_mod("a", "b")).show()
+-------------+
|try_mod(a, b)|
+-------------+
| 0|
| 1|
| NULL|
+-------------+
Example 2: Exception during division, resulting in NULL when ANSI mode is on
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.range(1)
... df.select(sf.try_mod(df.id, sf.lit(0))).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+--------------+
|try_mod(id, 0)|
+--------------+
| NULL|
+--------------+
"""
return _invoke_function_over_columns("try_mod", left, right)
@_try_remote_functions
def try_multiply(left: "ColumnOrName", right: "ColumnOrName") -> Column:
"""
Returns `left`*`right` and the result is null on overflow. The acceptable input types are the
same with the `*` operator.
.. versionadded:: 3.5.0
Parameters
----------
left : :class:`~pyspark.sql.Column` or str
multiplicand
right : :class:`~pyspark.sql.Column` or str
multiplier
Examples
--------
Example 1: Integer multiplied by Integer.
>>> import pyspark.sql.functions as sf
>>> spark.createDataFrame(
... [(6000, 15), (1990, 2)], ["a", "b"]
... ).select(sf.try_multiply("a", "b")).show()
+------------------+
|try_multiply(a, b)|
+------------------+
| 90000|
| 3980|
+------------------+
Example 2: Interval multiplied by Integer.
>>> import pyspark.sql.functions as sf
>>> spark.range(6).select(
... sf.try_multiply(sf.make_interval(sf.lit(0), sf.lit(3)), "id")
... ).show()
+----------------------------------------------------+
|try_multiply(make_interval(0, 3, 0, 0, 0, 0, 0), id)|
+----------------------------------------------------+
| 0 seconds|
| 3 months|
| 6 months|
| 9 months|
| 1 years|
| 1 years 3 months|
+----------------------------------------------------+
Example 3: Overflow results in NULL when ANSI mode is on
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.range(1)
... df.select(sf.try_multiply(sf.lit(sys.maxsize), sf.lit(sys.maxsize))).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+------------------------------------------------------+
|try_multiply(9223372036854775807, 9223372036854775807)|
+------------------------------------------------------+
| NULL|
+------------------------------------------------------+
"""
return _invoke_function_over_columns("try_multiply", left, right)
@_try_remote_functions
def try_subtract(left: "ColumnOrName", right: "ColumnOrName") -> Column:
"""
Returns `left`-`right` and the result is null on overflow. The acceptable input types are the
same with the `-` operator.
.. versionadded:: 3.5.0
Parameters
----------
left : :class:`~pyspark.sql.Column` or str
right : :class:`~pyspark.sql.Column` or str
Examples
--------
Example 1: Integer minus Integer.
>>> import pyspark.sql.functions as sf
>>> spark.createDataFrame(
... [(1982, 15), (1990, 2)], ["birth", "age"]
... ).select(sf.try_subtract("birth", "age")).show()
+------------------------+
|try_subtract(birth, age)|
+------------------------+
| 1967|
| 1988|
+------------------------+
Example 2: Date minus Integer.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (DATE('2015-10-01')) AS TAB(date)"
... ).select(sf.try_subtract("date", sf.lit(1))).show()
+---------------------+
|try_subtract(date, 1)|
+---------------------+
| 2015-09-30|
+---------------------+
Example 3: Date minus Interval.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (DATE('2015-09-30'), INTERVAL 1 YEAR) AS TAB(date, i)"
... ).select(sf.try_subtract("date", "i")).show()
+---------------------+
|try_subtract(date, i)|
+---------------------+
| 2014-09-30|
+---------------------+
Example 4: Interval minus Interval.
>>> import pyspark.sql.functions as sf
>>> spark.sql(
... "SELECT * FROM VALUES (INTERVAL 1 YEAR, INTERVAL 2 YEAR) AS TAB(i, j)"
... ).select(sf.try_subtract("i", "j")).show()
+------------------+
|try_subtract(i, j)|
+------------------+
|INTERVAL '-1' YEAR|
+------------------+
Example 5: Overflow results in NULL when ANSI mode is on
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.range(1)
... df.select(sf.try_subtract(sf.lit(-sys.maxsize), sf.lit(sys.maxsize))).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+-------------------------------------------------------+
|try_subtract(-9223372036854775807, 9223372036854775807)|
+-------------------------------------------------------+
| NULL|
+-------------------------------------------------------+
"""
return _invoke_function_over_columns("try_subtract", left, right)
@_try_remote_functions
def try_sum(col: "ColumnOrName") -> Column:
"""
Returns the sum calculated from values of a group and the result is null on overflow.
.. versionadded:: 3.5.0
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
Examples
--------
Example 1: Calculating the sum of values in a column
>>> from pyspark.sql import functions as sf
>>> df = spark.range(10)
>>> df.select(sf.try_sum(df["id"])).show()
+-----------+
|try_sum(id)|
+-----------+
| 45|
+-----------+
Example 2: Using a plus expression together to calculate the sum
>>> from pyspark.sql import functions as sf
>>> df = spark.createDataFrame([(1, 2), (3, 4)], ["A", "B"])
>>> df.select(sf.try_sum(sf.col("A") + sf.col("B"))).show()
+----------------+
|try_sum((A + B))|
+----------------+
| 10|
+----------------+
Example 3: Calculating the summation of ages with None
>>> import pyspark.sql.functions as sf
>>> df = spark.createDataFrame([(1982, None), (1990, 2), (2000, 4)], ["birth", "age"])
>>> df.select(sf.try_sum("age")).show()
+------------+
|try_sum(age)|
+------------+
| 6|
+------------+
Example 4: Overflow results in NULL when ANSI mode is on
>>> from decimal import Decimal
>>> import pyspark.sql.functions as sf
>>> origin = spark.conf.get("spark.sql.ansi.enabled")
>>> spark.conf.set("spark.sql.ansi.enabled", "true")
>>> try:
... df = spark.createDataFrame([(Decimal("1" * 38),)] * 10, "number DECIMAL(38, 0)")
... df.select(sf.try_sum(df.number)).show()
... finally:
... spark.conf.set("spark.sql.ansi.enabled", origin)
+---------------+
|try_sum(number)|
+---------------+
| NULL|
+---------------+
"""
return _invoke_function_over_columns("try_sum", col)
@_try_remote_functions
def abs(col: "ColumnOrName") -> Column:
"""
Mathematical Function: Computes the absolute value of the given column or expression.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
col : :class:`~pyspark.sql.Column` or str
The target column or expression to compute the absolute value on.
Returns
-------
:class:`~pyspark.sql.Column`
A new column object representing the absolute value of the input.
Examples
--------
Example 1: Compute the absolute value of a negative number
>>> from pyspark.sql import functions as sf
>>> df = spark.createDataFrame([(1, -1), (2, -2), (3, -3)], ["id", "value"])
>>> df.select(sf.abs(df.value)).show()
+----------+
|abs(value)|
+----------+
| 1|
| 2|
| 3|
+----------+
Example 2: Compute the absolute value of an expression
>>> from pyspark.sql import functions as sf
>>> df = spark.createDataFrame([(1, 1), (2, -2), (3, 3)], ["id", "value"])
>>> df.select(sf.abs(df.id - df.value)).show()
+-----------------+
|abs((id - value))|
+-----------------+
| 0|
| 4|
| 0|
+-----------------+
Example 3: Compute the absolute value of a column with null values
>>> from pyspark.sql import functions as sf
>>> df = spark.createDataFrame([(1, None), (2, -2), (3, None)], ["id", "value"])
>>> df.select(sf.abs(df.value)).show()
+----------+
|abs(value)|
+----------+
| NULL|
| 2|
| NULL|
+----------+
Example 4: Compute the absolute value of a column with double values
>>> from pyspark.sql import functions as sf
>>> df = spark.createDataFrame([(1, -1.5), (2, -2.5), (3, -3.5)], ["id", "value"])
>>> df.select(sf.abs(df.value)).show()
+----------+
|abs(value)|
+----------+
| 1.5|
| 2.5|
| 3.5|
+----------+
"""
return _invoke_function_over_columns("abs", col)