[
https://issues.apache.org/jira/browse/BEAM-7035?focusedWorklogId=227366&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-227366
]
ASF GitHub Bot logged work on BEAM-7035:
----------------------------------------
Author: ASF GitHub Bot
Created on: 14/Apr/19 17:29
Start Date: 14/Apr/19 17:29
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #8300: [BEAM-7035]
Compatible wire representation for timers in Python SDK
URL: https://github.com/apache/beam/pull/8300#discussion_r275166004
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -1071,7 +1071,11 @@ public void deleteTimer(StateNamespace namespace,
String timerId) {
@Override
public void deleteTimer(StateNamespace namespace, String timerId,
TimeDomain timeDomain) {
- throw new UnsupportedOperationException("Canceling of a timer by ID is
not yet supported.");
+ try {
+ cancelPendingTimerById(getContextTimerId(timerId, namespace));
Review comment:
This is not enough to clean the timer. The method call will only evict the
timer meta information used to delete timers by id (Flink does not allow to
delete timers by id, you always need to supply the original timestamp of the
the timer). See the deleteTimer method below for the additional delete step on
Flink's timer service.
----------------------------------------------------------------
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: 227366)
Time Spent: 1h 10m (was: 1h)
> Clear() method of OutputTimer is inconsistent
> ---------------------------------------------
>
> Key: BEAM-7035
> URL: https://issues.apache.org/jira/browse/BEAM-7035
> Project: Beam
> Issue Type: Bug
> Components: beam-model
> Reporter: Rakesh Kumar
> Assignee: Thomas Weise
> Priority: Major
> Time Spent: 1h 10m
> Remaining Estimate: 0h
>
> [Clear()|https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/worker/bundle_processor.py#L378]
> method of OutputTimer is not consistent ()
> The timestamp parameter is passed here but never used. Also in the [test
> cases
> |[https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/userstate_test.py#L501]]
> and direct runner timer doesn't pass any parameter in the clear method
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)