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

Skip to content

Commit d559544

Browse files
Ning KangTheNeuralBit
andauthored
[BEAM-10708] Clean up pipeline instrument (#14176)
* [BEAM-10708] Clean up pipeline instrument 1. Moved cacheable and cachekey to dataclasses. 2. Cleaned up logic of finding the user defined pipeline. 3. Fixed missing add_derived_pipeline in background_caching_job module. * add python_version<3.7 to the dataclasses dep comment in setup * Add python_version<"3.7" to dataclasses requirement Co-authored-by: Brian Hulette <[email protected]>
1 parent 3014959 commit d559544

9 files changed

Lines changed: 110 additions & 106 deletions

File tree

sdks/python/apache_beam/runners/interactive/background_caching_job.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,7 @@ def attempt_to_run_background_caching_job(
141141
from apache_beam.runners.interactive import pipeline_instrument as instr
142142
runner_pipeline = beam.pipeline.Pipeline.from_runner_api(
143143
user_pipeline.to_runner_api(use_fake_coders=True), runner, options)
144+
ie.current_env().add_derived_pipeline(user_pipeline, runner_pipeline)
144145
background_caching_job_result = beam.pipeline.Pipeline.from_runner_api(
145146
instr.build_pipeline_instrument(
146147
runner_pipeline).background_caching_pipeline_proto(),
Lines changed: 78 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,78 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#
17+
18+
"""Module for dataclasses to hold metadata of cacheable PCollections in the user
19+
code scope.
20+
21+
For internal use only; no backwards-compatibility guarantees.
22+
"""
23+
24+
# pytype: skip-file
25+
26+
from __future__ import absolute_import
27+
28+
from dataclasses import dataclass
29+
30+
import apache_beam as beam
31+
from apache_beam.runners.interactive.utils import obfuscate
32+
33+
34+
@dataclass
35+
class Cacheable:
36+
pcoll_id: str
37+
var: str
38+
version: str
39+
pcoll: beam.pvalue.PCollection
40+
producer_version: str
41+
42+
def __hash__(self):
43+
return hash((
44+
self.pcoll_id,
45+
self.var,
46+
self.version,
47+
self.pcoll,
48+
self.producer_version))
49+
50+
def to_key(self):
51+
return CacheKey(
52+
self.var,
53+
self.version,
54+
self.producer_version,
55+
str(id(self.pcoll.pipeline)))
56+
57+
58+
@dataclass
59+
class CacheKey:
60+
var: str
61+
version: str
62+
producer_version: str
63+
pipeline_id: str
64+
65+
def __post_init__(self):
66+
# Normalize arbitrary variable name to a fixed length hex str.
67+
self.var = obfuscate(self.var)[:10]
68+
69+
@staticmethod
70+
def from_str(r):
71+
r_split = r.split('-')
72+
ck = CacheKey(*r_split)
73+
ck.var = r_split[0]
74+
return ck
75+
76+
def __repr__(self):
77+
return '-'.join(
78+
[self.var, self.version, self.producer_version, self.pipeline_id])

sdks/python/apache_beam/runners/interactive/caching/streaming_cache.py

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
4040
from apache_beam.runners.interactive.cache_manager import CacheManager
4141
from apache_beam.runners.interactive.cache_manager import SafeFastPrimitivesCoder
42+
from apache_beam.runners.interactive.caching.cacheable import CacheKey
4243
from apache_beam.testing.test_stream import OutputFormat
4344
from apache_beam.testing.test_stream import ReverseTestStream
4445
from apache_beam.utils import timestamp
@@ -156,8 +157,6 @@ def __init__(self, cache_dir, labels, is_cache_complete=None, coder=None):
156157
self._labels = labels
157158
self._path = os.path.join(self._cache_dir, *self._labels)
158159
self._is_cache_complete = is_cache_complete
159-
160-
from apache_beam.runners.interactive.pipeline_instrument import CacheKey
161160
self._pipeline_id = CacheKey.from_str(labels[-1]).pipeline_id
162161

163162
def _wait_until_file_exists(self, timeout_secs=30):
@@ -168,7 +167,6 @@ def _wait_until_file_exists(self, timeout_secs=30):
168167
while not os.path.exists(self._path):
169168
time.sleep(1)
170169
if time.time() - start > timeout_secs:
171-
from apache_beam.runners.interactive.pipeline_instrument import CacheKey
172170
pcollection_var = CacheKey.from_str(self._labels[-1]).var
173171
raise RuntimeError(
174172
'Timed out waiting for cache file for PCollection `{}` to be '

sdks/python/apache_beam/runners/interactive/caching/streaming_cache_test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,8 @@
2727
from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileRecord
2828
from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
2929
from apache_beam.runners.interactive.cache_manager import SafeFastPrimitivesCoder
30+
from apache_beam.runners.interactive.caching.cacheable import CacheKey
3031
from apache_beam.runners.interactive.caching.streaming_cache import StreamingCache
31-
from apache_beam.runners.interactive.pipeline_instrument import CacheKey
3232
from apache_beam.runners.interactive.testing.test_cache_manager import FileRecordsBuilder
3333
from apache_beam.testing.test_pipeline import TestPipeline
3434
from apache_beam.testing.test_stream import TestStream

sdks/python/apache_beam/runners/interactive/interactive_runner.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -190,7 +190,7 @@ def visit_transform(self, transform_node):
190190

191191
if not self._skip_display:
192192
a_pipeline_graph = pipeline_graph.PipelineGraph(
193-
pipeline_instrument.original_pipeline,
193+
pipeline_instrument.original_pipeline_proto,
194194
render_option=self._render_option)
195195
a_pipeline_graph.display_graph()
196196

sdks/python/apache_beam/runners/interactive/pipeline_instrument.py

Lines changed: 18 additions & 101 deletions
Original file line numberDiff line numberDiff line change
@@ -32,65 +32,15 @@
3232
from apache_beam.runners.interactive import interactive_environment as ie
3333
from apache_beam.runners.interactive import pipeline_fragment as pf
3434
from apache_beam.runners.interactive import background_caching_job
35-
from apache_beam.runners.interactive.utils import obfuscate
35+
from apache_beam.runners.interactive.caching.cacheable import Cacheable
36+
from apache_beam.runners.interactive.caching.cacheable import CacheKey
3637
from apache_beam.testing import test_stream
3738
from apache_beam.transforms.window import WindowedValue
3839

3940
READ_CACHE = "_ReadCache_"
4041
WRITE_CACHE = "_WriteCache_"
4142

4243

43-
# TODO: turn this into a dataclass object when we finally get off of Python2.
44-
class Cacheable:
45-
def __init__(self, pcoll_id, var, version, pcoll, producer_version):
46-
self.pcoll_id = pcoll_id
47-
self.var = var
48-
self.version = version
49-
self.pcoll = pcoll
50-
self.producer_version = producer_version
51-
52-
def __eq__(self, other):
53-
return (
54-
self.pcoll_id == other.pcoll_id and self.var == other.var and
55-
self.version == other.version and self.pcoll == other.pcoll and
56-
self.producer_version == other.producer_version)
57-
58-
def __hash__(self):
59-
return hash((
60-
self.pcoll_id,
61-
self.var,
62-
self.version,
63-
self.pcoll,
64-
self.producer_version))
65-
66-
def to_key(self):
67-
return CacheKey(
68-
self.var,
69-
self.version,
70-
self.producer_version,
71-
str(id(self.pcoll.pipeline)))
72-
73-
74-
# TODO: turn this into a dataclass object when we finally get off of Python2.
75-
class CacheKey:
76-
def __init__(self, var, version, producer_version, pipeline_id):
77-
# Makes sure that the variable name is obfuscated and only first 10
78-
# characters taken so that the CacheKey has a constant length.
79-
self.var = obfuscate(var)[:10]
80-
self.version = version
81-
self.producer_version = producer_version
82-
self.pipeline_id = pipeline_id
83-
84-
@staticmethod
85-
def from_str(r):
86-
split = r.split('-')
87-
return CacheKey(split[0], split[1], split[2], split[3])
88-
89-
def __repr__(self):
90-
return '-'.join(
91-
[self.var, self.version, self.producer_version, self.pipeline_id])
92-
93-
9444
class PipelineInstrument(object):
9545
"""A pipeline instrument for pipeline to be executed by interactive runner.
9646
@@ -103,36 +53,34 @@ class PipelineInstrument(object):
10353
"""
10454
def __init__(self, pipeline, options=None):
10555
self._pipeline = pipeline
106-
# The cache manager per user-defined pipeline is lazily initiated the first
107-
# time accessed. It is owned by interactive_environment module. This
108-
# shortcut reference will be initialized when the user pipeline associated
109-
# to the given pipeline is identified.
110-
self._cache_manager = None
111-
112-
# Invoke a round trip through the runner API. This makes sure the Pipeline
113-
# proto is stable. The snapshot of pipeline will not be mutated within this
114-
# module and can be used to recover original pipeline if needed.
115-
self._pipeline_snap = beam.pipeline.Pipeline.from_runner_api(
116-
pipeline.to_runner_api(use_fake_coders=True), pipeline.runner, options)
117-
ie.current_env().add_derived_pipeline(self._pipeline, self._pipeline_snap)
56+
57+
self._user_pipeline = ie.current_env().user_pipeline(pipeline)
58+
if not self._user_pipeline:
59+
self._user_pipeline = pipeline
60+
self._cache_manager = ie.current_env().get_cache_manager(
61+
self._user_pipeline, create_if_absent=True)
62+
# Check if the user defined pipeline contains any source to cache.
63+
# If so, during the check, the cache manager is converted into a
64+
# streaming cache manager, thus re-assign.
65+
if background_caching_job.has_source_to_cache(self._user_pipeline):
66+
self._cache_manager = ie.current_env().get_cache_manager(
67+
self._user_pipeline)
11868

11969
self._background_caching_pipeline = beam.pipeline.Pipeline.from_runner_api(
12070
pipeline.to_runner_api(use_fake_coders=True), pipeline.runner, options)
12171
ie.current_env().add_derived_pipeline(
12272
self._pipeline, self._background_caching_pipeline)
12373

12474
# Snapshot of original pipeline information.
125-
(self._original_pipeline_proto,
126-
self._original_context) = self._pipeline_snap.to_runner_api(
127-
return_context=True, use_fake_coders=True)
75+
(self._original_pipeline_proto, context) = self._pipeline.to_runner_api(
76+
return_context=True, use_fake_coders=True)
12877

12978
# All compute-once-against-original-pipeline fields.
13079
self._unbounded_sources = unbounded_sources(
13180
self._background_caching_pipeline)
13281
# TODO(BEAM-7760): once cache scope changed, this is not needed to manage
13382
# relationships across pipelines, runners, and jobs.
134-
self._pcolls_to_pcoll_id = pcolls_to_pcoll_id(
135-
self._pipeline_snap, self._original_context)
83+
self._pcolls_to_pcoll_id = pcolls_to_pcoll_id(self._pipeline, context)
13684

13785
# A mapping from PCollection id to python id() value in user defined
13886
# pipeline instance.
@@ -149,11 +97,6 @@ def __init__(self, pipeline, options=None):
14997
# (Dict[str, AppliedPTransform]).
15098
self._cached_pcoll_read = {}
15199

152-
# Reference to the user defined pipeline instance based on the given
153-
# pipeline. The class never mutates it.
154-
# Note: the original pipeline is not the user pipeline.
155-
self._user_pipeline = None
156-
157100
# A dict from PCollections in the runner pipeline instance to their
158101
# corresponding PCollections in the user pipeline instance. Populated
159102
# after preprocess().
@@ -421,15 +364,9 @@ def pcolls_to_pcoll_id(self):
421364

422365
@property
423366
def original_pipeline_proto(self):
424-
"""Returns the portable proto representation of the pipeline before
425-
instrumentation."""
367+
"""Returns a snapshot of the pipeline proto before instrumentation."""
426368
return self._original_pipeline_proto
427369

428-
@property
429-
def original_pipeline(self):
430-
"""Returns a snapshot of the pipeline before instrumentation."""
431-
return self._pipeline_snap
432-
433370
@property
434371
def user_pipeline(self):
435372
"""Returns a reference to the pipeline instance defined by the user. If a
@@ -571,29 +508,11 @@ def _process(self, pcoll):
571508
cacheable_key = self._pin._cacheable_key(pcoll)
572509
user_pcoll = self._pin.cacheables[cacheable_key].pcoll
573510
if (cacheable_key in self._pin.cacheables and user_pcoll != pcoll):
574-
if not self._pin._user_pipeline:
575-
# Retrieve a reference to the user defined pipeline instance.
576-
self._pin._user_pipeline = user_pcoll.pipeline
577-
# Retrieve a reference to the cache manager for the user defined
578-
# pipeline instance.
579-
self._pin._cache_manager = ie.current_env().get_cache_manager(
580-
self._pin._user_pipeline, create_if_absent=True)
581-
# Check if the user defined pipeline contains any source to cache.
582-
# If so, during the check, the cache manager is converted into a
583-
# streaming cache manager, thus re-assign the reference.
584-
if background_caching_job.has_source_to_cache(
585-
self._pin._user_pipeline):
586-
self._pin._cache_manager = ie.current_env().get_cache_manager(
587-
self._pin._user_pipeline)
588511
self._pin._runner_pcoll_to_user_pcoll[pcoll] = user_pcoll
589512
self._pin.cacheables[cacheable_key].pcoll = pcoll
590513

591514
v = PreprocessVisitor(self)
592515
self._pipeline.visit(v)
593-
if not self._user_pipeline:
594-
self._user_pipeline = self._pipeline
595-
self._cache_manager = ie.current_env().get_cache_manager(
596-
self._user_pipeline, create_if_absent=True)
597516

598517
def _write_cache(
599518
self,
@@ -679,7 +598,6 @@ def _read_cache(self, pipeline, pcoll, is_unbounded_source_output):
679598
key = self.cache_key(pcoll)
680599
# Can only read from cache when the cache with expected key exists and its
681600
# computation has been completed.
682-
683601
is_cached = self._cache_manager.exists('full', key)
684602
is_computed = (
685603
pcoll in self._runner_pcoll_to_user_pcoll and
@@ -886,7 +804,6 @@ def cacheables(pcolls_to_pcoll_id):
886804
for watching in ie.current_env().watching():
887805
for key, val in watching:
888806
if isinstance(val, beam.pvalue.PCollection):
889-
890807
pcoll_id = pcolls_to_pcoll_id.get(str(val), None)
891808
# It's highly possible that PCollection str is not unique across
892809
# multiple pipelines, further check during instrument is needed.

sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -279,6 +279,7 @@ def test_instrument_example_pipeline_to_read_cache(self):
279279
self._mock_write_cache(p_origin, [b'1', b'4', b'9'], second_pcoll_cache_key)
280280
# Mark the completeness of PCollections from the original(user) pipeline.
281281
ie.current_env().mark_pcollection_computed((init_pcoll, second_pcoll))
282+
ie.current_env().add_derived_pipeline(p_origin, p_copy)
282283
instr.build_pipeline_instrument(p_copy)
283284

284285
cached_init_pcoll = (
@@ -315,6 +316,7 @@ def test_find_out_correct_user_pipeline(self):
315316
user_pipeline.to_runner_api(use_fake_coders=True),
316317
user_pipeline.runner,
317318
options=None)
319+
ie.current_env().add_derived_pipeline(user_pipeline, runner_pipeline)
318320
# This is a totally irrelevant user pipeline in the watched scope.
319321
irrelevant_user_pipeline = beam.Pipeline(
320322
interactive_runner.InteractiveRunner())
@@ -506,6 +508,7 @@ def test_instrument_mixed_streaming_batch(self):
506508
p_original.to_runner_api(),
507509
runner=interactive_runner.InteractiveRunner(),
508510
options=options)
511+
ie.current_env().add_derived_pipeline(p_original, p_copy)
509512
instrumenter = instr.build_pipeline_instrument(p_copy)
510513
actual_pipeline = beam.Pipeline.from_runner_api(
511514
proto=instrumenter.instrumented_pipeline_proto(),
@@ -766,6 +769,7 @@ def test_pipeline_pruned_when_input_pcoll_is_cached(self):
766769
user_pipeline.to_runner_api(use_fake_coders=True),
767770
user_pipeline.runner,
768771
None)
772+
ie.current_env().add_derived_pipeline(user_pipeline, runner_pipeline)
769773

770774
# Mock as if init_pcoll is cached.
771775
init_pcoll_cache_key = self.cache_key_of('init_pcoll', init_pcoll)

sdks/python/container/license_scripts/dep_urls_py.yaml

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,8 @@ pip_dependencies:
4747
license: "https://raw.githubusercontent.com/certifi/python-certifi/master/LICENSE"
4848
cython:
4949
license: "https://raw.githubusercontent.com/cython/cython/master/LICENSE.txt"
50+
dataclasses:
51+
license: "https://raw.githubusercontent.com/ericvsmith/dataclasses/master/LICENSE.txt"
5052
enum34:
5153
# The original repo is down. This license taken from somebody's clone:
5254
# https://github.com/jamespharaoh/python-enum34/blob/master/enum/LICENSE

sdks/python/setup.py

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -131,6 +131,10 @@ def get_version():
131131
# Avro 1.9.2 for python3 was broken. The issue was fixed in version 1.9.2.1
132132
'avro-python3>=1.8.1,!=1.9.2,<1.10.0',
133133
'crcmod>=1.7,<2.0',
134+
# dataclasses backport for python_version<3.7. No version bound because this
135+
# is Python standard since Python 3.7 and each Python version is compatible
136+
# with a specific dataclasses version.
137+
'dataclasses;python_version<"3.7"',
134138
# Dill doesn't have forwards-compatibility guarantees within minor version.
135139
# Pickles created with a new version of dill may not unpickle using older
136140
# version of dill. It is best to use the same version of dill on client and

0 commit comments

Comments
 (0)