|
38 | 38 | If `--render_port` is set to a non-negative value, a local http server will |
39 | 39 | be started which allows for interactive exploration of the pipeline graph. |
40 | 40 |
|
| 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 | +
|
41 | 44 | Requires the that graphviz dot executable be available in the path. |
42 | 45 | """ |
43 | 46 |
|
|
56 | 59 | import time |
57 | 60 | import urllib.parse |
58 | 61 |
|
| 62 | +from google.protobuf import json_format |
| 63 | +from google.protobuf import text_format |
| 64 | + |
59 | 65 | from apache_beam.options import pipeline_options |
| 66 | +from apache_beam.portability.api import beam_runner_api_pb2 |
60 | 67 | from apache_beam.runners import runner |
61 | 68 | from apache_beam.runners.portability import local_job_service |
62 | 69 | from apache_beam.runners.portability import local_job_service_main |
@@ -438,9 +445,47 @@ def run(argv): |
438 | 445 | type=int, |
439 | 446 | default=0, |
440 | 447 | help='port on which to serve the job api') |
| 448 | + parser.add_argument( |
| 449 | + '--pipeline_proto', help='file containing the beam pipeline definition') |
441 | 450 | RenderOptions._add_argparse_args(parser) |
442 | 451 | options = parser.parse_args(argv) |
443 | 452 |
|
| 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): |
444 | 489 | class RenderBeamJob(local_job_service.BeamJob): |
445 | 490 | def _invoke_runner(self): |
446 | 491 | return RenderRunner().run_pipeline( |
|
0 commit comments