-
Notifications
You must be signed in to change notification settings - Fork 29.2k
Expand file tree
/
Copy pathtest_session.py
More file actions
281 lines (225 loc) · 9.23 KB
/
test_session.py
File metadata and controls
281 lines (225 loc) · 9.23 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
#
# 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.
#
import threading
import time
import unittest
from typing import Optional
from pyspark import InheritableThread, inheritable_thread_target
from pyspark.sql.connect.client import DefaultChannelBuilder
from pyspark.sql.connect.session import SparkSession as RemoteSparkSession
from pyspark.testing.connectutils import should_test_connect
if should_test_connect:
from pyspark.testing.connectutils import ReusedConnectTestCase
class CustomChannelBuilder(DefaultChannelBuilder):
@property
def userId(self) -> Optional[str]:
return "abc"
class SparkSessionTestCase(unittest.TestCase):
def test_fails_to_create_session_without_remote_and_channel_builder(self):
with self.assertRaises(ValueError):
RemoteSparkSession.builder.getOrCreate()
def test_fails_to_create_when_both_remote_and_channel_builder_are_specified(self):
with self.assertRaises(ValueError):
(
RemoteSparkSession.builder.channelBuilder(CustomChannelBuilder("sc://localhost"))
.remote("sc://localhost")
.getOrCreate()
)
def test_creates_session_with_channel_builder(self):
test_session = RemoteSparkSession.builder.channelBuilder(
CustomChannelBuilder("sc://other")
).getOrCreate()
host = test_session.client.host
test_session.stop()
self.assertEqual("other", host)
def test_creates_session_with_remote(self):
test_session = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
host = test_session.client.host
test_session.stop()
self.assertEqual("other", host)
def test_session_stop(self):
session = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
self.assertFalse(session.is_stopped)
session.stop()
self.assertTrue(session.is_stopped)
def test_session_create_sets_active_session(self):
session = RemoteSparkSession.builder.remote("sc://abc").create()
session2 = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
self.assertIs(session, session2)
session.stop()
def test_active_session_expires_when_client_closes(self):
s1 = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
s2 = RemoteSparkSession.getActiveSession()
self.assertIs(s1, s2)
# We don't call close() to avoid executing ExecutePlanResponseReattachableIterator
s1._client._closed = True
self.assertIsNone(RemoteSparkSession.getActiveSession())
s3 = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
self.assertIsNot(s1, s3)
def test_default_session_expires_when_client_closes(self):
s1 = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
s2 = RemoteSparkSession.getDefaultSession()
self.assertIs(s1, s2)
# We don't call close() to avoid executing ExecutePlanResponseReattachableIterator
s1._client._closed = True
self.assertIsNone(RemoteSparkSession.getDefaultSession())
s3 = RemoteSparkSession.builder.remote("sc://other").getOrCreate()
self.assertIsNot(s1, s3)
class JobCancellationTests(ReusedConnectTestCase):
def test_tags(self):
self.spark.clearTags()
self.spark.addTag("a")
self.assertEqual(self.spark.getTags(), {"a"})
self.spark.addTag("b")
self.spark.removeTag("a")
self.assertEqual(self.spark.getTags(), {"b"})
self.spark.addTag("c")
self.spark.clearTags()
self.assertEqual(self.spark.getTags(), set())
self.spark.clearTags()
def test_tags_multithread(self):
output1 = None
output2 = None
def tag1():
nonlocal output1
self.spark.addTag("tag1")
output1 = self.spark.getTags()
def tag2():
nonlocal output2
self.spark.addTag("tag2")
output2 = self.spark.getTags()
t1 = threading.Thread(target=tag1)
t1.start()
t1.join()
t2 = threading.Thread(target=tag2)
t2.start()
t2.join()
self.assertIsNotNone(output1)
self.assertEquals(output1, {"tag1"})
self.assertIsNotNone(output2)
self.assertEquals(output2, {"tag2"})
def test_interrupt_tag(self):
thread_ids = range(4)
self.check_job_cancellation(
lambda job_group: self.spark.addTag(job_group),
lambda job_group: self.spark.interruptTag(job_group),
thread_ids,
[i for i in thread_ids if i % 2 == 0],
[i for i in thread_ids if i % 2 != 0],
)
self.spark.clearTags()
def test_interrupt_all(self):
thread_ids = range(4)
self.check_job_cancellation(
lambda job_group: None,
lambda job_group: self.spark.interruptAll(),
thread_ids,
thread_ids,
[],
)
self.spark.clearTags()
def check_job_cancellation(
self, setter, canceller, thread_ids, thread_ids_to_cancel, thread_ids_to_run
):
job_id_a = "job_ids_to_cancel"
job_id_b = "job_ids_to_run"
threads = []
# 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_id, index):
"""
Executes a job with the group ``job_group``. Each job waits for 3 seconds
and then exits.
"""
try:
setter(job_id)
def func(itr):
for pdf in itr:
time.sleep(pdf._1.iloc[0])
yield pdf
self.spark.createDataFrame([[20]]).repartition(1).mapInPandas(
func, schema="_1 LONG"
).collect()
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(job_id_a, 0)
self.assertFalse(is_job_cancelled[0])
self.spark.clearTags()
# Run jobs
for i in thread_ids_to_cancel:
t = threading.Thread(target=run_job, args=(job_id_a, i))
t.start()
threads.append(t)
for i in thread_ids_to_run:
t = threading.Thread(target=run_job, args=(job_id_b, i))
t.start()
threads.append(t)
# Wait to make sure all jobs are executed.
time.sleep(10)
# And then, cancel one job group.
canceller(job_id_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)
)
def test_inheritable_tags(self):
self.check_inheritable_tags(
create_thread=lambda target, session: InheritableThread(target, session=session)
)
self.check_inheritable_tags(
create_thread=lambda target, session: threading.Thread(
target=inheritable_thread_target(session)(target)
)
)
# Test decorator usage
@inheritable_thread_target(self.spark)
def func(target):
return target()
self.check_inheritable_tags(
create_thread=lambda target, session: threading.Thread(target=func, args=(target,))
)
def check_inheritable_tags(self, create_thread):
spark = self.spark
spark.addTag("a")
first = set()
second = set()
def get_inner_local_prop():
spark.addTag("c")
second.update(spark.getTags())
def get_outer_local_prop():
spark.addTag("b")
first.update(spark.getTags())
t2 = create_thread(target=get_inner_local_prop, session=spark)
t2.start()
t2.join()
t1 = create_thread(target=get_outer_local_prop, session=spark)
t1.start()
t1.join()
self.assertEqual(spark.getTags(), {"a"})
self.assertEqual(first, {"a", "b"})
self.assertEqual(second, {"a", "b", "c"})