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

Skip to content

Commit c01d630

Browse files
committed
Add the ability to pass a pipeline proto directly.
1 parent c1d82db commit c01d630

1 file changed

Lines changed: 45 additions & 0 deletions

File tree

sdks/python/apache_beam/runners/render.py

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,9 @@
3838
If `--render_port` is set to a non-negative value, a local http server will
3939
be started which allows for interactive exploration of the pipeline graph.
4040
41+
As an alternative to starting a job server, a single pipeline can be rendered
42+
by passing a pipeline proto file to `--pipeline_proto`.
43+
4144
Requires the that graphviz dot executable be available in the path.
4245
"""
4346

@@ -56,7 +59,11 @@
5659
import time
5760
import urllib.parse
5861

62+
from google.protobuf import json_format
63+
from google.protobuf import text_format
64+
5965
from apache_beam.options import pipeline_options
66+
from apache_beam.portability.api import beam_runner_api_pb2
6067
from apache_beam.runners import runner
6168
from apache_beam.runners.portability import local_job_service
6269
from apache_beam.runners.portability import local_job_service_main
@@ -438,9 +445,47 @@ def run(argv):
438445
type=int,
439446
default=0,
440447
help='port on which to serve the job api')
448+
parser.add_argument(
449+
'--pipeline_proto', help='file containing the beam pipeline definition')
441450
RenderOptions._add_argparse_args(parser)
442451
options = parser.parse_args(argv)
443452

453+
if options.pipeline_proto:
454+
render_one(options)
455+
else:
456+
run_server(options)
457+
458+
459+
def render_one(options):
460+
if options.pipeline_proto == '-':
461+
content = sys.stdin.buffer.read()
462+
if content[0] == b'{':
463+
ext = '.json'
464+
else:
465+
try:
466+
content.decode('utf-8')
467+
ext = '.textproto'
468+
except UnicodeDecodeError:
469+
ext = '.pb'
470+
else:
471+
with open(options.pipeline_proto, 'rb') as fin:
472+
content = fin.read()
473+
ext = os.path.splitext(options.pipeline_proto)[-1]
474+
475+
if ext == '.textproto':
476+
pipeline_proto = text_format.Parse(content, beam_runner_api_pb2.Pipeline())
477+
elif ext == '.json':
478+
pipeline_proto = json_format.Parse(content, beam_runner_api_pb2.Pipeline())
479+
else:
480+
pipeline_proto = beam_runner_api_pb2.Pipeline.ParseFromString(content)
481+
482+
RenderRunner().run_pipeline(
483+
None,
484+
pipeline_options.PipelineOptions(**vars(options)),
485+
self._pipeline_proto)
486+
487+
488+
def run_server(options):
444489
class RenderBeamJob(local_job_service.BeamJob):
445490
def _invoke_runner(self):
446491
return RenderRunner().run_pipeline(

0 commit comments

Comments
 (0)