def testOutputTimerTimestamp(self): class Coder(object): """Dummy coder to capture the timer result befor encoding.""" def encode_to_stream(self, timer, *args, **kwargs): self.timer = timer coder = Coder() ctx = FnApiUserStateContext(None, 'transform_id', None, None) ctx.add_timer_info( 'ts-event-timer', TimerInfo(coder, SizeBasedBufferingClosableOutputStream())) ctx.add_timer_info( 'ts-proc-timer', TimerInfo(coder, SizeBasedBufferingClosableOutputStream())) timer_spec1 = userstate.TimerSpec('event-timer', userstate.TimeDomain.WATERMARK) timer_spec2 = userstate.TimerSpec('proc-timer', userstate.TimeDomain.REAL_TIME) # Set event time timer event_timer = ctx.get_timer(timer_spec1, 'key', GlobalWindow, 23, None) event_timer.set(42) # Output timestamp should be equal to the fire timestamp self.assertEqual(coder.timer.hold_timestamp, 42) # Set processing time timer proc_timer = ctx.get_timer(timer_spec2, 'key', GlobalWindow, 23, None) proc_timer.set(42) # Output timestamp should be equal to the input timestamp self.assertEqual(coder.timer.hold_timestamp, 23)
class GenerateLoad(beam.DoFn): state_spec = userstate.CombiningValueStateSpec( 'bundles_remaining', combine_fn=sum) timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) def __init__(self, num_records_per_key, value_size, bundle_size=1000): self.num_records_per_key = num_records_per_key self.payload = os.urandom(value_size) self.bundle_size = bundle_size self.time_fn = time.time def process( self, _element, records_remaining=beam.DoFn.StateParam(state_spec), timer=beam.DoFn.TimerParam(timer_spec)): records_remaining.add(self.num_records_per_key) timer.set(0) @userstate.on_timer(timer_spec) def process_timer( self, key=beam.DoFn.KeyParam, records_remaining=beam.DoFn.StateParam(state_spec), timer=beam.DoFn.TimerParam(timer_spec)): cur_bundle_size = min(self.bundle_size, records_remaining.read()) for _ in range(cur_bundle_size): records_remaining.add(-1) yield key, self.payload if records_remaining.read() > 0: timer.set(0)
def _run_pardo_state_timers(self, windowed): state_spec = userstate.BagStateSpec('state', beam.coders.StrUtf8Coder()) timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) elements = list('abcdefgh') buffer_size = 3 class BufferDoFn(beam.DoFn): def process(self, kv, ts=beam.DoFn.TimestampParam, timer=beam.DoFn.TimerParam(timer_spec), state=beam.DoFn.StateParam(state_spec)): _, element = kv state.add(element) buffer = state.read() # For real use, we'd keep track of this size separately. if len(list(buffer)) >= 3: state.clear() yield buffer else: timer.set(ts + 1) @userstate.on_timer(timer_spec) def process_timer(self, state=beam.DoFn.StateParam(state_spec)): buffer = state.read() state.clear() yield buffer def is_buffered_correctly(actual): # Pickling self in the closure for asserts gives errors (only on jenkins). self = FnApiRunnerTest('__init__') # Acutal should be a grouping of the inputs into batches of size # at most buffer_size, but the actual batching is nondeterministic # based on ordering and trigger firing timing. self.assertEqual(sorted(sum((list(b) for b in actual), [])), elements) self.assertEqual(max(len(list(buffer)) for buffer in actual), buffer_size) if windowed: # Elements were assigned to windows based on their parity. # Assert that each grouping consists of elements belonging to the # same window to ensure states and timers were properly partitioned. for b in actual: parity = set(ord(e) % 2 for e in b) self.assertEqual(1, len(parity), b) with self.create_pipeline() as p: actual = ( p | beam.Create(elements) # Send even and odd elements to different windows. | beam.Map(lambda e: window.TimestampedValue(e, ord(e) % 2)) | beam.WindowInto(window.FixedWindows(1) if windowed else window.GlobalWindows()) | beam.Map(lambda x: ('key', x)) | beam.ParDo(BufferDoFn())) assert_that(actual, is_buffered_correctly)
def _create_deduplicate_fn(self): processing_timer_spec = userstate.TimerSpec('processing_timer', TimeDomain.REAL_TIME) event_timer_spec = userstate.TimerSpec('event_timer', TimeDomain.WATERMARK) state_spec = userstate.BagStateSpec('seen', BooleanCoder()) processing_time_duration = self.processing_time_duration event_time_duration = self.event_time_duration class DeduplicationFn(core.DoFn): def process( self, kv, ts=core.DoFn.TimestampParam, seen_state=core.DoFn.StateParam(state_spec), processing_timer=core.DoFn.TimerParam(processing_timer_spec), event_timer=core.DoFn.TimerParam(event_timer_spec)): if True in seen_state.read(): return if processing_time_duration is not None: processing_timer.set(timestamp.Timestamp.now() + processing_time_duration) if event_time_duration is not None: event_timer.set(ts + event_time_duration) seen_state.add(True) yield kv @userstate.on_timer(processing_timer_spec) def process_processing_timer( self, seen_state=core.DoFn.StateParam(state_spec)): seen_state.clear() @userstate.on_timer(event_timer_spec) def process_event_timer( self, seen_state=core.DoFn.StateParam(state_spec)): seen_state.clear() return DeduplicationFn()
def test_pardo_timers_clear(self): if type(self).__name__ != 'FlinkRunnerTest': # FnApiRunner fails to wire multiple timer collections # this method can replace test_pardo_timers when the issue is fixed self.skipTest('BEAM-7074: Multiple timer definitions not supported.') timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) clear_timer_spec = userstate.TimerSpec('clear_timer', userstate.TimeDomain.WATERMARK) class TimerDoFn(beam.DoFn): def process(self, element, timer=beam.DoFn.TimerParam(timer_spec), clear_timer=beam.DoFn.TimerParam(clear_timer_spec)): unused_key, ts = element timer.set(ts) timer.set(2 * ts) clear_timer.set(ts) clear_timer.clear() @userstate.on_timer(timer_spec) def process_timer(self): yield 'fired' @userstate.on_timer(clear_timer_spec) def process_clear_timer(self): yield 'should not fire' with self.create_pipeline() as p: actual = ( p | beam.Create([('k1', 10), ('k2', 100)]) | beam.ParDo(TimerDoFn()) | beam.Map(lambda x, ts=beam.DoFn.TimestampParam: (x, ts))) expected = [('fired', ts) for ts in (20, 200)] assert_that(actual, equal_to(expected))
def test_pardo_state_timers(self): state_spec = userstate.BagStateSpec('state', beam.coders.StrUtf8Coder()) timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) elements = list('abcdefgh') buffer_size = 3 class BufferDoFn(beam.DoFn): def process(self, kv, ts=beam.DoFn.TimestampParam, timer=beam.DoFn.TimerParam(timer_spec), state=beam.DoFn.StateParam(state_spec)): _, element = kv state.add(element) buffer = state.read() # For real use, we'd keep track of this size separately. if len(list(buffer)) >= 3: state.clear() yield buffer else: timer.set(ts + 1) @userstate.on_timer(timer_spec) def process_timer(self, state=beam.DoFn.StateParam(state_spec)): buffer = state.read() state.clear() yield buffer def is_buffered_correctly(actual): # Pickling self in the closure for asserts gives errors (only on jenkins). self = FnApiRunnerTest('__init__') # Acutal should be a grouping of the inputs into batches of size # at most buffer_size, but the actual batching is nondeterministic # based on ordering and trigger firing timing. self.assertEqual(sorted(sum((list(b) for b in actual), [])), elements) self.assertEqual(max(len(list(buffer)) for buffer in actual), buffer_size) with self.create_pipeline() as p: actual = (p | beam.Create(elements) | beam.Map(lambda x: ('key', x)) | beam.ParDo(BufferDoFn())) assert_that(actual, is_buffered_correctly)
def test_pardo_timers(self): timer_spec = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) class TimerDoFn(beam.DoFn): def process(self, element, timer=beam.DoFn.TimerParam(timer_spec)): unused_key, ts = element timer.set(ts) timer.set(2 * ts) @userstate.on_timer(timer_spec) def process_timer(self): yield 'fired' with self.create_pipeline() as p: actual = ( p | beam.Create([('k1', 10), ('k2', 100)]) | beam.ParDo(TimerDoFn()) | beam.Map(lambda x, ts=beam.DoFn.TimestampParam: (x, ts))) expected = [('fired', ts) for ts in (20, 200)] assert_that(actual, equal_to(expected))
class JoinFn(beam.DoFn): """ Join auctions and person by person id and emit their product one pair at a time. We know a person may submit any number of auctions. Thus new person event must have the person record stored in persistent state in order to match future auctions by that person. However we know that each auction is associated with at most one person, so only need to store auction records in persistent state until we have seen the corresponding person record. And of course may have already seen that record. """ AUCTIONS = 'auctions_state' PERSON = 'person_state' PERSON_EXPIRING = 'person_state_expiring' auction_spec = userstate.BagStateSpec(AUCTIONS, nexmark_model.Auction.CODER) person_spec = userstate.ReadModifyWriteStateSpec( PERSON, nexmark_model.Person.CODER) person_timer_spec = userstate.TimerSpec( PERSON_EXPIRING, userstate.TimeDomain.WATERMARK) def __init__(self, max_auction_wait_time): self.max_auction_wait_time = max_auction_wait_time def process( self, element, auction_state=beam.DoFn.StateParam(auction_spec), person_state=beam.DoFn.StateParam(person_spec), person_timer=beam.DoFn.TimerParam(person_timer_spec)): # extract group with tags from element tuple _, group = element existing_person = person_state.read() if existing_person: # the person exists in person_state for this person id for auction in group[nexmark_query_util.AUCTION_TAG]: yield auction, existing_person return new_person = None for person in group[nexmark_query_util.PERSON_TAG]: if not new_person: new_person = person else: logging.error( 'two new person wtih same key: %s and %s' % (person, new_person)) continue # read all pending auctions for this person id, output and flush it pending_auctions = auction_state.read() if pending_auctions: for pending_auction in pending_auctions: yield pending_auction, new_person auction_state.clear() # output new auction for this person id for auction in group[nexmark_query_util.AUCTION_TAG]: yield auction, new_person # remember person for max_auction_wait_time seconds for future auctions person_state.write(new_person) person_timer.set(new_person.date_time + self.max_auction_wait_time) # we are done if we have seen a new person if new_person: return # remember auction until we see person for auction in group[nexmark_query_util.AUCTION_TAG]: auction_state.add(auction) @on_timer(person_timer_spec) def expiry(self, person_state=beam.DoFn.StateParam(person_spec)): person_state.clear()