[
https://issues.apache.org/jira/browse/BEAM-10708?focusedWorklogId=575152&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-575152
]
ASF GitHub Bot logged work on BEAM-10708:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 31/Mar/21 21:25
Start Date: 31/Mar/21 21:25
Worklog Time Spent: 10m
Work Description: KevinGG commented on a change in pull request #14368:
URL: https://github.com/apache/beam/pull/14368#discussion_r605230817
##########
File path: sdks/python/apache_beam/runners/interactive/caching/write_cache.py
##########
@@ -0,0 +1,170 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Module to write cache for PCollections being computed.
+
+For internal use only; no backward-compatibility guarantees.
+"""
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+from typing import Tuple
+
+import apache_beam as beam
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.interactive import cache_manager as cache
+from apache_beam.runners.interactive.caching.cacheable import Cacheable
+from apache_beam.runners.pipeline_context import PipelineContext
+from apache_beam.testing import test_stream
+from apache_beam.transforms.ptransform import PTransform
+from apache_beam.transforms.window import WindowedValue
+
+
+class WriteCache:
+ """Class that facilitates writing cache for PCollections being computed.
+ """
+ def __init__(
+ self,
+ pipeline: beam_runner_api_pb2.Pipeline,
+ context: PipelineContext,
+ cache_manager: cache.CacheManager,
+ cacheable: Cacheable):
+ self._pipeline = pipeline
+ self._context = context
+ self._cache_manager = cache_manager
+ self._cacheable = cacheable
+ self._key = repr(cacheable.to_key())
+ self._label = '{}{}'.format('_cache_', self._key)
+
+ def write_cache(self) -> None:
+ """Writes cache for the cacheable PCollection that is being computed.
+ """
+ template, write_input_placeholder = self._build_runner_api_template()
+ input_placeholder_id = self._context.pcollections.get_id(
+ write_input_placeholder.placeholder_pcoll)
+ input_id = self._context.pcollections.get_id(self._cacheable.pcoll)
+
+ # Copy cache writing subgraph from the template to the pipeline proto.
+ for pcoll_id in template.components.pcollections:
Review comment:
This I can explain:
TL;DR: the `component_id_map` of a pipeline instance (also backs the
pipeline context) is where each Python object is assigned a unique
pipeline-scoped id. By making a shallow copy of it before augmenting and
passing it into the temporary pipelines as a shared context for new components,
we can guarantee all ids are unique even if they come from different pipeline
instances.
- The underlying mechanism of the `component_id_map` are:
- a dictionary holds counters of different object types as `Dict[type,
int]`;
- a dictionary holds object to id map as `Dict[Any, str]`.
- All the temporary pipelines will generate components as if a sub graph in
the original pipeline: ids of all the components are unique even to the current
state of the augmented pipeline.
- The code moves the top level transforms of that subgraph into the runner
api proto of the pipeline being augmented.
- The `pcoll_id in self._pipeline.components.pcollections ...` checks to
skip ids from the shared context (`component_id_map`) that definitely do not
exist in the temp pipeline and do not need copy at all.
- The `pcoll_id in write_input_placeholder.ignorable_components.pcollections
...` checks to skip unnecessary root transforms (because you don't need two
root transforms) and placeholder subgraphs (as if they are pruned when we add
cache transforms) since we only care about top level transforms that we need.
It's kind of like moving everything there then pruning the unnecessary parts.
- The copy should preserve the topological ordering since it reads the proto
of the temp pipeline in its original order and does the same thing to
`set_proto_map` in `pipeline_instrument`.
--
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 575152)
Time Spent: 15h 40m (was: 15.5h)
> InteractiveRunner cannot execute pipeline with cross-language transform
> -----------------------------------------------------------------------
>
> Key: BEAM-10708
> URL: https://issues.apache.org/jira/browse/BEAM-10708
> Project: Beam
> Issue Type: Bug
> Components: cross-language
> Reporter: Brian Hulette
> Assignee: Ning Kang
> Priority: P2
> Time Spent: 15h 40m
> Remaining Estimate: 0h
>
> The InteractiveRunner crashes when given a pipeline that includes a
> cross-language transform.
> Here's the example I tried to run in a jupyter notebook:
> {code:python}
> p = beam.Pipeline(InteractiveRunner())
> pc = (p | SqlTransform("""SELECT
> CAST(1 AS INT) AS `id`,
> CAST('foo' AS VARCHAR) AS `str`,
> CAST(3.14 AS DOUBLE) AS `flt`"""))
> df = interactive_beam.collect(pc)
> {code}
> The problem occurs when
> [pipeline_fragment.py|https://github.com/apache/beam/blob/dce1eb83b8d5137c56ac58568820c24bd8fda526/sdks/python/apache_beam/runners/interactive/pipeline_fragment.py#L66]
> creates a copy of the pipeline by [writing it to proto and reading it
> back|https://github.com/apache/beam/blob/dce1eb83b8d5137c56ac58568820c24bd8fda526/sdks/python/apache_beam/runners/interactive/pipeline_fragment.py#L120].
> Reading it back fails because some of the pipeline is not written in Python.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)