def run(self, pipeline): """Remotely executes entire pipeline or parts reachable from node.""" # Import here to avoid adding the dependency for local running scenarios. # pylint: disable=wrong-import-order, wrong-import-position from apache_beam.internal import apiclient self.job = apiclient.Job(pipeline.options) # The superclass's run will trigger a traversal of all reachable nodes. super(DataflowRunner, self).run(pipeline) standard_options = pipeline.options.view_as(StandardOptions) if standard_options.streaming: job_version = DataflowRunner.STREAMING_ENVIRONMENT_MAJOR_VERSION else: job_version = DataflowRunner.BATCH_ENVIRONMENT_MAJOR_VERSION # Get a Dataflow API client and set its options self.dataflow_client = apiclient.DataflowApplicationClient( pipeline.options, job_version) # Create the job self.result = DataflowPipelineResult( self.dataflow_client.create_job(self.job), self) if self.result.has_job and self.blocking: self.result.wait_until_finish() return self.result
def test_remote_runner_translation(self): remote_runner = DataflowRunner() p = Pipeline(remote_runner, options=PipelineOptions(self.default_properties)) (p | ptransform.Create([1, 2, 3]) # pylint: disable=expression-not-assigned | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)]) | ptransform.GroupByKey()) remote_runner.job = apiclient.Job(p.options) super(DataflowRunner, remote_runner).run(p)
def test_remote_runner_display_data(self): remote_runner = DataflowRunner() p = Pipeline(remote_runner, options=PipelineOptions(self.default_properties)) # TODO: Should not subclass ParDo. Switch to PTransform as soon as # composite transforms support display data. class SpecialParDo(beam.ParDo): def __init__(self, fn, now): super(SpecialParDo, self).__init__(fn) self.fn = fn self.now = now # Make this a list to be accessible within closure def display_data(self): return {'asubcomponent': self.fn, 'a_class': SpecialParDo, 'a_time': self.now} class SpecialDoFn(beam.DoFn): def display_data(self): return {'dofn_value': 42} def process(self): pass now = datetime.now() # pylint: disable=expression-not-assigned (p | ptransform.Create([1, 2, 3, 4, 5]) | 'Do' >> SpecialParDo(SpecialDoFn(), now)) remote_runner.job = apiclient.Job(p.options) super(DataflowRunner, remote_runner).run(p) job_dict = json.loads(str(remote_runner.job)) steps = [step for step in job_dict['steps'] if len(step['properties'].get('display_data', [])) > 0] step = steps[0] disp_data = step['properties']['display_data'] disp_data = sorted(disp_data, key=lambda x: x['namespace']+x['key']) nspace = SpecialParDo.__module__+ '.' expected_data = [{'type': 'TIMESTAMP', 'namespace': nspace+'SpecialParDo', 'value': DisplayDataItem._format_value(now, 'TIMESTAMP'), 'key': 'a_time'}, {'type': 'STRING', 'namespace': nspace+'SpecialParDo', 'value': nspace+'SpecialParDo', 'key': 'a_class', 'shortValue': 'SpecialParDo'}, {'type': 'INTEGER', 'namespace': nspace+'SpecialDoFn', 'value': 42, 'key': 'dofn_value'}] expected_data = sorted(expected_data, key=lambda x: x['namespace']+x['key']) self.assertEqual(len(disp_data), 3) self.assertEqual(disp_data, expected_data)
def test_bad_path(self): dummy_sdk_file = tempfile.NamedTemporaryFile() remote_runner = DataflowRunner() pipeline = Pipeline( remote_runner, options=PipelineOptions([ '--dataflow_endpoint=ignored', '--sdk_location=' + dummy_sdk_file.name, '--job_name=test-job', '--project=test-project', '--staging_location=ignored', '--temp_location=/dev/null', '--template_location=/bad/path', '--no_auth=True' ])) remote_runner.job = apiclient.Job(pipeline.options) with self.assertRaises(IOError): pipeline.run().wait_until_finish()