Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions sdks/python/apache_beam/runners/direct/watermark_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,8 @@ def extract_all_timers(self):
for applied_ptransform, tw in self._transform_to_watermarks.items():
fired_timers, had_realtime_timer = tw.extract_transform_timers()
if fired_timers:
# We should sort the timer firings, so they are fired in order.
fired_timers.sort(key=lambda ft: ft.timestamp)
all_timers.append((applied_ptransform, fired_timers))
if (had_realtime_timer
and tw.output_watermark < WatermarkManager.WATERMARK_POS_INF):
Expand Down
7 changes: 6 additions & 1 deletion sdks/python/apache_beam/transforms/userstate.py
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,12 @@ def _inner(method):


def get_dofn_specs(dofn):
"""Gets the state and timer specs for a DoFn, if any."""
"""Gets the state and timer specs for a DoFn, if any.

Args:
dofn (apache_beam.transforms.core.DoFn): The DoFn instance to introspect for
timer and state specs.
"""

# Avoid circular import.
from apache_beam.runners.common import MethodWrapper
Expand Down
43 changes: 43 additions & 0 deletions sdks/python/apache_beam/transforms/userstate_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import apache_beam as beam
from apache_beam.coders import BytesCoder
from apache_beam.coders import IterableCoder
from apache_beam.coders import StrUtf8Coder
from apache_beam.coders import VarIntCoder
from apache_beam.runners.common import DoFnSignature
from apache_beam.testing.test_pipeline import TestPipeline
Expand Down Expand Up @@ -366,6 +367,48 @@ def expiry_callback(self, buffer=DoFn.StateParam(BUFFER_STATE),
[b'A1A2A3', b'A1A2A3A4'],
StatefulDoFnOnDirectRunnerTest.all_records)

def test_clearing_bag_state(self):
class BagStateClearingStatefulDoFn(beam.DoFn):

BAG_STATE = BagStateSpec('bag_state', StrUtf8Coder())
EMIT_TIMER = TimerSpec('emit_timer', TimeDomain.WATERMARK)
CLEAR_TIMER = TimerSpec('clear_timer', TimeDomain.WATERMARK)

def process(self,
element,
bag_state=beam.DoFn.StateParam(BAG_STATE),
emit_timer=beam.DoFn.TimerParam(EMIT_TIMER),
clear_timer=beam.DoFn.TimerParam(CLEAR_TIMER)):
value = element[1]
bag_state.add(value)
clear_timer.set(100)
emit_timer.set(1000)

@on_timer(EMIT_TIMER)
def emit_values(self, bag_state=beam.DoFn.StateParam(BAG_STATE)):
for value in bag_state.read():
yield value
yield 'extra'

@on_timer(CLEAR_TIMER)
def clear_values(self, bag_state=beam.DoFn.StateParam(BAG_STATE)):
bag_state.clear()

with TestPipeline() as p:
test_stream = (TestStream()
.advance_watermark_to(0)
.add_elements([('key', 'value')])
.advance_watermark_to(100))

_ = (p
| test_stream
| beam.ParDo(BagStateClearingStatefulDoFn())
| beam.ParDo(self.record_dofn()))

self.assertEqual(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to check whether emit_values fired or not at all?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added a check for that. Thanks!

['extra'],
StatefulDoFnOnDirectRunnerTest.all_records)

def test_stateful_dofn_nonkeyed_input(self):
p = TestPipeline()
values = p | beam.Create([1, 2, 3])
Expand Down