-
Notifications
You must be signed in to change notification settings - Fork 28.2k
/
functions.py
3566 lines (2867 loc) · 135 KB
/
functions.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 sys
import functools
import warnings
from pyspark import since, SparkContext
from pyspark.rdd import PythonEvalType
from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal, \
_create_column_from_name
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.types import StringType, DataType
# Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409
from pyspark.sql.udf import UserDefinedFunction, _create_udf # noqa: F401
from pyspark.sql.udf import _create_udf
# 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
# Note to developers: all of PySpark functions here take string as column names whenever possible.
# Namely, if columns are referred as arguments, they can be always 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 to make every single overridden definition.
def _create_function(name, doc=""):
"""Create a PySpark function by its name"""
def _(col):
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
def _create_function_over_column(name, doc=""):
"""Similar with `_create_function` but creates a PySpark function that takes a column
(as string as well). This is mainly for PySpark functions to take strings as
column names.
"""
def _(col):
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)(_to_java_column(col))
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
def _wrap_deprecated_function(func, message):
""" Wrap the deprecated function to print out deprecation warnings"""
def _(col):
warnings.warn(message, DeprecationWarning)
return func(col)
return functools.wraps(func)(_)
def _create_binary_mathfunction(name, doc=""):
""" Create a binary mathfunction by name"""
def _(col1, col2):
sc = SparkContext._active_spark_context
# For legacy reasons, the arguments here can be implicitly converted into floats,
# if they are not columns or strings.
if isinstance(col1, Column):
arg1 = col1._jc
elif isinstance(col1, str):
arg1 = _create_column_from_name(col1)
else:
arg1 = float(col1)
if isinstance(col2, Column):
arg2 = col2._jc
elif isinstance(col2, str):
arg2 = _create_column_from_name(col2)
else:
arg2 = float(col2)
jc = getattr(sc._jvm.functions, name)(arg1, arg2)
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
def _create_window_function(name, doc=''):
""" Create a window function by name """
def _():
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)()
return Column(jc)
_.__name__ = name
_.__doc__ = 'Window function: ' + doc
return _
def _options_to_str(options):
return {key: to_str(value) for (key, value) in options.items()}
_lit_doc = """
Creates a :class:`Column` of literal value.
>>> df.select(lit(5).alias('height')).withColumn('spark_user', lit(True)).take(1)
[Row(height=5, spark_user=True)]
"""
_functions = {
'lit': _lit_doc,
'col': 'Returns a :class:`Column` based on the given column name.',
'column': 'Returns a :class:`Column` based on the given column name.',
'asc': 'Returns a sort expression based on the ascending order of the given column name.',
'desc': 'Returns a sort expression based on the descending order of the given column name.',
}
_functions_over_column = {
'sqrt': 'Computes the square root of the specified float value.',
'abs': 'Computes the absolute value.',
'max': 'Aggregate function: returns the maximum value of the expression in a group.',
'min': 'Aggregate function: returns the minimum value of the expression in a group.',
'count': 'Aggregate function: returns the number of items in a group.',
'sum': 'Aggregate function: returns the sum of all values in the expression.',
'avg': 'Aggregate function: returns the average of the values in a group.',
'mean': 'Aggregate function: returns the average of the values in a group.',
'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.',
}
_functions_1_4_over_column = {
# unary math functions
'acos': ':return: inverse cosine of `col`, as if computed by `java.lang.Math.acos()`',
'asin': ':return: inverse sine of `col`, as if computed by `java.lang.Math.asin()`',
'atan': ':return: inverse tangent of `col`, as if computed by `java.lang.Math.atan()`',
'cbrt': 'Computes the cube-root of the given value.',
'ceil': 'Computes the ceiling of the given value.',
'cos': """:param col: angle in radians
:return: cosine of the angle, as if computed by `java.lang.Math.cos()`.""",
'cosh': """:param col: hyperbolic angle
:return: hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh()`""",
'exp': 'Computes the exponential of the given value.',
'expm1': 'Computes the exponential of the given value minus one.',
'floor': 'Computes the floor of the given value.',
'log': 'Computes the natural logarithm of the given value.',
'log10': 'Computes the logarithm of the given value in Base 10.',
'log1p': 'Computes the natural logarithm of the given value plus one.',
'rint': 'Returns the double value that is closest in value to the argument and' +
' is equal to a mathematical integer.',
'signum': 'Computes the signum of the given value.',
'sin': """:param col: angle in radians
:return: sine of the angle, as if computed by `java.lang.Math.sin()`""",
'sinh': """:param col: hyperbolic angle
:return: hyperbolic sine of the given value,
as if computed by `java.lang.Math.sinh()`""",
'tan': """:param col: angle in radians
:return: tangent of the given value, as if computed by `java.lang.Math.tan()`""",
'tanh': """:param col: hyperbolic angle
:return: hyperbolic tangent of the given value,
as if computed by `java.lang.Math.tanh()`""",
'toDegrees': '.. note:: Deprecated in 2.1, use :func:`degrees` instead.',
'toRadians': '.. note:: Deprecated in 2.1, use :func:`radians` instead.',
'bitwiseNOT': 'Computes bitwise not.',
}
_functions_2_4 = {
'asc_nulls_first': 'Returns a sort expression based on the ascending order of the given' +
' column name, and null values return before non-null values.',
'asc_nulls_last': 'Returns a sort expression based on the ascending order of the given' +
' column name, and null values appear after non-null values.',
'desc_nulls_first': 'Returns a sort expression based on the descending order of the given' +
' column name, and null values appear before non-null values.',
'desc_nulls_last': 'Returns a sort expression based on the descending order of the given' +
' column name, and null values appear after non-null values',
}
_collect_list_doc = """
Aggregate function: returns a list of objects with duplicates.
.. note:: The function is non-deterministic because the order of collected results depends
on the order of the rows which may be non-deterministic after a shuffle.
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
>>> df2.agg(collect_list('age')).collect()
[Row(collect_list(age)=[2, 5, 5])]
"""
_collect_set_doc = """
Aggregate function: returns a set of objects with duplicate elements eliminated.
.. note:: The function is non-deterministic because the order of collected results depends
on the order of the rows which may be non-deterministic after a shuffle.
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
>>> df2.agg(collect_set('age')).collect()
[Row(collect_set(age)=[5, 2])]
"""
_functions_1_6_over_column = {
# unary math functions
'stddev': 'Aggregate function: alias for stddev_samp.',
'stddev_samp': 'Aggregate function: returns the unbiased sample standard deviation of' +
' the expression in a group.',
'stddev_pop': 'Aggregate function: returns population standard deviation of' +
' the expression in a group.',
'variance': 'Aggregate function: alias for var_samp.',
'var_samp': 'Aggregate function: returns the unbiased sample variance of' +
' the values in a group.',
'var_pop': 'Aggregate function: returns the population variance of the values in a group.',
'skewness': 'Aggregate function: returns the skewness of the values in a group.',
'kurtosis': 'Aggregate function: returns the kurtosis of the values in a group.',
'collect_list': _collect_list_doc,
'collect_set': _collect_set_doc
}
_functions_2_1_over_column = {
# unary math functions
'degrees': """
Converts an angle measured in radians to an approximately equivalent angle
measured in degrees.
:param col: angle in radians
:return: angle in degrees, as if computed by `java.lang.Math.toDegrees()`
""",
'radians': """
Converts an angle measured in degrees to an approximately equivalent angle
measured in radians.
:param col: angle in degrees
:return: angle in radians, as if computed by `java.lang.Math.toRadians()`
""",
}
# math functions that take two arguments as input
_binary_mathfunctions = {
'atan2': """
:param col1: coordinate on y-axis
:param col2: coordinate on x-axis
:return: the `theta` component of the point
(`r`, `theta`)
in polar coordinates that corresponds to the point
(`x`, `y`) in Cartesian coordinates,
as if computed by `java.lang.Math.atan2()`
""",
'hypot': 'Computes ``sqrt(a^2 + b^2)`` without intermediate overflow or underflow.',
'pow': 'Returns the value of the first argument raised to the power of the second argument.',
}
_window_functions = {
'row_number':
"""returns a sequential number starting at 1 within a window partition.""",
'dense_rank':
"""returns the rank of rows within a window partition, without any gaps.
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
sequence when there are ties. That is, if you were ranking a competition using dense_rank
and had three people tie for second place, you would say that all three were in second
place and that the next person came in third. Rank would give me sequential numbers, making
the person that came in third place (after the ties) would register as coming in fifth.
This is equivalent to the DENSE_RANK function in SQL.""",
'rank':
"""returns the rank of rows within a window partition.
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
sequence when there are ties. That is, if you were ranking a competition using dense_rank
and had three people tie for second place, you would say that all three were in second
place and that the next person came in third. Rank would give me sequential numbers, making
the person that came in third place (after the ties) would register as coming in fifth.
This is equivalent to the RANK function in SQL.""",
'cume_dist':
"""returns the cumulative distribution of values within a window partition,
i.e. the fraction of rows that are below the current row.""",
'percent_rank':
"""returns the relative rank (i.e. percentile) of rows within a window partition.""",
}
# Wraps deprecated functions (keys) with the messages (values).
_functions_deprecated = {
'toDegrees': 'Deprecated in 2.1, use degrees instead.',
'toRadians': 'Deprecated in 2.1, use radians instead.',
}
for _name, _doc in _functions.items():
globals()[_name] = since(1.3)(_create_function(_name, _doc))
for _name, _doc in _functions_over_column.items():
globals()[_name] = since(1.3)(_create_function_over_column(_name, _doc))
for _name, _doc in _functions_1_4_over_column.items():
globals()[_name] = since(1.4)(_create_function_over_column(_name, _doc))
for _name, _doc in _binary_mathfunctions.items():
globals()[_name] = since(1.4)(_create_binary_mathfunction(_name, _doc))
for _name, _doc in _window_functions.items():
globals()[_name] = since(1.6)(_create_window_function(_name, _doc))
for _name, _doc in _functions_1_6_over_column.items():
globals()[_name] = since(1.6)(_create_function_over_column(_name, _doc))
for _name, _doc in _functions_2_1_over_column.items():
globals()[_name] = since(2.1)(_create_function_over_column(_name, _doc))
for _name, _message in _functions_deprecated.items():
globals()[_name] = _wrap_deprecated_function(globals()[_name], _message)
for _name, _doc in _functions_2_4.items():
globals()[_name] = since(2.4)(_create_function(_name, _doc))
del _name, _doc
@since(1.3)
def approxCountDistinct(col, rsd=None):
"""
.. note:: Deprecated in 2.1, use :func:`approx_count_distinct` instead.
"""
warnings.warn("Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning)
return approx_count_distinct(col, rsd)
@since(2.1)
def approx_count_distinct(col, rsd=None):
"""Aggregate function: returns a new :class:`Column` for approximate distinct count of
column `col`.
:param rsd: maximum relative standard deviation allowed (default = 0.05).
For rsd < 0.01, it is more efficient to use :func:`countDistinct`
>>> df.agg(approx_count_distinct(df.age).alias('distinct_ages')).collect()
[Row(distinct_ages=2)]
"""
sc = SparkContext._active_spark_context
if rsd is None:
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col))
else:
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col), rsd)
return Column(jc)
@since(1.6)
def broadcast(df):
"""Marks a DataFrame as small enough for use in broadcast joins."""
sc = SparkContext._active_spark_context
return DataFrame(sc._jvm.functions.broadcast(df._jdf), df.sql_ctx)
@since(1.4)
def coalesce(*cols):
"""Returns the first column that is not null.
>>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b"))
>>> cDf.show()
+----+----+
| a| b|
+----+----+
|null|null|
| 1|null|
|null| 2|
+----+----+
>>> cDf.select(coalesce(cDf["a"], cDf["b"])).show()
+--------------+
|coalesce(a, b)|
+--------------+
| null|
| 1|
| 2|
+--------------+
>>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show()
+----+----+----------------+
| a| b|coalesce(a, 0.0)|
+----+----+----------------+
|null|null| 0.0|
| 1|null| 1.0|
|null| 2| 0.0|
+----+----+----------------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.coalesce(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.6)
def corr(col1, col2):
"""Returns a new :class:`Column` for the Pearson Correlation Coefficient for ``col1``
and ``col2``.
>>> a = range(20)
>>> b = [2 * x for x in range(20)]
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(corr("a", "b").alias('c')).collect()
[Row(c=1.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.corr(_to_java_column(col1), _to_java_column(col2)))
@since(2.0)
def covar_pop(col1, col2):
"""Returns a new :class:`Column` for the population covariance of ``col1`` and ``col2``.
>>> a = [1] * 10
>>> b = [1] * 10
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(covar_pop("a", "b").alias('c')).collect()
[Row(c=0.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.covar_pop(_to_java_column(col1), _to_java_column(col2)))
@since(2.0)
def covar_samp(col1, col2):
"""Returns a new :class:`Column` for the sample covariance of ``col1`` and ``col2``.
>>> a = [1] * 10
>>> b = [1] * 10
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(covar_samp("a", "b").alias('c')).collect()
[Row(c=0.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.covar_samp(_to_java_column(col1), _to_java_column(col2)))
@since(1.3)
def countDistinct(col, *cols):
"""Returns a new :class:`Column` for distinct count of ``col`` or ``cols``.
>>> df.agg(countDistinct(df.age, df.name).alias('c')).collect()
[Row(c=2)]
>>> df.agg(countDistinct("age", "name").alias('c')).collect()
[Row(c=2)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.countDistinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.3)
def first(col, ignorenulls=False):
"""Aggregate function: returns the first value in a group.
The function by default returns the first values it sees. It will return the first non-null
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
.. note:: The function is non-deterministic because its results depends on the order of the
rows which may be non-deterministic after a shuffle.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.first(_to_java_column(col), ignorenulls)
return Column(jc)
@since(2.0)
def grouping(col):
"""
Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated
or not, returns 1 for aggregated or 0 for not aggregated in the result set.
>>> df.cube("name").agg(grouping("name"), sum("age")).orderBy("name").show()
+-----+--------------+--------+
| name|grouping(name)|sum(age)|
+-----+--------------+--------+
| null| 1| 7|
|Alice| 0| 2|
| Bob| 0| 5|
+-----+--------------+--------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.grouping(_to_java_column(col))
return Column(jc)
@since(2.0)
def grouping_id(*cols):
"""
Aggregate function: returns the level of grouping, equals to
(grouping(c1) << (n-1)) + (grouping(c2) << (n-2)) + ... + grouping(cn)
.. note:: The list of columns should match with grouping columns exactly, or empty (means all
the grouping columns).
>>> df.cube("name").agg(grouping_id(), sum("age")).orderBy("name").show()
+-----+-------------+--------+
| name|grouping_id()|sum(age)|
+-----+-------------+--------+
| null| 1| 7|
|Alice| 0| 2|
| Bob| 0| 5|
+-----+-------------+--------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.grouping_id(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.6)
def input_file_name():
"""Creates a string column for the file name of the current Spark task.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.input_file_name())
@since(1.6)
def isnan(col):
"""An expression that returns true iff the column is NaN.
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
>>> df.select(isnan("a").alias("r1"), isnan(df.a).alias("r2")).collect()
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.isnan(_to_java_column(col)))
@since(1.6)
def isnull(col):
"""An expression that returns true iff the column is null.
>>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b"))
>>> df.select(isnull("a").alias("r1"), isnull(df.a).alias("r2")).collect()
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.isnull(_to_java_column(col)))
@since(1.3)
def last(col, ignorenulls=False):
"""Aggregate function: returns the last value in a group.
The function by default returns the last values it sees. It will return the last non-null
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
.. note:: The function is non-deterministic because its results depends on the order of the
rows which may be non-deterministic after a shuffle.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.last(_to_java_column(col), ignorenulls)
return Column(jc)
@since(1.6)
def monotonically_increasing_id():
"""A column that generates monotonically increasing 64-bit integers.
The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive.
The current implementation puts the partition ID in the upper 31 bits, and the record number
within each partition in the lower 33 bits. The assumption is that the data frame has
less than 1 billion partitions, and each partition has less than 8 billion records.
.. note:: The function is non-deterministic because its result depends on partition IDs.
As an example, consider a :class:`DataFrame` with two partitions, each with 3 records.
This expression would return the following IDs:
0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594.
>>> df0 = sc.parallelize(range(2), 2).mapPartitions(lambda x: [(1,), (2,), (3,)]).toDF(['col1'])
>>> df0.select(monotonically_increasing_id().alias('id')).collect()
[Row(id=0), Row(id=1), Row(id=2), Row(id=8589934592), Row(id=8589934593), Row(id=8589934594)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.monotonically_increasing_id())
@since(1.6)
def nanvl(col1, col2):
"""Returns col1 if it is not NaN, or col2 if col1 is NaN.
Both inputs should be floating point columns (:class:`DoubleType` or :class:`FloatType`).
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
>>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect()
[Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.nanvl(_to_java_column(col1), _to_java_column(col2)))
@since(3.1)
def percentile_approx(col, percentage, accuracy=10000):
"""Returns the approximate `percentile` of the numeric column `col` which is the smallest value
in the ordered `col` values (sorted from least to greatest) such that no more than `percentage`
of `col` values is less than the value or equal to that value.
The value of percentage must be between 0.0 and 1.0.
The accuracy parameter (default: 10000)
is a positive numeric literal which controls approximation accuracy at the cost of memory.
Higher value of accuracy yields better accuracy, 1.0/accuracy is the relative error
of the approximation.
When percentage is an array, each value of the percentage array must be between 0.0 and 1.0.
In this case, returns the approximate percentile array of column col
at the given percentage array.
>>> key = (col("id") % 3).alias("key")
>>> value = (randn(42) + key * 10).alias("value")
>>> df = spark.range(0, 1000, 1, 1).select(key, value)
>>> df.select(
... percentile_approx("value", [0.25, 0.5, 0.75], 1000000).alias("quantiles")
... ).printSchema()
root
|-- quantiles: array (nullable = true)
| |-- element: double (containsNull = false)
>>> df.groupBy("key").agg(
... percentile_approx("value", 0.5, lit(1000000)).alias("median")
... ).printSchema()
root
|-- key: long (nullable = true)
|-- median: double (nullable = true)
"""
sc = SparkContext._active_spark_context
if isinstance(percentage, (list, tuple)):
# A local list
percentage = sc._jvm.functions.array(_to_seq(sc, [
_create_column_from_literal(x) for x in percentage
]))
elif isinstance(percentage, Column):
# Already a Column
percentage = _to_java_column(percentage)
else:
# Probably scalar
percentage = _create_column_from_literal(percentage)
accuracy = (
_to_java_column(accuracy) if isinstance(accuracy, Column)
else _create_column_from_literal(accuracy)
)
return Column(sc._jvm.functions.percentile_approx(_to_java_column(col), percentage, accuracy))
@since(1.4)
def rand(seed=None):
"""Generates a random column with independent and identically distributed (i.i.d.) samples
uniformly distributed in [0.0, 1.0).
.. note:: The function is non-deterministic in general case.
>>> df.withColumn('rand', rand(seed=42) * 3).collect()
[Row(age=2, name='Alice', rand=2.4052597283576684),
Row(age=5, name='Bob', rand=2.3913904055683974)]
"""
sc = SparkContext._active_spark_context
if seed is not None:
jc = sc._jvm.functions.rand(seed)
else:
jc = sc._jvm.functions.rand()
return Column(jc)
@since(1.4)
def randn(seed=None):
"""Generates a column with independent and identically distributed (i.i.d.) samples from
the standard normal distribution.
.. note:: The function is non-deterministic in general case.
>>> df.withColumn('randn', randn(seed=42)).collect()
[Row(age=2, name='Alice', randn=1.1027054481455365),
Row(age=5, name='Bob', randn=0.7400395449950132)]
"""
sc = SparkContext._active_spark_context
if seed is not None:
jc = sc._jvm.functions.randn(seed)
else:
jc = sc._jvm.functions.randn()
return Column(jc)
@since(1.5)
def round(col, scale=0):
"""
Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0
or at integral part when `scale` < 0.
>>> spark.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect()
[Row(r=3.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.round(_to_java_column(col), scale))
@since(2.0)
def bround(col, scale=0):
"""
Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0
or at integral part when `scale` < 0.
>>> spark.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.bround(_to_java_column(col), scale))
@since(1.5)
def shiftLeft(col, numBits):
"""Shift the given value numBits left.
>>> spark.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect()
[Row(r=42)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.shiftLeft(_to_java_column(col), numBits))
@since(1.5)
def shiftRight(col, numBits):
"""(Signed) shift the given value numBits right.
>>> spark.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect()
[Row(r=21)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.shiftRight(_to_java_column(col), numBits)
return Column(jc)
@since(1.5)
def shiftRightUnsigned(col, numBits):
"""Unsigned shift the given value numBits right.
>>> df = spark.createDataFrame([(-42,)], ['a'])
>>> df.select(shiftRightUnsigned('a', 1).alias('r')).collect()
[Row(r=9223372036854775787)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.shiftRightUnsigned(_to_java_column(col), numBits)
return Column(jc)
@since(1.6)
def spark_partition_id():
"""A column for partition ID.
.. note:: This is indeterministic because it depends on data partitioning and task scheduling.
>>> df.repartition(1).select(spark_partition_id().alias("pid")).collect()
[Row(pid=0), Row(pid=0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.spark_partition_id())
@since(1.5)
def expr(str):
"""Parses the expression string into the column that it represents
>>> df.select(expr("length(name)")).collect()
[Row(length(name)=5), Row(length(name)=3)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.expr(str))
@since(1.4)
def struct(*cols):
"""Creates a new struct column.
:param cols: list of column names (string) or list of :class:`Column` expressions
>>> df.select(struct('age', 'name').alias("struct")).collect()
[Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))]
>>> df.select(struct([df.age, df.name]).alias("struct")).collect()
[Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))]
"""
sc = SparkContext._active_spark_context
if len(cols) == 1 and isinstance(cols[0], (list, set)):
cols = cols[0]
jc = sc._jvm.functions.struct(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.5)
def greatest(*cols):
"""
Returns the greatest value of the list of column names, skipping null values.
This function takes at least 2 parameters. It will return null iff all parameters are null.
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
>>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect()
[Row(greatest=4)]
"""
if len(cols) < 2:
raise ValueError("greatest should take at least two columns")
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.greatest(_to_seq(sc, cols, _to_java_column)))
@since(1.5)
def least(*cols):
"""
Returns the least value of the list of column names, skipping null values.
This function takes at least 2 parameters. It will return null iff all parameters are null.
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
>>> df.select(least(df.a, df.b, df.c).alias("least")).collect()
[Row(least=1)]
"""
if len(cols) < 2:
raise ValueError("least should take at least two columns")
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.least(_to_seq(sc, cols, _to_java_column)))
@since(1.4)
def when(condition, value):
"""Evaluates a list of conditions and returns one of multiple possible result expressions.
If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
:param condition: a boolean :class:`Column` expression.
:param value: a literal value, or a :class:`Column` expression.
>>> df.select(when(df['age'] == 2, 3).otherwise(4).alias("age")).collect()
[Row(age=3), Row(age=4)]
>>> df.select(when(df.age == 2, df.age + 1).alias("age")).collect()
[Row(age=3), Row(age=None)]
"""
sc = SparkContext._active_spark_context
if not isinstance(condition, Column):
raise TypeError("condition should be a Column")
v = value._jc if isinstance(value, Column) else value
jc = sc._jvm.functions.when(condition._jc, v)
return Column(jc)
@since(1.5)
def log(arg1, arg2=None):
"""Returns the first argument-based logarithm of the second argument.
If there is only one argument, then this takes the natural logarithm of the argument.
>>> df.select(log(10.0, df.age).alias('ten')).rdd.map(lambda l: str(l.ten)[:7]).collect()
['0.30102', '0.69897']
>>> df.select(log(df.age).alias('e')).rdd.map(lambda l: str(l.e)[:7]).collect()
['0.69314', '1.60943']
"""
sc = SparkContext._active_spark_context
if arg2 is None:
jc = sc._jvm.functions.log(_to_java_column(arg1))
else:
jc = sc._jvm.functions.log(arg1, _to_java_column(arg2))
return Column(jc)
@since(1.5)
def log2(col):
"""Returns the base-2 logarithm of the argument.
>>> spark.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect()
[Row(log2=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.log2(_to_java_column(col)))
@since(1.5)
def conv(col, fromBase, toBase):
"""
Convert a number in a string column from one base to another.
>>> df = spark.createDataFrame([("010101",)], ['n'])
>>> df.select(conv(df.n, 2, 16).alias('hex')).collect()
[Row(hex='15')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.conv(_to_java_column(col), fromBase, toBase))
@since(1.5)
def factorial(col):
"""
Computes the factorial of the given value.
>>> df = spark.createDataFrame([(5,)], ['n'])
>>> df.select(factorial(df.n).alias('f')).collect()
[Row(f=120)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.factorial(_to_java_column(col)))
# --------------- Window functions ------------------------
@since(1.4)
def lag(col, offset=1, default=None):
"""
Window function: returns the value that is `offset` rows before the current row, and
`defaultValue` if there is less than `offset` rows before the current row. For example,
an `offset` of one will return the previous row at any given point in the window partition.
This is equivalent to the LAG function in SQL.
:param col: name of column or expression
:param offset: number of row to extend
:param default: default value
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.lag(_to_java_column(col), offset, default))
@since(1.4)
def lead(col, offset=1, default=None):
"""
Window function: returns the value that is `offset` rows after the current row, and
`defaultValue` if there is less than `offset` rows after the current row. For example,
an `offset` of one will return the next row at any given point in the window partition.
This is equivalent to the LEAD function in SQL.
:param col: name of column or expression
:param offset: number of row to extend
:param default: default value
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.lead(_to_java_column(col), offset, default))
@since(1.4)
def ntile(n):
"""
Window function: returns the ntile group id (from 1 to `n` inclusive)
in an ordered window partition. For example, if `n` is 4, the first
quarter of the rows will get value 1, the second quarter will get 2,
the third quarter will get 3, and the last quarter will get 4.
This is equivalent to the NTILE function in SQL.
:param n: an integer
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.ntile(int(n)))
# ---------------------- Date/Timestamp functions ------------------------------
@since(1.5)
def current_date():
"""
Returns the current date as a :class:`DateType` column.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.current_date())
def current_timestamp():
"""
Returns the current timestamp as a :class:`TimestampType` column.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.current_timestamp())
@since(1.5)
def date_format(date, format):
"""
Converts a date/timestamp/string to a value of string in the format specified by the date
format given by the second argument.
A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All
pattern letters of `datetime pattern`_. can be used.
.. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
.. note:: Use when ever possible specialized functions like `year`. These benefit from a
specialized implementation.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(date_format('dt', 'MM/dd/yyy').alias('date')).collect()
[Row(date='04/08/2015')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.date_format(_to_java_column(date), format))
@since(1.5)
def year(col):
"""
Extract the year of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(year('dt').alias('year')).collect()
[Row(year=2015)]