def get_proto_pipeline(pipeline, options): # type: (Pipeline, PipelineOptions) -> beam_runner_api_pb2.Pipeline portable_options = options.view_as(PortableOptions) proto_pipeline = pipeline.to_runner_api( default_environment=PortableRunner._create_environment( portable_options)) # Preemptively apply combiner lifting, until all runners support it. # These optimizations commute and are idempotent. pre_optimize = options.view_as(DebugOptions).lookup_experiment( 'pre_optimize', 'lift_combiners').lower() if not options.view_as(StandardOptions).streaming: flink_known_urns = frozenset([ common_urns.composites.RESHUFFLE.urn, common_urns.primitives.IMPULSE.urn, common_urns.primitives.FLATTEN.urn, common_urns.primitives.GROUP_BY_KEY.urn ]) if pre_optimize == 'none': pass elif pre_optimize == 'all': proto_pipeline = translations.optimize_pipeline( proto_pipeline, phases=[ translations.annotate_downstream_side_inputs, translations.annotate_stateful_dofns_as_roots, translations.fix_side_input_pcoll_coders, translations.eliminate_common_key_with_none, translations.pack_combiners, translations.lift_combiners, translations.expand_sdf, translations.fix_flatten_coders, # fn_api_runner_transforms.sink_flattens, translations.greedily_fuse, translations.read_to_impulse, translations.extract_impulse_stages, translations.remove_data_plane_ops, translations.sort_stages ], known_runner_urns=flink_known_urns) else: phases = [] for phase_name in pre_optimize.split(','): # For now, these are all we allow. if phase_name in 'lift_combiners': phases.append(getattr(translations, phase_name)) else: raise ValueError( 'Unknown or inapplicable phase for pre_optimize: %s' % phase_name) proto_pipeline = translations.optimize_pipeline( proto_pipeline, phases=phases, known_runner_urns=flink_known_urns, partial=True) return proto_pipeline
def get_proto_pipeline(pipeline, options): # type: (Pipeline, PipelineOptions) -> beam_runner_api_pb2.Pipeline portable_options = options.view_as(PortableOptions) proto_pipeline = pipeline.to_runner_api( default_environment=PortableRunner._create_environment( portable_options)) # TODO: https://github.com/apache/beam/issues/19493 # Eventually remove the 'pre_optimize' option alltogether and only perform # the equivalent of the 'default' case below (minus the 'lift_combiners' # part). pre_optimize = options.view_as(DebugOptions).lookup_experiment( 'pre_optimize', 'default').lower() if (not options.view_as(StandardOptions).streaming and pre_optimize != 'none'): if pre_optimize == 'default': phases = [ # TODO: https://github.com/apache/beam/issues/18584 # https://github.com/apache/beam/issues/18586 # Eventually remove the 'lift_combiners' phase from 'default'. translations.pack_combiners, translations.lift_combiners, translations.sort_stages ] partial = True elif pre_optimize == 'all': phases = [ translations.annotate_downstream_side_inputs, translations.annotate_stateful_dofns_as_roots, translations.fix_side_input_pcoll_coders, translations.pack_combiners, translations.lift_combiners, translations.expand_sdf, translations.fix_flatten_coders, # translations.sink_flattens, translations.greedily_fuse, translations.read_to_impulse, translations.extract_impulse_stages, translations.remove_data_plane_ops, translations.sort_stages ] partial = False elif pre_optimize == 'all_except_fusion': # TODO(BEAM-7248): Delete this branch after PortableRunner supports # beam:runner:executable_stage:v1. phases = [ translations.annotate_downstream_side_inputs, translations.annotate_stateful_dofns_as_roots, translations.fix_side_input_pcoll_coders, translations.pack_combiners, translations.lift_combiners, translations.expand_sdf, translations.fix_flatten_coders, # translations.sink_flattens, # translations.greedily_fuse, translations.read_to_impulse, translations.extract_impulse_stages, translations.remove_data_plane_ops, translations.sort_stages ] partial = True else: phases = [] for phase_name in pre_optimize.split(','): # For now, these are all we allow. if phase_name in ('pack_combiners', 'lift_combiners'): phases.append(getattr(translations, phase_name)) else: raise ValueError( 'Unknown or inapplicable phase for pre_optimize: %s' % phase_name) phases.append(translations.sort_stages) partial = True # All (known) portable runners (ie Flink and Spark) support these URNs. known_urns = frozenset([ common_urns.composites.RESHUFFLE.urn, common_urns.primitives.IMPULSE.urn, common_urns.primitives.FLATTEN.urn, common_urns.primitives.GROUP_BY_KEY.urn ]) proto_pipeline = translations.optimize_pipeline( proto_pipeline, phases=phases, known_runner_urns=known_urns, partial=partial) return proto_pipeline