[GitHub] [beam] robertwb commented on a change in pull request #12185: [BEAM-10409] Add combiner packing to graph optimizer phases

2020-08-25 Thread GitBox


robertwb commented on a change in pull request #12185:
URL: https://github.com/apache/beam/pull/12185#discussion_r476702785



##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -690,6 +692,200 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+coder.component_coder_ids) == 2:
+  return coder.component_coder_ids[index]
+return _get_fallback_coder_id()
+
+  def _get_key_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 0)
+
+  def _get_value_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 1)
+
+  def _try_fuse_stages(a, b):
+if a.can_fuse(b, context):
+  return a.fuse(b)
+else:
+  raise ValueError
+
+  # Group stages by parent and environment, yielding ineligible stages.
+  combine_stages_by_input_pcoll_id = collections.defaultdict(list)
+  for stage in stages:
+is_packable_combine = False
+
+if (len(stage.transforms) == 1 and
+stage.environment is not None and
+python_urns.PACKED_COMBINE_FN in
+context.components.environments[stage.environment].capabilities):
+  transform = only_transform(stage.transforms)
+  if (transform.spec.urn == common_urns.composites.COMBINE_PER_KEY.urn and
+  len(transform.inputs) == 1 and
+  len(transform.outputs) == 1):
+combine_payload = proto_utils.parse_Bytes(
+transform.spec.payload, beam_runner_api_pb2.CombinePayload)
+if combine_payload.combine_fn.urn == python_urns.PICKLED_COMBINE_FN:
+  is_packable_combine = True
+
+if is_packable_combine:
+  input_pcoll_id = only_element(transform.inputs.values())
+  stage_key = (input_pcoll_id, stage.environment)
+  combine_stages_by_input_pcoll_id[stage_key].append(stage)
+else:
+  yield stage
+
+  for stage_key, packable_stages in combine_stages_by_input_pcoll_id.items():
+input_pcoll_id, _ = stage_key
+try:
+  if not len(packable_stages) > 1:
+raise ValueError('Only one stage in this group: Skipping stage 
packing')
+  # Fused stage is used as template and is not yielded.
+  fused_stage = functools.reduce(_try_fuse_stages, packable_stages)
+except ValueError:
+  # Skip packing stages in this group.
+  # Yield the stages unmodified, and then continue to the next group.
+  for stage in packable_stages:
+yield stage
+  continue
+
+transforms = [only_transform(stage.transforms) for stage in 
packable_stages]
+combine_payloads = [
+proto_utils.parse_Bytes(transform.spec.payload,
+beam_runner_api_pb2.CombinePayload)
+for transform in transforms
+]
+output_pcoll_ids = [
+only_element(transform.outputs.values()) for transform in transforms
+]
+
+# Build accumulator coder for (acc1, acc2, ...)
+accumulator_coder_ids = [
+combine_payload.accumulator_coder_id
+for combine_payload in combine_payloads
+]
+tuple_accumulator_coder_id = context.add_or_get_coder_id(
+beam_runner_api_pb2.Coder(
+spec=beam_runner_api_pb2.FunctionSpec(
+urn=common_urns.coders.KV.urn),

Review comment:
   Now use `python_urns.TUPLE_CODER` here rather than the KV one.

##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -818,7 +819,7 @@ def _try_fuse_stages(a, b):
 for output_kv_coder_id in output_kv_coder_ids
 ]
 pack_output_value_coder = beam_runner_api_pb2.Coder(
-

[GitHub] [beam] robertwb commented on a change in pull request #12185: [BEAM-10409] Add combiner packing to graph optimizer phases

2020-08-19 Thread GitBox


robertwb commented on a change in pull request #12185:
URL: https://github.com/apache/beam/pull/12185#discussion_r470936066



##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -690,6 +692,199 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+coder.component_coder_ids) == 2:
+  return coder.component_coder_ids[index]
+return _get_fallback_coder_id()
+
+  def _get_key_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 0)
+
+  def _get_value_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 1)
+
+  def _try_fuse_stages(a, b):
+if a.can_fuse(b, context):
+  return a.fuse(b)
+else:
+  raise ValueError
+
+  # Group stages by parent, yielding ineligible stages.
+  combine_stages_by_input_pcoll_id = collections.defaultdict(list)
+  for stage in stages:
+is_packable_combine = False
+
+if (len(stage.transforms) == 1 and 
+stage.environment is not None and
+python_urns.PACKED_COMBINE_FN in
+context.components.environments[stage.environment].capabilities):
+  transform = only_transform(stage.transforms)
+  if (transform.spec.urn == common_urns.composites.COMBINE_PER_KEY.urn and
+  len(transform.inputs) == 1 and
+  len(transform.outputs) == 1):
+combine_payload = proto_utils.parse_Bytes(
+transform.spec.payload, beam_runner_api_pb2.CombinePayload)
+if combine_payload.combine_fn.urn == python_urns.PICKLED_COMBINE_FN:
+  is_packable_combine = True
+
+if is_packable_combine:
+  input_pcoll_id = only_element(transform.inputs.values())
+  combine_stages_by_input_pcoll_id[input_pcoll_id].append(stage)

Review comment:
   Perhaps key by environment and input pcollection (to not even attempt to 
fuse across environments). 

##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -690,6 +692,196 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+coder.component_coder_ids) == 2:
+  return coder.component_coder_ids[index]
+return _get_fallback_coder_id()
+
+  def _get_key_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 0)
+
+ 

[GitHub] [beam] robertwb commented on a change in pull request #12185: [BEAM-10409] Add combiner packing to graph optimizer phases

2020-08-10 Thread GitBox


robertwb commented on a change in pull request #12185:
URL: https://github.com/apache/beam/pull/12185#discussion_r468102004



##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -690,6 +692,196 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+coder.component_coder_ids) == 2:
+  return coder.component_coder_ids[index]
+return _get_fallback_coder_id()
+
+  def _get_key_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 0)
+
+  def _get_value_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 1)
+
+  def _try_fuse_stages(a, b):
+if a.can_fuse(b, context):
+  return a.fuse(b)
+else:
+  raise ValueError
+
+  # Group stages by parent, yielding ineligible stages.
+  combine_stages_by_input_pcoll_id = collections.defaultdict(list)
+  for stage in stages:
+transform = only_transform(stage.transforms)
+if transform.spec.urn == common_urns.composites.COMBINE_PER_KEY.urn and 
len(
+transform.inputs) == 1 and len(transform.outputs) == 1:
+  input_pcoll_id = only_element(transform.inputs.values())
+  combine_stages_by_input_pcoll_id[input_pcoll_id].append(stage)
+else:
+  yield stage
+
+  for input_pcoll_id, packable_stages in 
combine_stages_by_input_pcoll_id.items(
+  ):
+# Yield stage and continue if it has no siblings.
+if len(packable_stages) == 1:
+  yield packable_stages[0]
+  continue
+
+transforms = [only_transform(stage.transforms) for stage in 
packable_stages]
+combine_payloads = [
+proto_utils.parse_Bytes(transform.spec.payload,
+beam_runner_api_pb2.CombinePayload)
+for transform in transforms
+]
+
+# Yield stages and continue if they cannot be packed.
+try:
+  # Fused stage is used as template and is not yielded.
+  fused_stage = functools.reduce(_try_fuse_stages, packable_stages)
+  merged_transform_environment_id = functools.reduce(

Review comment:
   Can't we pull this off of the fused_stage?

##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -690,6 +692,196 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+

[GitHub] [beam] robertwb commented on a change in pull request #12185: [BEAM-10409] Add combiner packing to graph optimizer phases

2020-07-08 Thread GitBox


robertwb commented on a change in pull request #12185:
URL: https://github.com/apache/beam/pull/12185#discussion_r451894237



##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##
@@ -289,6 +289,8 @@ def create_stages(
 phases=[
 translations.annotate_downstream_side_inputs,
 translations.fix_side_input_pcoll_coders,
+translations.eliminate_common_siblings,

Review comment:
   We could be able to annotate/recognize certain DoFns (e.g. by URN) for 
which this deduplication would be safe to apply. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] robertwb commented on a change in pull request #12185: [BEAM-10409] Add combiner packing to graph optimizer phases

2020-07-08 Thread GitBox


robertwb commented on a change in pull request #12185:
URL: https://github.com/apache/beam/pull/12185#discussion_r451758742



##
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
##
@@ -685,6 +687,264 @@ def fix_side_input_pcoll_coders(stages, pipeline_context):
   return stages
 
 
+def eliminate_common_siblings(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterable[Stage]
+  """Runs common subexpression elimination for common siblings.
+
+  If stages have common input, an identical transform, and one output each,
+  then all but one stages will be eliminated, and the output of the remaining
+  will be connected to the original output PCollections of the eliminated
+  stages. This elimination runs only once, not recursively, and will only
+  eliminate the first stage after a common input, rather than a chain of
+  stages.
+  """
+
+  SiblingKey = collections.namedtuple(
+  'SiblingKey', ['spec_urn', 'spec_payload', 'inputs', 'environment_id'])
+
+  def get_sibling_key(transform):
+"""Returns a key that will be identical for common siblings."""
+transform_output_keys = list(transform.outputs.keys())
+# Return None as the sibling key for ineligible transforms.
+if len(transform_output_keys
+  ) != 1 or transform.spec.urn != common_urns.primitives.PAR_DO.urn:
+  return None
+return SiblingKey(
+spec_urn=transform.spec.urn,
+spec_payload=transform.spec.payload,
+inputs=tuple(transform.inputs.items()),
+environment_id=transform.environment_id)
+
+  # Group stages by keys.
+  stages_by_sibling_key = collections.defaultdict(list)
+  for stage in stages:
+transform = only_transform(stage.transforms)
+stages_by_sibling_key[get_sibling_key(transform)].append(stage)
+
+  # Eliminate stages and build the output PCollection remapping dictionary.
+  pcoll_id_remap = {}
+  for sibling_key, sibling_stages in stages_by_sibling_key.items():
+if sibling_key is None or len(sibling_stages) == 1:
+  continue
+output_pcoll_ids = [
+only_element(stage.transforms[0].outputs.values())
+for stage in sibling_stages
+]
+to_delete_pcoll_ids = output_pcoll_ids[1:]
+for to_delete_pcoll_id in to_delete_pcoll_ids:
+  pcoll_id_remap[to_delete_pcoll_id] = output_pcoll_ids[0]
+  del context.components.pcollections[to_delete_pcoll_id]
+del sibling_stages[1:]
+
+  # Yield stages while remapping output PCollections if needed.
+  for sibling_key, sibling_stages in stages_by_sibling_key.items():
+for stage in sibling_stages:
+  input_keys_to_remap = []
+  for input_key, input_pcoll_id in stage.transforms[0].inputs.items():
+if input_pcoll_id in pcoll_id_remap:
+  input_keys_to_remap.append(input_key)
+  for input_key_to_remap in input_keys_to_remap:
+stage.transforms[0].inputs[input_key_to_remap] = pcoll_id_remap[
+stage.transforms[0].inputs[input_key_to_remap]]
+  yield stage
+
+
+def pack_combiners(stages, context):
+  # type: (Iterable[Stage], TransformContext) -> Iterator[Stage]
+  """Packs sibling CombinePerKey stages into a single CombinePerKey.
+
+  If CombinePerKey stages have a common input, one input each, and one output
+  each, pack the stages into a single stage that runs all CombinePerKeys and
+  outputs resulting tuples to a new PCollection. A subsequent stage unpacks
+  tuples from this PCollection and sends them to the original output
+  PCollections.
+  """
+
+  class _UnpackFn(core.DoFn):
+"""A DoFn that unpacks a packed to multiple tagged outputs.
+
+Example:
+  tags = (T1, T2, ...)
+  input = (K, (V1, V2, ...))
+  output = TaggedOutput(T1, (K, V1)), TaggedOutput(T2, (K, V1)), ...
+"""
+
+def __init__(self, tags):
+  self._tags = tags
+
+def process(self, element):
+  key, values = element
+  return [
+  core.pvalue.TaggedOutput(tag, (key, value))
+  for tag, value in zip(self._tags, values)
+  ]
+
+  def _get_fallback_coder_id():
+return context.add_or_get_coder_id(
+coders.registry.get_coder(object).to_runner_api(None))
+
+  def _get_component_coder_id_from_kv_coder(coder, index):
+assert index < 2
+if coder.spec.urn == common_urns.coders.KV.urn and len(
+coder.component_coder_ids) == 2:
+  return coder.component_coder_ids[index]
+return _get_fallback_coder_id()
+
+  def _get_key_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 0)
+
+  def _get_value_coder_id_from_kv_coder(coder):
+return _get_component_coder_id_from_kv_coder(coder, 1)
+
+  def _try_fuse_stages(a, b):
+if a.can_fuse(b, context):
+  return a.fuse(b)
+else:
+  raise ValueError
+
+  def _try_merge_environments(env1, env2):

Review comment:
   Was this copied from above? If needed, perhaps refactor? (Similarly for 
try_fuse_stages.)

##