-
Notifications
You must be signed in to change notification settings - Fork 28k
/
test_rdd.py
939 lines (790 loc) · 38 KB
/
test_rdd.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
#
# 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.
#
from datetime import datetime, timedelta
import hashlib
import os
import random
import tempfile
import time
import unittest
from glob import glob
from py4j.protocol import Py4JJavaError
from pyspark import shuffle, RDD
from pyspark.resource import ExecutorResourceRequests, ResourceProfileBuilder, TaskResourceRequests
from pyspark.serializers import (
CloudPickleSerializer,
BatchedSerializer,
CPickleSerializer,
MarshalSerializer,
UTF8Deserializer,
NoOpSerializer,
)
from pyspark.sql import SparkSession
from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME, QuietTest, have_numpy
from pyspark.testing.sqlutils import have_pandas
global_func = lambda: "Hi" # noqa: E731
class RDDTests(ReusedPySparkTestCase):
def test_range(self):
self.assertEqual(self.sc.range(1, 1).count(), 0)
self.assertEqual(self.sc.range(1, 0, -1).count(), 1)
self.assertEqual(self.sc.range(0, 1 << 40, 1 << 39).count(), 2)
def test_id(self):
rdd = self.sc.parallelize(range(10))
id = rdd.id()
self.assertEqual(id, rdd.id())
rdd2 = rdd.map(str).filter(bool)
id2 = rdd2.id()
self.assertEqual(id + 1, id2)
self.assertEqual(id2, rdd2.id())
def test_empty_rdd(self):
rdd = self.sc.emptyRDD()
self.assertTrue(rdd.isEmpty())
def test_sum(self):
self.assertEqual(0, self.sc.emptyRDD().sum())
self.assertEqual(6, self.sc.parallelize([1, 2, 3]).sum())
def test_to_localiterator(self):
rdd = self.sc.parallelize([1, 2, 3])
it = rdd.toLocalIterator()
self.assertEqual([1, 2, 3], sorted(it))
rdd2 = rdd.repartition(1000)
it2 = rdd2.toLocalIterator()
self.assertEqual([1, 2, 3], sorted(it2))
def test_to_localiterator_prefetch(self):
# Test that we fetch the next partition in parallel
# We do this by returning the current time and:
# reading the first elem, waiting, and reading the second elem
# If not in parallel then these would be at different times
# But since they are being computed in parallel we see the time
# is "close enough" to the same.
rdd = self.sc.parallelize(range(2), 2)
times1 = rdd.map(lambda x: datetime.now())
times2 = rdd.map(lambda x: datetime.now())
times_iter_prefetch = times1.toLocalIterator(prefetchPartitions=True)
times_iter = times2.toLocalIterator(prefetchPartitions=False)
times_prefetch_head = next(times_iter_prefetch)
times_head = next(times_iter)
time.sleep(2)
times_next = next(times_iter)
times_prefetch_next = next(times_iter_prefetch)
self.assertTrue(times_next - times_head >= timedelta(seconds=2))
self.assertTrue(times_prefetch_next - times_prefetch_head < timedelta(seconds=1))
def test_save_as_textfile_with_unicode(self):
# Regression test for SPARK-970
x = "\u00A1Hola, mundo!"
data = self.sc.parallelize([x])
tempFile = tempfile.NamedTemporaryFile(delete=True)
tempFile.close()
data.saveAsTextFile(tempFile.name)
raw_contents = b"".join(open(p, "rb").read() for p in glob(tempFile.name + "/part-0000*"))
self.assertEqual(x, raw_contents.strip().decode("utf-8"))
def test_save_as_textfile_with_utf8(self):
x = "\u00A1Hola, mundo!"
data = self.sc.parallelize([x.encode("utf-8")])
tempFile = tempfile.NamedTemporaryFile(delete=True)
tempFile.close()
data.saveAsTextFile(tempFile.name)
raw_contents = b"".join(open(p, "rb").read() for p in glob(tempFile.name + "/part-0000*"))
self.assertEqual(x, raw_contents.strip().decode("utf8"))
def test_transforming_cartesian_result(self):
# Regression test for SPARK-1034
rdd1 = self.sc.parallelize([1, 2])
rdd2 = self.sc.parallelize([3, 4])
cart = rdd1.cartesian(rdd2)
cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect()
def test_transforming_pickle_file(self):
# Regression test for SPARK-2601
data = self.sc.parallelize(["Hello", "World!"])
tempFile = tempfile.NamedTemporaryFile(delete=True)
tempFile.close()
data.saveAsPickleFile(tempFile.name)
pickled_file = self.sc.pickleFile(tempFile.name)
pickled_file.map(lambda x: x).collect()
def test_cartesian_on_textfile(self):
# Regression test for
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
a = self.sc.textFile(path)
result = a.cartesian(a).collect()
(x, y) = result[0]
self.assertEqual("Hello World!", x.strip())
self.assertEqual("Hello World!", y.strip())
def test_cartesian_chaining(self):
# Tests for SPARK-16589
rdd = self.sc.parallelize(range(10), 2)
self.assertSetEqual(
set(rdd.cartesian(rdd).cartesian(rdd).collect()),
set([((x, y), z) for x in range(10) for y in range(10) for z in range(10)]),
)
self.assertSetEqual(
set(rdd.cartesian(rdd.cartesian(rdd)).collect()),
set([(x, (y, z)) for x in range(10) for y in range(10) for z in range(10)]),
)
self.assertSetEqual(
set(rdd.cartesian(rdd.zip(rdd)).collect()),
set([(x, (y, y)) for x in range(10) for y in range(10)]),
)
def test_zip_chaining(self):
# Tests for SPARK-21985
rdd = self.sc.parallelize("abc", 2)
self.assertSetEqual(set(rdd.zip(rdd).zip(rdd).collect()), set([((x, x), x) for x in "abc"]))
self.assertSetEqual(set(rdd.zip(rdd.zip(rdd)).collect()), set([(x, (x, x)) for x in "abc"]))
def test_union_pair_rdd(self):
# SPARK-31788: test if pair RDDs can be combined by union.
rdd = self.sc.parallelize([1, 2])
pair_rdd = rdd.zip(rdd)
unionRDD = self.sc.union([pair_rdd, pair_rdd])
self.assertEqual(set(unionRDD.collect()), set([(1, 1), (2, 2), (1, 1), (2, 2)]))
self.assertEqual(unionRDD.count(), 4)
def test_deleting_input_files(self):
# Regression test for SPARK-1025
tempFile = tempfile.NamedTemporaryFile(delete=False)
tempFile.write(b"Hello World!")
tempFile.close()
data = self.sc.textFile(tempFile.name)
filtered_data = data.filter(lambda x: True)
self.assertEqual(1, filtered_data.count())
os.unlink(tempFile.name)
with QuietTest(self.sc):
self.assertRaises(Exception, lambda: filtered_data.count())
def test_sampling_default_seed(self):
# Test for SPARK-3995 (default seed setting)
data = self.sc.parallelize(range(1000), 1)
subset = data.takeSample(False, 10)
self.assertEqual(len(subset), 10)
def test_aggregate_mutable_zero_value(self):
# Test for SPARK-9021; uses aggregate and treeAggregate to build dict
# representing a counter of ints
from collections import defaultdict
# Show that single or multiple partitions work
data1 = self.sc.range(10, numSlices=1)
data2 = self.sc.range(10, numSlices=2)
def seqOp(x, y):
x[y] += 1
return x
def comboOp(x, y):
for key, val in y.items():
x[key] += val
return x
counts1 = data1.aggregate(defaultdict(int), seqOp, comboOp)
counts2 = data2.aggregate(defaultdict(int), seqOp, comboOp)
counts3 = data1.treeAggregate(defaultdict(int), seqOp, comboOp, 2)
counts4 = data2.treeAggregate(defaultdict(int), seqOp, comboOp, 2)
ground_truth = defaultdict(int, dict((i, 1) for i in range(10)))
self.assertEqual(counts1, ground_truth)
self.assertEqual(counts2, ground_truth)
self.assertEqual(counts3, ground_truth)
self.assertEqual(counts4, ground_truth)
def test_aggregate_by_key_mutable_zero_value(self):
# Test for SPARK-9021; uses aggregateByKey to make a pair RDD that
# contains lists of all values for each key in the original RDD
# list(range(...)) for Python 3.x compatibility (can't use * operator
# on a range object)
# list(zip(...)) for Python 3.x compatibility (want to parallelize a
# collection, not a zip object)
tuples = list(zip(list(range(10)) * 2, [1] * 20))
# Show that single or multiple partitions work
data1 = self.sc.parallelize(tuples, 1)
data2 = self.sc.parallelize(tuples, 2)
def seqOp(x, y):
x.append(y)
return x
def comboOp(x, y):
x.extend(y)
return x
values1 = data1.aggregateByKey([], seqOp, comboOp).collect()
values2 = data2.aggregateByKey([], seqOp, comboOp).collect()
# Sort lists to ensure clean comparison with ground_truth
values1.sort()
values2.sort()
ground_truth = [(i, [1] * 2) for i in range(10)]
self.assertEqual(values1, ground_truth)
self.assertEqual(values2, ground_truth)
def test_fold_mutable_zero_value(self):
# Test for SPARK-9021; uses fold to merge an RDD of dict counters into
# a single dict
from collections import defaultdict
counts1 = defaultdict(int, dict((i, 1) for i in range(10)))
counts2 = defaultdict(int, dict((i, 1) for i in range(3, 8)))
counts3 = defaultdict(int, dict((i, 1) for i in range(4, 7)))
counts4 = defaultdict(int, dict((i, 1) for i in range(5, 6)))
all_counts = [counts1, counts2, counts3, counts4]
# Show that single or multiple partitions work
data1 = self.sc.parallelize(all_counts, 1)
data2 = self.sc.parallelize(all_counts, 2)
def comboOp(x, y):
for key, val in y.items():
x[key] += val
return x
fold1 = data1.fold(defaultdict(int), comboOp)
fold2 = data2.fold(defaultdict(int), comboOp)
ground_truth = defaultdict(int)
for counts in all_counts:
for key, val in counts.items():
ground_truth[key] += val
self.assertEqual(fold1, ground_truth)
self.assertEqual(fold2, ground_truth)
def test_fold_by_key_mutable_zero_value(self):
# Test for SPARK-9021; uses foldByKey to make a pair RDD that contains
# lists of all values for each key in the original RDD
tuples = [(i, range(i)) for i in range(10)] * 2
# Show that single or multiple partitions work
data1 = self.sc.parallelize(tuples, 1)
data2 = self.sc.parallelize(tuples, 2)
def comboOp(x, y):
x.extend(y)
return x
values1 = data1.foldByKey([], comboOp).collect()
values2 = data2.foldByKey([], comboOp).collect()
# Sort lists to ensure clean comparison with ground_truth
values1.sort()
values2.sort()
# list(range(...)) for Python 3.x compatibility
ground_truth = [(i, list(range(i)) * 2) for i in range(10)]
self.assertEqual(values1, ground_truth)
self.assertEqual(values2, ground_truth)
def test_aggregate_by_key(self):
data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2)
def seqOp(x, y):
x.add(y)
return x
def combOp(x, y):
x |= y
return x
sets = dict(data.aggregateByKey(set(), seqOp, combOp).collect())
self.assertEqual(3, len(sets))
self.assertEqual(set([1]), sets[1])
self.assertEqual(set([2]), sets[3])
self.assertEqual(set([1, 3]), sets[5])
def test_itemgetter(self):
rdd = self.sc.parallelize([range(10)])
from operator import itemgetter
self.assertEqual([1], rdd.map(itemgetter(1)).collect())
self.assertEqual([(2, 3)], rdd.map(itemgetter(2, 3)).collect())
def test_namedtuple_in_rdd(self):
from collections import namedtuple
Person = namedtuple("Person", "id firstName lastName")
jon = Person(1, "Jon", "Doe")
jane = Person(2, "Jane", "Doe")
theDoes = self.sc.parallelize([jon, jane])
self.assertEqual([jon, jane], theDoes.collect())
def test_large_broadcast(self):
N = 10000
data = [[float(i) for i in range(300)] for i in range(N)]
bdata = self.sc.broadcast(data) # 27MB
m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
self.assertEqual(N, m)
def test_unpersist(self):
N = 1000
data = [[float(i) for i in range(300)] for i in range(N)]
bdata = self.sc.broadcast(data) # 3MB
bdata.unpersist()
m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
self.assertEqual(N, m)
bdata.destroy(blocking=True)
try:
self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum()
except Exception:
pass
else:
raise AssertionError("job should fail after destroy the broadcast")
def test_multiple_broadcasts(self):
N = 1 << 21
b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM
r = list(range(1 << 15))
random.shuffle(r)
s = str(r).encode()
checksum = hashlib.md5(s).hexdigest()
b2 = self.sc.broadcast(s)
r = list(
set(
self.sc.parallelize(range(10), 10)
.map(lambda x: (len(b1.value), hashlib.md5(b2.value).hexdigest()))
.collect()
)
)
self.assertEqual(1, len(r))
size, csum = r[0]
self.assertEqual(N, size)
self.assertEqual(checksum, csum)
random.shuffle(r)
s = str(r).encode()
checksum = hashlib.md5(s).hexdigest()
b2 = self.sc.broadcast(s)
r = list(
set(
self.sc.parallelize(range(10), 10)
.map(lambda x: (len(b1.value), hashlib.md5(b2.value).hexdigest()))
.collect()
)
)
self.assertEqual(1, len(r))
size, csum = r[0]
self.assertEqual(N, size)
self.assertEqual(checksum, csum)
def test_multithread_broadcast_pickle(self):
import threading
b1 = self.sc.broadcast(list(range(3)))
b2 = self.sc.broadcast(list(range(3)))
def f1():
return b1.value
def f2():
return b2.value
funcs_num_pickled = {f1: None, f2: None}
def do_pickle(f, sc):
command = (f, None, sc.serializer, sc.serializer)
ser = CloudPickleSerializer()
ser.dumps(command)
def process_vars(sc):
broadcast_vars = list(sc._pickled_broadcast_vars)
num_pickled = len(broadcast_vars)
sc._pickled_broadcast_vars.clear()
return num_pickled
def run(f, sc):
do_pickle(f, sc)
funcs_num_pickled[f] = process_vars(sc)
# pickle f1, adds b1 to sc._pickled_broadcast_vars in main thread local storage
do_pickle(f1, self.sc)
# run all for f2, should only add/count/clear b2 from worker thread local storage
t = threading.Thread(target=run, args=(f2, self.sc))
t.start()
t.join()
# count number of vars pickled in main thread, only b1 should be counted and cleared
funcs_num_pickled[f1] = process_vars(self.sc)
self.assertEqual(funcs_num_pickled[f1], 1)
self.assertEqual(funcs_num_pickled[f2], 1)
self.assertEqual(len(list(self.sc._pickled_broadcast_vars)), 0)
def test_large_closure(self):
N = 200000
data = [float(i) for i in range(N)]
rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data))
self.assertEqual(N, rdd.first())
# regression test for SPARK-6886
self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count())
def test_zip_with_different_serializers(self):
a = self.sc.parallelize(range(5))
b = self.sc.parallelize(range(100, 105))
self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)])
a = a._reserialize(BatchedSerializer(CPickleSerializer(), 2))
b = b._reserialize(MarshalSerializer())
self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)])
# regression test for SPARK-4841
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
t = self.sc.textFile(path)
cnt = t.count()
self.assertEqual(cnt, t.zip(t).count())
rdd = t.map(str)
self.assertEqual(cnt, t.zip(rdd).count())
# regression test for bug in _reserializer()
self.assertEqual(cnt, t.zip(rdd).count())
def test_zip_with_different_object_sizes(self):
# regress test for SPARK-5973
a = self.sc.parallelize(range(10000)).map(lambda i: "*" * i)
b = self.sc.parallelize(range(10000, 20000)).map(lambda i: "*" * i)
self.assertEqual(10000, a.zip(b).count())
def test_zip_with_different_number_of_items(self):
a = self.sc.parallelize(range(5), 2)
# different number of partitions
b = self.sc.parallelize(range(100, 106), 3)
self.assertRaises(ValueError, lambda: a.zip(b))
with QuietTest(self.sc):
# different number of batched items in JVM
b = self.sc.parallelize(range(100, 104), 2)
self.assertRaises(Exception, lambda: a.zip(b).count())
# different number of items in one pair
b = self.sc.parallelize(range(100, 106), 2)
self.assertRaises(Exception, lambda: a.zip(b).count())
# same total number of items, but different distributions
a = self.sc.parallelize([2, 3], 2).flatMap(range)
b = self.sc.parallelize([3, 2], 2).flatMap(range)
self.assertEqual(a.count(), b.count())
self.assertRaises(Exception, lambda: a.zip(b).count())
def test_count_approx_distinct(self):
rdd = self.sc.parallelize(range(1000))
self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050)
self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050)
self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050)
self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050)
rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7)
self.assertTrue(18 < rdd.countApproxDistinct() < 22)
self.assertTrue(18 < rdd.map(float).countApproxDistinct() < 22)
self.assertTrue(18 < rdd.map(str).countApproxDistinct() < 22)
self.assertTrue(18 < rdd.map(lambda x: (x, -x)).countApproxDistinct() < 22)
self.assertRaises(ValueError, lambda: rdd.countApproxDistinct(0.00000001))
def test_histogram(self):
# empty
rdd = self.sc.parallelize([])
self.assertEqual([0], rdd.histogram([0, 10])[1])
self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1])
self.assertRaises(ValueError, lambda: rdd.histogram(1))
# out of range
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEqual([0], rdd.histogram([0, 10])[1])
self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1])
# in range with one bucket
rdd = self.sc.parallelize(range(1, 5))
self.assertEqual([4], rdd.histogram([0, 10])[1])
self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1])
# in range with one bucket exact match
self.assertEqual([4], rdd.histogram([1, 4])[1])
# out of range with two buckets
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1])
# out of range with two uneven buckets
rdd = self.sc.parallelize([10.01, -0.01])
self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1])
# in range with two buckets
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1])
# in range with two bucket and None
rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float("nan")])
self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1])
# in range with two uneven buckets
rdd = self.sc.parallelize([1, 2, 3, 5, 6])
self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1])
# mixed range with two uneven buckets
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01])
self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1])
# mixed range with four uneven buckets
rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1])
self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
# mixed range with uneven buckets and NaN
rdd = self.sc.parallelize(
[-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, None, float("nan")]
)
self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1])
# out of range with infinite buckets
rdd = self.sc.parallelize([10.01, -0.01, float("nan"), float("inf")])
self.assertEqual([1, 2], rdd.histogram([float("-inf"), 0, float("inf")])[1])
# invalid buckets
self.assertRaises(ValueError, lambda: rdd.histogram([]))
self.assertRaises(ValueError, lambda: rdd.histogram([1]))
self.assertRaises(ValueError, lambda: rdd.histogram(0))
self.assertRaises(TypeError, lambda: rdd.histogram({}))
# without buckets
rdd = self.sc.parallelize(range(1, 5))
self.assertEqual(([1, 4], [4]), rdd.histogram(1))
# without buckets single element
rdd = self.sc.parallelize([1])
self.assertEqual(([1, 1], [1]), rdd.histogram(1))
# without bucket no range
rdd = self.sc.parallelize([1] * 4)
self.assertEqual(([1, 1], [4]), rdd.histogram(1))
# without buckets basic two
rdd = self.sc.parallelize(range(1, 5))
self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2))
# without buckets with more requested than elements
rdd = self.sc.parallelize([1, 2])
buckets = [1 + 0.2 * i for i in range(6)]
hist = [1, 0, 0, 0, 1]
self.assertEqual((buckets, hist), rdd.histogram(5))
# invalid RDDs
rdd = self.sc.parallelize([1, float("inf")])
self.assertRaises(ValueError, lambda: rdd.histogram(2))
rdd = self.sc.parallelize([float("nan")])
self.assertRaises(ValueError, lambda: rdd.histogram(2))
# string
rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2)
self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1])
self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1))
self.assertRaises(TypeError, lambda: rdd.histogram(2))
def test_repartitionAndSortWithinPartitions_asc(self):
rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2)
repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2, True)
partitions = repartitioned.glom().collect()
self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)])
self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)])
def test_repartitionAndSortWithinPartitions_desc(self):
rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2)
repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2, False)
partitions = repartitioned.glom().collect()
self.assertEqual(partitions[0], [(2, 6), (0, 5), (0, 8)])
self.assertEqual(partitions[1], [(3, 8), (3, 8), (1, 3)])
def test_repartition_no_skewed(self):
num_partitions = 20
a = self.sc.parallelize(range(int(1000)), 2)
xs = a.repartition(num_partitions).glom().map(len).collect()
zeros = len([x for x in xs if x == 0])
self.assertTrue(zeros == 0)
xs = a.coalesce(num_partitions, True).glom().map(len).collect()
zeros = len([x for x in xs if x == 0])
self.assertTrue(zeros == 0)
def test_repartition_on_textfile(self):
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
rdd = self.sc.textFile(path)
result = rdd.repartition(1).collect()
self.assertEqual("Hello World!", result[0])
def test_distinct(self):
rdd = self.sc.parallelize((1, 2, 3) * 10, 10)
self.assertEqual(rdd.getNumPartitions(), 10)
self.assertEqual(rdd.distinct().count(), 3)
result = rdd.distinct(5)
self.assertEqual(result.getNumPartitions(), 5)
self.assertEqual(result.count(), 3)
def test_external_group_by_key(self):
self.sc._conf.set("spark.python.worker.memory", "1m")
N = 2000001
kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x))
gkv = kv.groupByKey().cache()
self.assertEqual(3, gkv.count())
filtered = gkv.filter(lambda kv: kv[0] == 1)
self.assertEqual(1, filtered.count())
self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect())
self.assertEqual(
[(N // 3, N // 3)], filtered.values().map(lambda x: (len(x), len(list(x)))).collect()
)
result = filtered.collect()[0][1]
self.assertEqual(N // 3, len(result))
self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList))
def test_sort_on_empty_rdd(self):
self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect())
def test_sample(self):
rdd = self.sc.parallelize(range(0, 100), 4)
wo = rdd.sample(False, 0.1, 2).collect()
wo_dup = rdd.sample(False, 0.1, 2).collect()
self.assertSetEqual(set(wo), set(wo_dup))
wr = rdd.sample(True, 0.2, 5).collect()
wr_dup = rdd.sample(True, 0.2, 5).collect()
self.assertSetEqual(set(wr), set(wr_dup))
wo_s10 = rdd.sample(False, 0.3, 10).collect()
wo_s20 = rdd.sample(False, 0.3, 20).collect()
self.assertNotEqual(set(wo_s10), set(wo_s20))
wr_s11 = rdd.sample(True, 0.4, 11).collect()
wr_s21 = rdd.sample(True, 0.4, 21).collect()
self.assertNotEqual(set(wr_s11), set(wr_s21))
def test_null_in_rdd(self):
jrdd = self.sc._jvm.PythonUtils.generateRDDWithNull(self.sc._jsc)
rdd = RDD(jrdd, self.sc, UTF8Deserializer())
self.assertEqual(["a", None, "b"], rdd.collect())
rdd = RDD(jrdd, self.sc, NoOpSerializer())
self.assertEqual([b"a", None, b"b"], rdd.collect())
def test_multiple_python_java_RDD_conversions(self):
# Regression test for SPARK-5361
data = [("1", {"director": "David Lean"}), ("2", {"director": "Andrew Dominik"})]
data_rdd = self.sc.parallelize(data)
data_java_rdd = data_rdd._to_java_object_rdd()
data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd)
converted_rdd = RDD(data_python_rdd, self.sc)
self.assertEqual(2, converted_rdd.count())
# conversion between python and java RDD threw exceptions
data_java_rdd = converted_rdd._to_java_object_rdd()
data_python_rdd = self.sc._jvm.SerDeUtil.javaToPython(data_java_rdd)
converted_rdd = RDD(data_python_rdd, self.sc)
self.assertEqual(2, converted_rdd.count())
# Regression test for SPARK-6294
def test_take_on_jrdd(self):
rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x))
rdd._jrdd.first()
@unittest.skipIf(not have_numpy or not have_pandas, "NumPy or Pandas not installed")
def test_take_on_jrdd_with_large_rows_should_not_cause_deadlock(self):
# Regression test for SPARK-38677.
#
# Create a DataFrame with many columns, call a Python function on each row, and take only
# the first result row.
#
# This produces large rows that trigger a deadlock involving the following three threads:
#
# 1. The Scala task executor thread. During task execution, this is responsible for reading
# output produced by the Python process. However, in this case the task has finished
# early, and this thread is no longer reading output produced by the Python process.
# Instead, it is waiting for the Scala WriterThread to exit so that it can finish the
# task.
#
# 2. The Scala WriterThread. This is trying to send a large row to the Python process, and
# is waiting for the Python process to read that row.
#
# 3. The Python process. This is trying to send a large output to the Scala task executor
# thread, and is waiting for that thread to read that output.
#
# For this test to succeed rather than hanging, the Scala MonitorThread must detect this
# deadlock and kill the Python worker.
import numpy as np
import pandas as pd
num_rows = 100000
num_columns = 134
data = np.zeros((num_rows, num_columns))
columns = map(str, range(num_columns))
df = SparkSession(self.sc).createDataFrame(pd.DataFrame(data, columns=columns))
actual = CPickleSerializer().loads(df.rdd.map(list)._jrdd.first())
expected = [list(data[0])]
self.assertEqual(expected, actual)
def test_sortByKey_uses_all_partitions_not_only_first_and_last(self):
# Regression test for SPARK-5969
seq = [(i * 59 % 101, i) for i in range(101)] # unsorted sequence
rdd = self.sc.parallelize(seq)
for ascending in [True, False]:
sort = rdd.sortByKey(ascending=ascending, numPartitions=5)
self.assertEqual(sort.collect(), sorted(seq, reverse=not ascending))
sizes = sort.glom().map(len).collect()
for size in sizes:
self.assertGreater(size, 0)
def test_pipe_functions(self):
data = ["1", "2", "3"]
rdd = self.sc.parallelize(data)
with QuietTest(self.sc):
self.assertEqual([], rdd.pipe("java").collect())
self.assertRaises(Py4JJavaError, rdd.pipe("java", checkCode=True).collect)
result = rdd.pipe("cat").collect()
result.sort()
for x, y in zip(data, result):
self.assertEqual(x, y)
self.assertRaises(Py4JJavaError, rdd.pipe("grep 4", checkCode=True).collect)
self.assertEqual([], rdd.pipe("grep 4").collect())
def test_pipe_unicode(self):
# Regression test for SPARK-20947
data = ["\u6d4b\u8bd5", "1"]
rdd = self.sc.parallelize(data)
result = rdd.pipe("cat").collect()
self.assertEqual(data, result)
def test_stopiteration_in_user_code(self):
def stopit(*x):
raise StopIteration()
seq_rdd = self.sc.parallelize(range(10))
keyed_rdd = self.sc.parallelize((x % 2, x) for x in range(10))
msg = "Caught StopIteration thrown from user's code; failing the task"
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.map(stopit).collect)
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.filter(stopit).collect)
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.foreach, stopit)
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.reduce, stopit)
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.fold, 0, stopit)
self.assertRaisesRegex(Py4JJavaError, msg, seq_rdd.foreach, stopit)
self.assertRaisesRegex(
Py4JJavaError, msg, seq_rdd.cartesian(seq_rdd).flatMap(stopit).collect
)
# these methods call the user function both in the driver and in the executor
# the exception raised is different according to where the StopIteration happens
# RuntimeError is raised if in the driver
# Py4JJavaError is raised if in the executor (wraps the RuntimeError raised in the worker)
self.assertRaisesRegex(
(Py4JJavaError, RuntimeError), msg, keyed_rdd.reduceByKeyLocally, stopit
)
self.assertRaisesRegex(
(Py4JJavaError, RuntimeError), msg, seq_rdd.aggregate, 0, stopit, lambda *x: 1
)
self.assertRaisesRegex(
(Py4JJavaError, RuntimeError), msg, seq_rdd.aggregate, 0, lambda *x: 1, stopit
)
def test_overwritten_global_func(self):
# Regression test for SPARK-27000
global global_func
self.assertEqual(self.sc.parallelize([1]).map(lambda _: global_func()).first(), "Hi")
global_func = lambda: "Yeah" # noqa: E731
self.assertEqual(self.sc.parallelize([1]).map(lambda _: global_func()).first(), "Yeah")
def test_to_local_iterator_failure(self):
# SPARK-27548 toLocalIterator task failure not propagated to Python driver
def fail(_):
raise RuntimeError("local iterator error")
rdd = self.sc.range(10).map(fail)
with self.assertRaisesRegex(Exception, "local iterator error"):
for _ in rdd.toLocalIterator():
pass
def test_to_local_iterator_collects_single_partition(self):
# Test that partitions are not computed until requested by iteration
def fail_last(x):
if x == 9:
raise RuntimeError("This should not be hit")
return x
rdd = self.sc.range(12, numSlices=4).map(fail_last)
it = rdd.toLocalIterator()
# Only consume first 4 elements from partitions 1 and 2, this should not collect the last
# partition which would trigger the error
for i in range(4):
self.assertEqual(i, next(it))
def test_resourceprofile(self):
rp_builder = ResourceProfileBuilder()
ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g")
ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com")
treqs = TaskResourceRequests().cpus(2).resource("gpu", 2)
def assert_request_contents(exec_reqs, task_reqs):
self.assertEqual(len(exec_reqs), 5)
self.assertEqual(exec_reqs["cores"].amount, 2)
self.assertEqual(exec_reqs["memory"].amount, 6144)
self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024)
self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048)
self.assertEqual(exec_reqs["gpu"].amount, 2)
self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus")
self.assertEqual(exec_reqs["gpu"].resourceName, "gpu")
self.assertEqual(exec_reqs["gpu"].vendor, "nvidia.com")
self.assertEqual(len(task_reqs), 2)
self.assertEqual(task_reqs["cpus"].amount, 2.0)
self.assertEqual(task_reqs["gpu"].amount, 2.0)
assert_request_contents(ereqs.requests, treqs.requests)
rp = rp_builder.require(ereqs).require(treqs).build
assert_request_contents(rp.executorResources, rp.taskResources)
rdd = self.sc.parallelize(range(10)).withResources(rp)
return_rp = rdd.getResourceProfile()
assert_request_contents(return_rp.executorResources, return_rp.taskResources)
rddWithoutRp = self.sc.parallelize(range(10))
self.assertEqual(rddWithoutRp.getResourceProfile(), None)
def test_multiple_group_jobs(self):
import threading
group_a = "job_ids_to_cancel"
group_b = "job_ids_to_run"
threads = []
thread_ids = range(4)
thread_ids_to_cancel = [i for i in thread_ids if i % 2 == 0]
thread_ids_to_run = [i for i in thread_ids if i % 2 != 0]
# A list which records whether job is cancelled.
# The index of the array is the thread index which job run in.
is_job_cancelled = [False for _ in thread_ids]
def run_job(job_group, index):
"""
Executes a job with the group ``job_group``. Each job waits for 3 seconds
and then exits.
"""
try:
self.sc.parallelize([15]).map(lambda x: time.sleep(x)).collectWithJobGroup(
job_group, "test rdd collect with setting job group"
)
is_job_cancelled[index] = False
except Exception:
# Assume that exception means job cancellation.
is_job_cancelled[index] = True
# Test if job succeeded when not cancelled.
run_job(group_a, 0)
self.assertFalse(is_job_cancelled[0])
# Run jobs
for i in thread_ids_to_cancel:
t = threading.Thread(target=run_job, args=(group_a, i))
t.start()
threads.append(t)
for i in thread_ids_to_run:
t = threading.Thread(target=run_job, args=(group_b, i))
t.start()
threads.append(t)
# Wait to make sure all jobs are executed.
time.sleep(3)
# And then, cancel one job group.
self.sc.cancelJobGroup(group_a)
# Wait until all threads launching jobs are finished.
for t in threads:
t.join()
for i in thread_ids_to_cancel:
self.assertTrue(
is_job_cancelled[i], "Thread {i}: Job in group A was not cancelled.".format(i=i)
)
for i in thread_ids_to_run:
self.assertFalse(
is_job_cancelled[i], "Thread {i}: Job in group B did not succeeded.".format(i=i)
)
if __name__ == "__main__":
import unittest
from pyspark.tests.test_rdd import * # noqa: F401
try:
import xmlrunner # type: ignore[import]
testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
except ImportError:
testRunner = None
unittest.main(testRunner=testRunner, verbosity=2)