-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
portable_runner_test.py
353 lines (304 loc) · 12.7 KB
/
portable_runner_test.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
#
# 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 __future__ import absolute_import
from __future__ import print_function
import inspect
import json
import logging
import platform
import signal
import socket
import subprocess
import sys
import threading
import time
import traceback
import unittest
import grpc
import apache_beam as beam
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import DirectOptions
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import PortableOptions
from apache_beam.portability import python_urns
from apache_beam.portability.api import beam_job_api_pb2
from apache_beam.portability.api import beam_job_api_pb2_grpc
from apache_beam.runners.portability import fn_api_runner_test
from apache_beam.runners.portability import portable_runner
from apache_beam.runners.portability.local_job_service import LocalJobServicer
from apache_beam.runners.portability.portable_runner import PortableRunner
from apache_beam.runners.worker import worker_pool_main
from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
from apache_beam.transforms import environments
class PortableRunnerTest(fn_api_runner_test.FnApiRunnerTest):
TIMEOUT_SECS = 60
# Controls job service interaction, not sdk harness interaction.
_use_subprocesses = False
def setUp(self):
if platform.system() != 'Windows':
def handler(signum, frame):
msg = 'Timed out after %s seconds.' % self.TIMEOUT_SECS
print('=' * 20, msg, '=' * 20)
traceback.print_stack(frame)
threads_by_id = {th.ident: th for th in threading.enumerate()}
for thread_id, stack in sys._current_frames().items():
th = threads_by_id.get(thread_id)
print()
print('# Thread:', th or thread_id)
traceback.print_stack(stack)
raise BaseException(msg)
signal.signal(signal.SIGALRM, handler)
signal.alarm(self.TIMEOUT_SECS)
def tearDown(self):
if platform.system() != 'Windows':
signal.alarm(0)
@classmethod
def _pick_unused_port(cls):
return cls._pick_unused_ports(num_ports=1)[0]
@staticmethod
def _pick_unused_ports(num_ports):
"""Not perfect, but we have to provide a port to the subprocess."""
# TODO(robertwb): Consider letting the subprocess communicate a choice of
# port back.
sockets = []
ports = []
for _ in range(0, num_ports):
s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
sockets.append(s)
s.bind(('localhost', 0))
_, port = s.getsockname()
ports.append(port)
try:
return ports
finally:
for s in sockets:
s.close()
@classmethod
def _start_local_runner_subprocess_job_service(cls):
cls._maybe_kill_subprocess()
# TODO(robertwb): Consider letting the subprocess pick one and
# communicate it back...
# pylint: disable=unbalanced-tuple-unpacking
job_port, expansion_port = cls._pick_unused_ports(num_ports=2)
logging.info('Starting server on port %d.', job_port)
cls._subprocess = subprocess.Popen(
cls._subprocess_command(job_port, expansion_port))
address = 'localhost:%d' % job_port
job_service = beam_job_api_pb2_grpc.JobServiceStub(
GRPCChannelFactory.insecure_channel(address))
logging.info('Waiting for server to be ready...')
start = time.time()
timeout = 30
while True:
time.sleep(0.1)
if cls._subprocess.poll() is not None:
raise RuntimeError(
'Subprocess terminated unexpectedly with exit code %d.' %
cls._subprocess.returncode)
elif time.time() - start > timeout:
raise RuntimeError(
'Pipeline timed out waiting for job service subprocess.')
else:
try:
job_service.GetState(
beam_job_api_pb2.GetJobStateRequest(job_id='[fake]'))
break
except grpc.RpcError as exn:
if exn.code() != grpc.StatusCode.UNAVAILABLE:
# We were able to contact the service for our fake state request.
break
logging.info('Server ready.')
return address
@classmethod
def _get_job_endpoint(cls):
if '_job_endpoint' not in cls.__dict__:
cls._job_endpoint = cls._create_job_endpoint()
return cls._job_endpoint
@classmethod
def _create_job_endpoint(cls):
if cls._use_subprocesses:
return cls._start_local_runner_subprocess_job_service()
else:
cls._servicer = LocalJobServicer()
return 'localhost:%d' % cls._servicer.start_grpc_server()
@classmethod
def get_runner(cls):
return portable_runner.PortableRunner()
@classmethod
def tearDownClass(cls):
cls._maybe_kill_subprocess()
@classmethod
def _maybe_kill_subprocess(cls):
if hasattr(cls, '_subprocess') and cls._subprocess.poll() is None:
cls._subprocess.kill()
time.sleep(0.1)
def create_options(self):
def get_pipeline_name():
for _, _, _, method_name, _, _ in inspect.stack():
if method_name.find('test') != -1:
return method_name
return 'unknown_test'
# Set the job name for better debugging.
options = PipelineOptions.from_dictionary({
'job_name': get_pipeline_name() + '_' + str(time.time())
})
options.view_as(PortableOptions).job_endpoint = self._get_job_endpoint()
# Override the default environment type for testing.
options.view_as(PortableOptions).environment_type = (
python_urns.EMBEDDED_PYTHON)
# Enable caching (disabled by default)
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
return options
def create_pipeline(self):
return beam.Pipeline(self.get_runner(), self.create_options())
def test_metrics(self):
self.skipTest('Metrics not supported.')
# Inherits all other tests from fn_api_runner_test.FnApiRunnerTest
@unittest.skip("BEAM-7248")
class PortableRunnerOptimized(PortableRunnerTest):
def create_options(self):
options = super(PortableRunnerOptimized, self).create_options()
options.view_as(DebugOptions).add_experiment('pre_optimize=all')
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
return options
class PortableRunnerTestWithExternalEnv(PortableRunnerTest):
@classmethod
def setUpClass(cls):
cls._worker_address, cls._worker_server = (
worker_pool_main.BeamFnExternalWorkerPoolServicer.start(
state_cache_size=100))
@classmethod
def tearDownClass(cls):
cls._worker_server.stop(1)
def create_options(self):
options = super(PortableRunnerTestWithExternalEnv, self).create_options()
options.view_as(PortableOptions).environment_type = 'EXTERNAL'
options.view_as(PortableOptions).environment_config = self._worker_address
return options
class PortableRunnerTestWithSubprocesses(PortableRunnerTest):
_use_subprocesses = True
def create_options(self):
options = super(PortableRunnerTestWithSubprocesses, self).create_options()
options.view_as(PortableOptions).environment_type = (
python_urns.SUBPROCESS_SDK)
options.view_as(PortableOptions).environment_config = (
b'%s -m apache_beam.runners.worker.sdk_worker_main' %
sys.executable.encode('ascii')).decode('utf-8')
# Enable caching (disabled by default)
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
return options
@classmethod
def _subprocess_command(cls, job_port, _):
return [
sys.executable,
'-m', 'apache_beam.runners.portability.local_job_service_main',
'-p', str(job_port),
]
class PortableRunnerTestWithSubprocessesAndMultiWorkers(
PortableRunnerTestWithSubprocesses):
_use_subprocesses = True
def create_options(self):
options = super(PortableRunnerTestWithSubprocessesAndMultiWorkers, self) \
.create_options()
options.view_as(DirectOptions).direct_num_workers = 2
return options
class PortableRunnerInternalTest(unittest.TestCase):
def test__create_default_environment(self):
docker_image = PortableRunner.default_docker_image()
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({})),
environments.DockerEnvironment(container_image=docker_image))
def test__create_docker_environment(self):
docker_image = 'py-docker'
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': 'DOCKER',
'environment_config': docker_image,
})), environments.DockerEnvironment(container_image=docker_image))
def test__create_docker_environment_json_config(self):
docker_image = 'py-docker'
env_map = {'ENVKEY1': 'ENVVALUE1'}
config_map = {'docker_image': docker_image, 'env': env_map}
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': 'DOCKER',
'environment_config': json.dumps(config_map),
})), environments.DockerEnvironment(container_image=docker_image,
env=env_map))
with self.assertRaises(ValueError):
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': 'DOCKER',
'environment_config': json.dumps({'env': env_map}),
}))
def test__create_process_environment(self):
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': "PROCESS",
'environment_config': '{"os": "linux", "arch": "amd64", '
'"command": "run.sh", '
'"env":{"k1": "v1"} }',
})), environments.ProcessEnvironment('run.sh', os='linux', arch='amd64',
env={'k1': 'v1'}))
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': 'PROCESS',
'environment_config': '{"command": "run.sh"}',
})), environments.ProcessEnvironment('run.sh'))
def test__create_external_environment(self):
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': "EXTERNAL",
'environment_config': 'localhost:50000',
})), environments.ExternalEnvironment('localhost:50000'))
raw_config = ' {"url":"localhost:50000", "params":{"k1":"v1"} '
for env_config in (raw_config, raw_config.lstrip(), raw_config.strip()):
self.assertEqual(
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': "EXTERNAL",
'environment_config': env_config,
})), environments.ExternalEnvironment('localhost:50000',
params={"k1":"v1"}))
with self.assertRaises(ValueError):
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': "EXTERNAL",
'environment_config': '{invalid}',
}))
with self.assertRaises(ValueError) as ctx:
PortableRunner._create_environment(PipelineOptions.from_dictionary({
'environment_type': "EXTERNAL",
'environment_config': '{"params":{"k1":"v1"}}',
}))
self.assertIn(
'External environment endpoint must be set.', ctx.exception.args)
def hasDockerImage():
image = PortableRunner.default_docker_image()
try:
check_image = subprocess.check_output("docker images -q %s" % image,
shell=True)
return check_image != ''
except Exception:
return False
@unittest.skipIf(not hasDockerImage(), "docker not installed or "
"no docker image")
class PortableRunnerTestWithLocalDocker(PortableRunnerTest):
def create_options(self):
options = super(PortableRunnerTestWithLocalDocker, self).create_options()
options.view_as(PortableOptions).job_endpoint = 'embed'
return options
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()