-
Notifications
You must be signed in to change notification settings - Fork 44
/
dataframe.py
1763 lines (1525 loc) · 63.5 KB
/
dataframe.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
import warnings
from pysparkling import StorageLevel
from pysparkling.sql.column import parse, Column
from pysparkling.sql.expressions.fields import FieldAsExpression
from pysparkling.sql.internal_utils.joins import JOIN_TYPES, CROSS_JOIN
from pysparkling.sql.internals import InternalGroupedDataFrame, ROLLUP_TYPE, CUBE_TYPE
from pysparkling.sql.types import ByteType, ShortType, IntegerType, FloatType, IntegralType, \
TimestampType, _check_series_convert_timestamps_local_tz
from pysparkling.sql.utils import IllegalArgumentException, AnalysisException, \
require_minimum_pandas_version
_NoValue = object()
class DataFrame(object):
def __init__(self, jdf, sql_ctx):
self._jdf = jdf
self.sql_ctx = sql_ctx
@property
def rdd(self):
return self._jdf.rdd()
@property
def is_cached(self):
return self._jdf.is_cached()
@property
def na(self):
"""Returns a :class:`DataFrameNaFunctions` for handling missing values.
"""
return DataFrameNaFunctions(self)
@property
def stat(self):
return DataFrameStatFunctions(self)
def toJSON(self, use_unicode=True):
"""
Return an RDD containing all items after JSONification
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(2)
>>> df.toJSON().collect()
['{"id": 0}', '{"id": 1}']
"""
return self._jdf.toJSON(use_unicode)
def createTempView(self, name):
self._jdf.createTempView(name)
def createOrReplaceTempView(self, name):
self._jdf.createOrReplaceTempView(name)
def createGlobalTempView(self, name):
self._jdf.createGlobalTempView(name)
def createOrReplaceGlobalTempView(self, name):
self._jdf.createOrReplaceGlobalTempView(name)
@property
def write(self):
# Top level import would cause cyclic dependencies
# pylint: disable=import-outside-toplevel
from pysparkling.sql.readwriter import DataFrameWriter
return DataFrameWriter(self)
@property
def writeStream(self):
raise NotImplementedError("Pysparkling does not support yet writing to stream")
@property
def schema(self):
return self._jdf.unbound_schema
def printSchema(self):
print(self.schema.treeString())
def explain(self, extended=False):
print("Pysparkling does not provide query execution explanation")
def exceptAll(self, other):
"""Return a new :class:`DataFrame` containing rows in this :class:`DataFrame` but
not in another :class:`DataFrame` while preserving duplicates.
This is equivalent to `EXCEPT ALL` in SQL.
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df1 = spark.createDataFrame([
... ("a", 1),
... ("a", 1),
... ("a", 1),
... ("a", 2),
... ("b", 3),
... ("c", 4)
... ], ["C1", "C2"])
>>> df2 = spark.createDataFrame([("a", 1), ("b", 3)], ["C1", "C2"])
>>> df1.exceptAll(df2).show()
+---+---+
| C1| C2|
+---+---+
| a| 1|
| a| 1|
| a| 2|
| c| 4|
+---+---+
Also as standard in SQL, this function resolves columns by position (not by name).
"""
# noinspection PyProtectedMember
return DataFrame(self._jdf.exceptAll(other._jdf), self.sql_ctx)
def isLocal(self):
return True
def isStreaming(self):
# pylint: disable=fixme
# todo: Add support of streaming
return False
def show(self, n=20, truncate=True, vertical=False):
"""
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> from pysparkling.sql.functions import col
>>> spark = SparkSession(Context())
>>> df = spark.createDataFrame(
... [Row(age=5, name='Bob'), Row(age=2, name='Alice')]
... )
>>> df.show()
+---+-----+
|age| name|
+---+-----+
| 5| Bob|
| 2|Alice|
+---+-----+
>>> from pysparkling.sql.functions import map_from_arrays, array, col
>>> df = spark.range(3)
>>> df.select(array(df.id, df.id * 2)).show()
+-------------------+
|array(id, (id * 2))|
+-------------------+
| [0, 0]|
| [1, 2]|
| [2, 4]|
+-------------------+
>>> df.select(map_from_arrays(array(df.id), array(df.id))).show()
+-------------------------------------+
|map_from_arrays(array(id), array(id))|
+-------------------------------------+
| [0 -> 0]|
| [1 -> 1]|
| [2 -> 2]|
+-------------------------------------+
>>> df.select(map_from_arrays(array(df.id, df.id * 2), array(df.id, df.id * 2))).show()
+---------------------------------------------------------+
|map_from_arrays(array(id, (id * 2)), array(id, (id * 2)))|
+---------------------------------------------------------+
| [0 -> 0]|
| [1 -> 1, 2 -> 2]|
| [2 -> 2, 4 -> 4]|
+---------------------------------------------------------+
>>> c = col("id")
>>> (spark.range(9, 11)
... .select(c, c*2, c**2)
... .show(vertical=True)) # doctest: +NORMALIZE_WHITESPACE
-RECORD 0-------------
id | 9
(id * 2) | 18
POWER(id, 2) | 81.0
-RECORD 1-------------
id | 10
(id * 2) | 20
POWER(id, 2) | 100.0
"""
if truncate is True:
print(self._jdf.showString(n, 20, vertical))
else:
print(self._jdf.showString(n, int(truncate), vertical))
def __repr__(self):
return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes))
def checkpoint(self, eager=True):
raise NotImplementedError("Streaming is not supported in PySparkling")
def localCheckpoint(self, eager=True):
raise NotImplementedError("Streaming is not supported in PySparkling")
def withWatermark(self, eventTime, delayThreshold):
raise NotImplementedError("Streaming is not supported in PySparkling")
def hint(self, name, *parameters):
if len(parameters) == 1 and isinstance(parameters[0], list):
parameters = parameters[0]
if not isinstance(name, str):
raise TypeError("name should be provided as str, got {0}".format(type(name)))
allowed_types = (str, list, float, int)
for p in parameters:
if not isinstance(p, allowed_types):
raise TypeError(
"all parameters should be in {0}, got {1} of type {2}".format(
allowed_types, p, type(p)))
# No hint are supported by pysparkling hence nothing is done here
jdf = self._jdf
return DataFrame(jdf, self.sql_ctx)
def count(self):
"""Returns the number of rows in this :class:`DataFrame`.
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(2)
>>> df.count()
2
"""
return self._jdf.count()
def collect(self):
"""Returns the number of rows in this :class:`DataFrame`.
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(2)
>>> df.collect()
[Row(id=0), Row(id=1)]
"""
return self._jdf.collect()
def toLocalIterator(self):
"""Returns an iterator on the content of this DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(2)
>>> list(df.toLocalIterator())
[Row(id=0), Row(id=1)]
"""
return self._jdf.toLocalIterator()
def limit(self, n):
"""Restrict the DataFrame to the first n items
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(2).limit(1)
>>> df.show()
+---+
| id|
+---+
| 0|
+---+
"""
return DataFrame(self._jdf.limit(n), self.sql_ctx)
def take(self, n):
"""Return a list with the first n items of the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> spark.range(2).take(1)
[Row(id=0)]
"""
return self._jdf.take(n)
def foreach(self, f):
"""Execute a function for each item of the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> result = spark.range(2).foreach(print)
Row(id=0)
Row(id=1)
>>> result is None
True
"""
self._jdf.foreach(f)
def foreachPartition(self, f):
"""Execute a function for each partition of the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> result = (spark.range(4, numPartitions=2)
... .foreachPartition(lambda partition: print(list(partition))))
[Row(id=0), Row(id=1)]
[Row(id=2), Row(id=3)]
>>> result is None
True
"""
self._jdf.foreachPartition(f)
def cache(self):
"""Cache the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(4, numPartitions=2).cache()
>>> df.is_cached
True
"""
return DataFrame(self._jdf.cache(), self.sql_ctx)
def persist(self, storageLevel=StorageLevel.MEMORY_ONLY):
"""Cache the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(4, numPartitions=2).persist()
>>> df.is_cached
True
>>> df.storageLevel == StorageLevel.MEMORY_ONLY
True
"""
if storageLevel != StorageLevel.MEMORY_ONLY:
raise NotImplementedError(
"Pysparkling currently only supports memory as the storage level"
)
return DataFrame(self._jdf.persist(storageLevel), self.sql_ctx)
@property
def storageLevel(self):
"""Cache the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(4, numPartitions=2)
>>> df.storageLevel
StorageLevel(False, False, False, False, 1)
>>> persisted_df = df.persist()
>>> persisted_df.is_cached
True
>>> persisted_df.storageLevel
StorageLevel(False, True, False, False, 1)
"""
if self.is_cached:
return self._jdf.storageLevel
return StorageLevel(False, False, False, False, 1)
def unpersist(self, blocking=False):
"""Cache the DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(4, numPartitions=2)
>>> df.storageLevel
StorageLevel(False, False, False, False, 1)
>>> persisted_df = df.persist()
>>> persisted_df.is_cached
True
>>> persisted_df.storageLevel
StorageLevel(False, True, False, False, 1)
>>> unpersisted_df = persisted_df.unpersist()
>>> unpersisted_df.storageLevel
StorageLevel(False, False, False, False, 1)
"""
return DataFrame(self._jdf.unpersist(blocking), self.sql_ctx)
def coalesce(self, numPartitions):
"""Coalesce the dataframe
:param int numPartitions: Max number of partitions in the resulting dataframe.
:rtype: DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> spark.range(4, numPartitions=2).coalesce(1).rdd.getNumPartitions()
1
>>> spark.range(4, numPartitions=2).coalesce(4).rdd.getNumPartitions()
2
>>> spark.range(3).coalesce(1).collect()
[Row(id=0), Row(id=1), Row(id=2)]
>>> df = spark.range(200).repartition(300)
>>> df = df.filter(df.id % 2 == 0).select(df.id * 2)
>>> df = df.coalesce(299)
>>> df.rdd.getNumPartitions()
299
>>> df = df.coalesce(298)
>>> df.rdd.getNumPartitions()
298
>>> df = df.coalesce(174)
>>> df.rdd.getNumPartitions()
174
>>> df = df.coalesce(75)
>>> df.rdd.getNumPartitions()
75
>>> df = df.coalesce(1)
>>> df.rdd.getNumPartitions()
1
>>> df.count()
100
"""
return DataFrame(self._jdf.coalesce(numPartitions), self.sql_ctx)
def repartition(self, numPartitions, *cols):
"""Repartition the dataframe
:param int numPartitions: Number of partitions in the resulting dataframe.
:rtype: DataFrame
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> spark.range(4, numPartitions=2).repartition(1).rdd.getNumPartitions()
1
>>> spark.range(4, numPartitions=2).repartition(4).rdd.getNumPartitions()
4
>>> spark.range(4, numPartitions=2).repartition("id").rdd.getNumPartitions()
200
>>> spark.createDataFrame(
... [[0], [1], [1], [2]],
... ["v"]
... ).repartition(3, "v").rdd.foreachPartition(lambda x: print((list(x))))
[Row(v=0)]
[Row(v=1), Row(v=1)]
[Row(v=2)]
"""
if isinstance(numPartitions, int):
if not cols:
return DataFrame(self._jdf.simple_repartition(numPartitions), self.sql_ctx)
cols = [parse(col) for col in cols]
repartitioned_jdf = self._jdf.repartition(numPartitions, cols)
return DataFrame(repartitioned_jdf, self.sql_ctx)
if isinstance(numPartitions, (str, Column)):
return self.repartition(200, numPartitions, *cols)
raise TypeError("numPartitions should be an int, str or Column")
def repartitionByRange(self, numPartitions, *cols):
"""
Returns a new :class:`DataFrame` partitioned by the given partitioning expressions. The
resulting DataFrame is range partitioned.
:param numPartitions:
can be an int to specify the target number of partitions or a Column.
If it is a Column, it will be used as the first partitioning column. If not specified,
the default number of partitions is used.
At least one partition-by expression must be specified.
Note that due to performance reasons this method uses sampling to estimate the ranges.
Hence, the output may not be consistent, since sampling can return different values.
Sort orders are not supported in this pysparkling implementation
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> spark.range(4, numPartitions=2).repartitionByRange(1, "id").rdd.getNumPartitions()
1
>>> spark.createDataFrame(
... [[0], [1], [1], [2], [4]],
... ["v"]
... ).repartitionByRange(3, "v").rdd.foreachPartition(lambda x: print((list(x))))
[Row(v=0), Row(v=1), Row(v=1)]
[Row(v=2)]
[Row(v=4)]
"""
# pylint: disable=fixme
# todo: support sort orders and assume "ascending nulls first" if needed
if isinstance(numPartitions, int):
if not cols:
raise ValueError("At least one partition-by expression must be specified.")
cols = [parse(col) for col in cols]
repartitioned_jdf = self._jdf.repartitionByRange(numPartitions, *cols)
return DataFrame(repartitioned_jdf, self.sql_ctx)
if isinstance(numPartitions, (str, Column)):
return self.repartitionByRange(200, numPartitions, *cols)
raise TypeError("numPartitions should be an int, str or Column")
def distinct(self):
return DataFrame(self._jdf.distinct(), self.sql_ctx)
def sample(self, withReplacement=None, fraction=None, seed=None):
is_withReplacement_set = isinstance(withReplacement, bool) and isinstance(fraction, float)
is_withReplacement_omitted_kwargs = withReplacement is None and isinstance(fraction, float)
is_withReplacement_omitted_args = isinstance(withReplacement, float)
if not (is_withReplacement_set
or is_withReplacement_omitted_kwargs
or is_withReplacement_omitted_args):
argtypes = [
str(type(arg))
for arg in [withReplacement, fraction, seed]
if arg is not None
]
raise TypeError(
"withReplacement (optional), fraction (required) and seed (optional)"
" should be a bool, float and number; however, "
"got [%s]." % ", ".join(argtypes))
if is_withReplacement_omitted_args:
if fraction is not None:
seed = fraction
fraction = withReplacement
withReplacement = None
seed = int(seed) if seed is not None else None
args = [arg for arg in [withReplacement, fraction, seed] if arg is not None]
jdf = self._jdf.sample(*args)
return DataFrame(jdf, self.sql_ctx)
def sampleBy(self, col, fractions, seed=None):
"""
Returns a stratified sample without replacement based on the
fraction given on each stratum.
:param col: column that defines strata
:param fractions:
sampling fraction for each stratum. If a stratum is not
specified, we treat its fraction as zero.
:param seed: random seed
:return: a new DataFrame that represents the stratified sample
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> from pysparkling.sql.functions import count, lit
>>> spark = SparkSession(Context())
>>> dataset = spark.createDataFrame(
... [[i % 3] for i in range(100)],
... ["key"]
... )
>>> sampled = dataset.sampleBy("key", fractions={0: 0.5, 1: 0.25}, seed=0)
>>> sampled.groupBy("key").agg(count(lit(1))).show()
+---+--------+
|key|count(1)|
+---+--------+
| 0| 17|
| 1| 8|
+---+--------+
>>> sampled.groupBy("key").count().show()
+---+-----+
|key|count|
+---+-----+
| 0| 17|
| 1| 8|
+---+-----+
>>> sampled.groupBy("key").count().orderBy("key").show()
+---+-----+
|key|count|
+---+-----+
| 0| 17|
| 1| 8|
+---+-----+
>>> dataset.sampleBy("key", fractions={2: 1.0}, seed=0).count()
33
"""
return DataFrame(self._jdf.sampleBy(parse(col), fractions, seed), self.sql_ctx)
def randomSplit(self, weights, seed=None):
for w in weights:
if w < 0.0:
raise ValueError("Weights must be positive. Found weight value: {}".format(w))
seed = int(seed) if seed is not None else None
rdd_array = self._jdf.randomSplit(weights, seed)
return [DataFrame(rdd, self.sql_ctx) for rdd in rdd_array]
@property
def dtypes(self):
return [(f.name, f.dataType.simpleString()) for f in self.schema.fields]
@property
def columns(self):
return [f.name for f in self.schema.fields]
def alias(self, alias):
assert isinstance(alias, str), "alias should be a string"
raise NotImplementedError("Pysparkling does not currently support SQL catalog")
def crossJoin(self, other):
"""
Returns the cartesian product of self and other
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.createDataFrame([
... Row(age=2, name='Alice'),
... Row(age=5, name='Bob')
... ])
>>> df2 = spark.createDataFrame([
... Row(name='Tom', height=80),
... Row(name='Bob', height=85)
... ])
>>> df.select("age", "name").collect()
[Row(age=2, name='Alice'), Row(age=5, name='Bob')]
>>> df2.select("name", "height").collect()
[Row(name='Tom', height=80), Row(name='Bob', height=85)]
>>> df.crossJoin(df2.select("height")).select("age", "name", "height").show()
+---+-----+------+
|age| name|height|
+---+-----+------+
| 2|Alice| 80|
| 2|Alice| 85|
| 5| Bob| 80|
| 5| Bob| 85|
+---+-----+------+
"""
# noinspection PyProtectedMember
jdf = self._jdf.crossJoin(other._jdf)
return DataFrame(jdf, self.sql_ctx)
def join(self, other, on=None, how="inner"):
"""
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> from pysparkling.sql.functions import length, col, lit
>>> spark = SparkSession(Context())
>>> left_df = spark.range(1, 3).select(
... lit("test_value"),
... (col("id")*2).alias("id"),
... lit("left").alias("side")
... )
>>> right_df = spark.range(1, 3).select(
... lit("test_value"),
... col("id"),
... lit("right").alias("side")
... )
>>>
>>> left_df.join(right_df, on="id", how="inner").orderBy("id").show()
+---+----------+----+----------+-----+
| id|test_value|side|test_value| side|
+---+----------+----+----------+-----+
| 2|test_value|left|test_value|right|
+---+----------+----+----------+-----+
>>> left_df.join(right_df, on="id", how="left_outer").orderBy("id").show()
+---+----------+----+----------+-----+
| id|test_value|side|test_value| side|
+---+----------+----+----------+-----+
| 2|test_value|left|test_value|right|
| 4|test_value|left| null| null|
+---+----------+----+----------+-----+
>>> left_df.join(right_df, on="id", how="right_outer").orderBy("id").show()
+---+----------+----+----------+-----+
| id|test_value|side|test_value| side|
+---+----------+----+----------+-----+
| 1| null|null|test_value|right|
| 2|test_value|left|test_value|right|
+---+----------+----+----------+-----+
>>> left_df.join(right_df, on="id", how="full_outer").orderBy("id").show()
+---+----------+----+----------+-----+
| id|test_value|side|test_value| side|
+---+----------+----+----------+-----+
| 1| null|null|test_value|right|
| 2|test_value|left|test_value|right|
| 4|test_value|left| null| null|
+---+----------+----+----------+-----+
>>> left_df.join(right_df, on="id", how="leftsemi").orderBy("id").show()
+---+----------+----+
| id|test_value|side|
+---+----------+----+
| 2|test_value|left|
+---+----------+----+
>>> left_df.join(right_df, on="id", how="leftanti").orderBy("id").show()
+---+----------+----+
| id|test_value|side|
+---+----------+----+
| 4|test_value|left|
+---+----------+----+
>>> # Degenerated case:
>>> degen_left = left_df.withColumn("left_id", left_df.id).select(
... left_df.id, (left_df.id*2).alias("id"), "left_id"
... )
>>> degen_right = right_df.withColumn("right_id", right_df.id).select(
... right_df.id, (right_df.id*2).alias("id"), "right_id"
... )
>>> degen_left.join(degen_right, on="id", how="outer").orderBy("left_id").show()
+---+----+-------+----+--------+
| id| id|left_id| id|right_id|
+---+----+-------+----+--------+
| 1|null| null| 2| 1|
| 2| 4| 2| 4| 2|
| 4| 8| 4|null| null|
+---+----+-------+----+--------+
>>> a = spark.createDataFrame([Row(name='o', time=1479441846)])
>>> b = spark.createDataFrame([["a"],["b"],["o"]]).select(col("_1").alias("n"))
>>> a.join(b, on=length(a.name) * 2 == length(b.n) + length(a.name)).orderBy("n").show()
+----+----------+---+
|name| time| n|
+----+----------+---+
| o|1479441846| a|
| o|1479441846| b|
| o|1479441846| o|
+----+----------+---+
>>> c = spark.createDataFrame([["a"],["b"],["o"]]).select(col("_1").alias("name"))
>>> a.join(c, on=(a.name == c.name)).show()
+----+----------+----+
|name| time|name|
+----+----------+----+
| o|1479441846| o|
+----+----------+----+
>>> a.join(c, on=(a.name != c.name)).show()
+----+----------+----+
|name| time|name|
+----+----------+----+
| o|1479441846| a|
| o|1479441846| b|
+----+----------+----+
"""
# noinspection PyProtectedMember
if isinstance(on, str):
return self.join(other=other, on=[on], how=how)
how = how.lower().replace("_", "")
if how not in JOIN_TYPES:
raise IllegalArgumentException("Invalid how argument in join: {0}".format(how))
how = JOIN_TYPES[how]
if how == CROSS_JOIN and on is not None:
raise IllegalArgumentException("`on` must be None for a crossJoin")
if how != CROSS_JOIN and on is None:
raise IllegalArgumentException(
"Join condition is missing. "
"Use the CROSS JOIN syntax to allow cartesian products"
)
return DataFrame(self._jdf.join(other._jdf, on, how), self.sql_ctx)
def sortWithinPartitions(self, *cols, **kwargs):
"""
>>> from pysparkling import Context
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.range(4, numPartitions=2)
>>> (df.sortWithinPartitions("id", ascending=False)
... .foreachPartition(lambda p: print(list(p))))
[Row(id=1), Row(id=0)]
[Row(id=3), Row(id=2)]
"""
ascending = kwargs.pop("ascending", True)
if kwargs:
raise TypeError("Unrecognized arguments: {0}".format(kwargs))
sorted_jdf = self._jdf.sortWithinPartitions(cols, ascending=ascending)
return DataFrame(sorted_jdf, self.sql_ctx)
def sort(self, *cols, **kwargs):
"""Returns a new :class:`DataFrame` sorted by the specified column(s).
:param cols: list of :class:`Column` or column names to sort by.
:param ascending: boolean or list of boolean (default True).
Sort ascending vs. descending. Specify list for multiple sort orders.
If a list is specified, length of the list must equal length of the `cols`.
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> from pysparkling.sql.functions import desc
>>> spark = SparkSession(Context())
>>> df = spark.createDataFrame(
... [Row(age=5, name='Bob'), Row(age=2, name='Alice')]
... )
>>> df.sort("age", ascending=False).show()
+---+-----+
|age| name|
+---+-----+
| 5| Bob|
| 2|Alice|
+---+-----+
>>> df.sort("age", ascending=False).show()
+---+-----+
|age| name|
+---+-----+
| 5| Bob|
| 2|Alice|
+---+-----+
>>> df.sort("age").show()
+---+-----+
|age| name|
+---+-----+
| 2|Alice|
| 5| Bob|
+---+-----+
>>> df.orderBy(desc("age"), "name").collect()
[Row(age=5, name='Bob'), Row(age=2, name='Alice')]
>>> df.orderBy(["age", "name"], ascending=[0, 1]).collect()
[Row(age=5, name='Bob'), Row(age=2, name='Alice')]
"""
if len(cols) == 1 and isinstance(cols[0], list):
cols = cols[0]
exprs = [parse(col) for col in cols]
sorting_cols = self._sort_cols(exprs, kwargs)
sorted_jdf = self._jdf.sort(sorting_cols)
return DataFrame(sorted_jdf, self.sql_ctx)
def orderBy(self, *cols, **kwargs):
return self.sort(*cols, **kwargs)
@staticmethod
def _sort_cols(cols, kwargs):
""" Return a list of Columns that describes the sort order
"""
if not cols:
raise ValueError("should sort by at least one column")
ascending = kwargs.pop('ascending', True)
if isinstance(ascending, (bool, int)):
if not ascending:
cols = [jc.desc() for jc in cols]
elif isinstance(ascending, list):
cols = [jc if asc else jc.desc() for asc, jc in zip(ascending, cols)]
else:
raise TypeError("ascending can only be boolean or list, but got %s" % type(ascending))
if kwargs:
raise TypeError("Unrecognized arguments: {0}".format(kwargs))
return cols
def describe(self, *cols):
"""Computes basic statistics for numeric and string columns.
This include count, mean, stddev, min, and max. If no columns are
given, this function computes statistics for all numerical or string columns.
.. note:: This function is meant for exploratory data analysis, as we make no
guarantee about the backward compatibility of the schema of the resulting DataFrame.
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.createDataFrame(
... [Row(age=5, name='Bob'), Row(age=2, name='Alice')]
... )
>>> df.describe(['age']).show()
+-------+------------------+
|summary| age|
+-------+------------------+
| count| 2|
| mean| 3.5|
| stddev|2.1213203435596424|
| min| 2|
| max| 5|
+-------+------------------+
>>> df.describe().show()
+-------+------------------+-----+
|summary| age| name|
+-------+------------------+-----+
| count| 2| 2|
| mean| 3.5| null|
| stddev|2.1213203435596424| null|
| min| 2|Alice|
| max| 5| Bob|
+-------+------------------+-----+
Use summary for expanded statistics and control over which statistics to compute.
"""
if len(cols) == 1 and isinstance(cols[0], list):
cols = cols[0]
if not cols:
cols = ["*"]
return DataFrame(self._jdf.describe(cols), self.sql_ctx)
def summary(self, *statistics):
"""Computes specified statistics for numeric and string columns. Available statistics are:
- count
- mean
- stddev
- min
- max
- arbitrary approximate percentiles specified as a percentage (eg, 75%)
If no statistics are given, this function computes count, mean, stddev, min,
approximate quartiles (percentiles at 25%, 50%, and 75%), and max.
.. note:: This function is meant for exploratory data analysis, as we make no
guarantee about the backward compatibility of the schema of the resulting DataFrame.
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> df = spark.createDataFrame(
... [Row(age=5, name='Bob'), Row(age=2, name='Alice')]
... )
>>> df.summary().show()
+-------+------------------+-----+
|summary| age| name|
+-------+------------------+-----+
| count| 2| 2|
| mean| 3.5| null|
| stddev|2.1213203435596424| null|
| min| 2|Alice|
| 25%| 2| null|
| 50%| 2| null|
| 75%| 5| null|
| max| 5| Bob|
+-------+------------------+-----+
>>> df.summary("count", "min", "25%", "75%", "max").show()
+-------+---+-----+
|summary|age| name|
+-------+---+-----+
| count| 2| 2|
| min| 2|Alice|
| 25%| 2| null|
| 75%| 5| null|
| max| 5| Bob|
+-------+---+-----+
To do a summary for specific columns first select them:
>>> df.select("age", "name").summary("count").show()
+-------+---+----+
|summary|age|name|
+-------+---+----+
| count| 2| 2|
+-------+---+----+
See also describe for basic statistics.
"""
if len(statistics) == 1 and isinstance(statistics[0], list):
statistics = statistics[0]
jdf = self._jdf.summary(statistics)
return DataFrame(jdf, self.sql_ctx)
def head(self, n=None):
if n is None:
rs = self.head(1)
return rs[0] if rs else None
return self.take(n)
def first(self):
return self.head()
def __getitem__(self, item):
if isinstance(item, str):
return getattr(self, item)
if isinstance(item, Column):
return self.filter(item)
if isinstance(item, (list, tuple)):
return self.select(*item)
if isinstance(item, int):
return Column(FieldAsExpression(self._jdf.bound_schema[item]))
raise TypeError("unexpected item type: %s" % type(item))
def __getattr__(self, name):
if name.startswith("_"):
raise AttributeError(name)
try:
field_position = Column(name).find_position_in_schema(self.schema)
except AnalysisException:
raise AttributeError
return self[field_position]
def select(self, *cols):
"""Projects a set of expressions and returns a new :class:`DataFrame`.
:param cols: list of column names (string) or expressions (:class:`Column`).
If one of the column names is '*', that column is expanded to include all columns
in the current DataFrame.
>>> from pysparkling import Context, Row
>>> from pysparkling.sql.session import SparkSession
>>> spark = SparkSession(Context())
>>> from pysparkling.sql.functions import (explode, split, posexplode,
... posexplode_outer, col, avg)
>>> df = spark.createDataFrame(
... [Row(age=2, name='Alice'), Row(age=5, name='Bob')]
... )
>>> df.select('*').show()
+---+-----+
|age| name|
+---+-----+
| 2|Alice|
| 5| Bob|
+---+-----+
>>> df.select('name', 'age').show()
+-----+---+
| name|age|
+-----+---+