github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/interactive/caching/reify.py (about) 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 transforms that reifies and unreifies PCollection values with 19 window info. 20 21 For internal use only; no backwards-compatibility guarantees. 22 """ 23 24 # pytype: skip-file 25 26 from typing import Optional 27 28 import apache_beam as beam 29 from apache_beam.runners.interactive import cache_manager as cache 30 from apache_beam.testing import test_stream 31 from apache_beam.transforms.window import WindowedValue 32 33 READ_CACHE = 'ReadCache_' 34 WRITE_CACHE = 'WriteCache_' 35 36 37 class Reify(beam.DoFn): 38 """Reifies elements with window info into windowed values. 39 40 Internally used to capture window info with each element into cache for 41 replayability. 42 """ 43 def process( 44 self, 45 e, 46 w=beam.DoFn.WindowParam, 47 p=beam.DoFn.PaneInfoParam, 48 t=beam.DoFn.TimestampParam): 49 yield test_stream.WindowedValueHolder(WindowedValue(e, t, [w], p)) 50 51 52 class Unreify(beam.DoFn): 53 """Unreifies elements from windowed values. 54 55 Cached values are elements with window info. This unpacks the elements. 56 """ 57 def process(self, e): 58 # Row coder was used when encoding windowed values. 59 if isinstance(e, beam.Row) and hasattr(e, 'windowed_value'): 60 yield e.windowed_value 61 62 63 def reify_to_cache( 64 pcoll: beam.pvalue.PCollection, 65 cache_key: str, 66 cache_manager: cache.CacheManager, 67 reify_label: Optional[str] = None, 68 write_cache_label: Optional[str] = None, 69 is_capture: bool = False) -> beam.pvalue.PValue: 70 """Reifies elements into windowed values and write to cache. 71 72 Args: 73 pcoll: The PCollection to be cached. 74 cache_key: The key of the cache. 75 cache_manager: The cache manager to manage the cache. 76 reify_label: (optional) A transform label for the Reify transform. 77 write_cache_label: (optional) A transform label for the cache-writing 78 transform. 79 is_capture: Whether the cache is capturing a record of recordable sources. 80 """ 81 if not reify_label: 82 reify_label = '{}{}{}'.format('ReifyBefore_', WRITE_CACHE, cache_key) 83 if not write_cache_label: 84 write_cache_label = '{}{}'.format(WRITE_CACHE, cache_key) 85 return ( 86 pcoll | reify_label >> beam.ParDo(Reify()) 87 | write_cache_label >> cache.WriteCache( 88 cache_manager, cache_key, is_capture=is_capture)) 89 90 91 def unreify_from_cache( 92 pipeline: beam.Pipeline, 93 cache_key: str, 94 cache_manager: cache.CacheManager, 95 element_type: Optional[type] = None, 96 source_label: Optional[str] = None, 97 unreify_label: Optional[str] = None) -> beam.pvalue.PCollection: 98 """Reads from cache and unreifies elements from windowed values. 99 100 pipeline: The pipeline that's reading from the cache. 101 cache_key: The key of the cache. 102 cache_manager: The cache manager to manage the cache. 103 element_type: (optional) The element type of the PCollection's elements. 104 source_label: (optional) A transform label for the cache-reading transform. 105 unreify_label: (optional) A transform label for the Unreify transform. 106 """ 107 if not source_label: 108 source_label = '{}{}'.format(READ_CACHE, cache_key) 109 if not unreify_label: 110 unreify_label = '{}{}{}'.format('UnreifyAfter_', READ_CACHE, cache_key) 111 read_cache = pipeline | source_label >> cache.ReadCache( 112 cache_manager, cache_key) 113 if element_type: 114 # If the PCollection is schema-aware, explicitly sets the output types. 115 return read_cache | unreify_label >> beam.ParDo( 116 Unreify()).with_output_types(element_type) 117 return read_cache | unreify_label >> beam.ParDo(Unreify())