[
https://issues.apache.org/jira/browse/BEAM-9562?focusedWorklogId=419191&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-419191
]
ASF GitHub Bot logged work on BEAM-9562:
----------------------------------------
Author: ASF GitHub Bot
Created on: 09/Apr/20 07:21
Start Date: 09/Apr/20 07:21
Worklog Time Spent: 10m
Work Description: robertwb commented on pull request #11314: [BEAM-9562]
Send Timers over Data Channel as Elements
URL: https://github.com/apache/beam/pull/11314#discussion_r405989307
##########
File path: sdks/python/apache_beam/runners/worker/bundle_processor.py
##########
@@ -562,45 +562,45 @@ class OutputTimer(object):
def __init__(self,
key,
window, # type: windowed_value.BoundedWindow
- receiver # type: operations.ConsumerSet
+ paneinfo,
+ timer_family_id,
+ timer_coder_impl,
+ output_stream
):
self._key = key
self._window = window
- self._receiver = receiver
+ self._paneinfo = paneinfo
+ self._timer_family_id = timer_family_id
+ self._output_stream = output_stream
+ self._timer_coder_impl = timer_coder_impl
def set(self, ts):
ts = timestamp.Timestamp.of(ts)
- # TODO(BEAM-9562): Plumb through actual timer fields.
- self._receiver.receive(
- windowed_value.WindowedValue((
- self._key,
- userstate.Timer(
- user_key='',
- dynamic_timer_tag='',
- windows=(self._window, ),
- clear_bit=False,
- fire_timestamp=ts,
- hold_timestamp=ts,
- paneinfo=windowed_value.PANE_INFO_UNKNOWN)),
- ts, (self._window, )))
+ timer = userstate.Timer(
+ user_key=self._key,
+ dynamic_timer_tag='',
+ windows=(self._window, ),
+ clear_bit=False,
+ fire_timestamp=ts,
+ hold_timestamp=ts,
+ paneinfo=self._paneinfo)
+ self._timer_coder_impl.encode_to_stream(timer, self._output_stream, True)
+ self._output_stream.maybe_flush()
def clear(self):
# type: () -> None
dummy_millis = int(common_urns.constants.MAX_TIMESTAMP_MILLIS.constant) + 1
clear_ts = timestamp.Timestamp(micros=dummy_millis * 1000)
- # TODO(BEAM-9562): Plumb through actual paneinfo.
- self._receiver.receive(
- windowed_value.WindowedValue((
- self._key,
- userstate.Timer(
- user_key='',
- dynamic_timer_tag='',
- windows=(self._window, ),
- clear_bit=False,
- fire_timestamp=timestamp.Timestamp.of(clear_ts),
- hold_timestamp=timestamp.Timestamp.of(0),
- paneinfo=windowed_value.PANE_INFO_UNKNOWN)),
- 0, (self._window, )))
+ timer = userstate.Timer(
+ user_key=self._key,
+ dynamic_timer_tag='',
+ windows=(self._window, ),
+ clear_bit=False,
+ fire_timestamp=clear_ts,
Review comment:
They're meaningless when we're clearing a timer (e.g. it won't fire, hold
back the watermark, or have a pane info).
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 419191)
Time Spent: 17h (was: 16h 50m)
> Remove timer from PCollection and treat timers as Elements
> -----------------------------------------------------------
>
> Key: BEAM-9562
> URL: https://issues.apache.org/jira/browse/BEAM-9562
> Project: Beam
> Issue Type: New Feature
> Components: sdk-java-harness, sdk-py-harness
> Reporter: Boyuan Zhang
> Assignee: Boyuan Zhang
> Priority: Major
> Fix For: 2.21.0
>
> Time Spent: 17h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)