Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 55e1031

Browse files
authored
Various render runner improvements. (#28285)
* Error message rather than silence when no rendering requested. * Fix render_leaf_composite_nodes to act on names rather than ids. * Allow rendering of raw .dot files when graphviz is not installed.
1 parent 7c1c250 commit 55e1031

3 files changed

Lines changed: 49 additions & 3 deletions

File tree

sdks/python/apache_beam/io/iobase.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -908,7 +908,8 @@ def expand(self, pbegin):
908908
return (
909909
pbegin
910910
| Impulse()
911-
| core.Map(lambda _: self.source).with_output_types(BoundedSource)
911+
| 'EmitSource' >>
912+
core.Map(lambda _: self.source).with_output_types(BoundedSource)
912913
| SDFBoundedSourceReader(display_data))
913914
elif isinstance(self.source, ptransform.PTransform):
914915
# The Read transform can also admit a full PTransform as an input

sdks/python/apache_beam/runners/render.py

Lines changed: 34 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -129,6 +129,12 @@ def _add_argparse_args(cls, parser):
129129
help='Set to also log input pipeline proto to stdout.')
130130
return parser
131131

132+
def __init__(self, *args, render_testing=False, **kwargs):
133+
super().__init__(*args, **kwargs)
134+
if self.render_port < 0 and not self.render_output and not render_testing:
135+
raise ValueError(
136+
'At least one of --render_port or --render_output must be provided.')
137+
132138

133139
class PipelineRenderer:
134140
def __init__(self, pipeline, options):
@@ -147,8 +153,10 @@ def __init__(self, pipeline, options):
147153

148154
# Figure out at what point to stop rendering composite internals.
149155
if options.render_leaf_composite_nodes:
150-
is_leaf = lambda name: any(
151-
re.match(pattern, name)
156+
is_leaf = lambda transform_id: any(
157+
re.match(
158+
pattern,
159+
self.pipeline.components.transforms[transform_id].unique_name)
152160
for patterns in options.render_leaf_composite_nodes
153161
for pattern in patterns.split(','))
154162
self.leaf_composites = set()
@@ -403,6 +411,30 @@ def run_pipeline(self, pipeline_object, options, pipeline_proto=None):
403411
if render_options.log_proto:
404412
logging.info(pipeline_proto)
405413
renderer = PipelineRenderer(pipeline_proto, render_options)
414+
try:
415+
subprocess.run(['dotX', '-V'], capture_output=True, check=True)
416+
except FileNotFoundError as exn:
417+
# If dot is not available, we can at least output the raw .dot files.
418+
dot_files = [
419+
output for output in render_options.render_output
420+
if output.endswith('.dot')
421+
]
422+
for output in dot_files:
423+
with open(output, 'w') as fout:
424+
fout.write(renderer.to_dot())
425+
logging.info("Wrote pipeline as %s", output)
426+
427+
non_dot_files = set(render_options.render_output) - set(dot_files)
428+
if non_dot_files:
429+
raise RuntimeError(
430+
"Graphviz dot executable not available "
431+
f"for rendering non-dot output files {non_dot_files}") from exn
432+
elif render_options.render_port >= 0:
433+
raise RuntimeError(
434+
"Graphviz dot executable not available for serving") from exn
435+
436+
return RenderPipelineResult(None)
437+
406438
renderer.page()
407439

408440
if render_options.render_port >= 0:

sdks/python/apache_beam/runners/render_test.py

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -83,6 +83,19 @@ def test_dot_well_formed(self):
8383
renderer.update(toggle=[create_transform_id])
8484
renderer.render_data()
8585

86+
def test_leaf_composite_filter(self):
87+
try:
88+
subprocess.run(['dot', '-V'], capture_output=True, check=True)
89+
except FileNotFoundError:
90+
self.skipTest('dot executable not installed')
91+
p = beam.Pipeline()
92+
_ = p | beam.Create([1, 2, 3]) | beam.Map(lambda x: x * x)
93+
dot = render.PipelineRenderer(
94+
p.to_runner_api(),
95+
render.RenderOptions(['--render_leaf_composite_nodes=Create'],
96+
render_testing=True)).to_dot()
97+
self.assertEqual(dot.count('->'), 1)
98+
8699

87100
if __name__ == '__main__':
88101
logging.getLogger().setLevel(logging.INFO)

0 commit comments

Comments
 (0)