github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/deduplicate.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 # pytype: skip-file 19 20 """a collection of ptransforms for deduplicating elements.""" 21 22 import typing 23 24 from apache_beam import typehints 25 from apache_beam.coders.coders import BooleanCoder 26 from apache_beam.transforms import core 27 from apache_beam.transforms import ptransform 28 from apache_beam.transforms import userstate 29 from apache_beam.transforms.timeutil import TimeDomain 30 from apache_beam.utils import timestamp 31 32 __all__ = [ 33 'Deduplicate', 34 'DeduplicatePerKey', 35 ] 36 37 K = typing.TypeVar('K') 38 V = typing.TypeVar('V') 39 40 41 @typehints.with_input_types(typing.Tuple[K, V]) 42 @typehints.with_output_types(typing.Tuple[K, V]) 43 class DeduplicatePerKey(ptransform.PTransform): 44 """ A PTransform which deduplicates <key, value> pair over a time domain and 45 threshold. Values in different windows will NOT be considered duplicates of 46 each other. Deduplication is guaranteed with respect of time domain and 47 duration. 48 49 Time durations are required so as to avoid unbounded memory and/or storage 50 requirements within a runner and care might need to be used to ensure that the 51 deduplication time limit is long enough to remove duplicates but short enough 52 to not cause performance problems within a runner. Each runner may provide an 53 optimized implementation of their choice using the deduplication time domain 54 and threshold specified. 55 56 Does not preserve any order the input PCollection might have had. 57 """ 58 def __init__(self, processing_time_duration=None, event_time_duration=None): 59 if processing_time_duration is None and event_time_duration is None: 60 raise ValueError( 61 'DeduplicatePerKey requires at lease provide either' 62 'processing_time_duration or event_time_duration.') 63 self.processing_time_duration = processing_time_duration 64 self.event_time_duration = event_time_duration 65 66 def _create_deduplicate_fn(self): 67 processing_timer_spec = userstate.TimerSpec( 68 'processing_timer', TimeDomain.REAL_TIME) 69 event_timer_spec = userstate.TimerSpec('event_timer', TimeDomain.WATERMARK) 70 state_spec = userstate.BagStateSpec('seen', BooleanCoder()) 71 processing_time_duration = self.processing_time_duration 72 event_time_duration = self.event_time_duration 73 74 class DeduplicationFn(core.DoFn): 75 def process( 76 self, 77 kv, 78 ts=core.DoFn.TimestampParam, 79 seen_state=core.DoFn.StateParam(state_spec), 80 processing_timer=core.DoFn.TimerParam(processing_timer_spec), 81 event_timer=core.DoFn.TimerParam(event_timer_spec)): 82 if True in seen_state.read(): 83 return 84 85 if processing_time_duration is not None: 86 processing_timer.set( 87 timestamp.Timestamp.now() + processing_time_duration) 88 if event_time_duration is not None: 89 event_timer.set(ts + event_time_duration) 90 seen_state.add(True) 91 yield kv 92 93 @userstate.on_timer(processing_timer_spec) 94 def process_processing_timer( 95 self, seen_state=core.DoFn.StateParam(state_spec)): 96 seen_state.clear() 97 98 @userstate.on_timer(event_timer_spec) 99 def process_event_timer( 100 self, seen_state=core.DoFn.StateParam(state_spec)): 101 seen_state.clear() 102 103 return DeduplicationFn() 104 105 def expand(self, pcoll): 106 return ( 107 pcoll 108 | 'DeduplicateFn' >> core.ParDo(self._create_deduplicate_fn())) 109 110 111 class Deduplicate(ptransform.PTransform): 112 """Similar to DeduplicatePerKey, the Deduplicate transform takes any arbitrary 113 value as input and uses value as key to deduplicate among certain amount of 114 time duration. 115 """ 116 def __init__(self, processing_time_duration=None, event_time_duration=None): 117 if processing_time_duration is None and event_time_duration is None: 118 raise ValueError( 119 'Deduplicate requires at least providing either ' 120 'processing_time_duration or event_time_duration.') 121 self.processing_time_duration = processing_time_duration 122 self.event_time_duration = event_time_duration 123 124 def expand(self, pcoll): 125 return ( 126 pcoll 127 | 'Use Value as Key' >> core.Map(lambda x: (x, None)) 128 | 'DeduplicatePerKey' >> DeduplicatePerKey( 129 processing_time_duration=self.processing_time_duration, 130 event_time_duration=self.event_time_duration) 131 | 'Output Value' >> core.Map(lambda kv: kv[0]))