[ 
https://issues.apache.org/jira/browse/BEAM-7035?focusedWorklogId=227367&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-227367
 ]

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_r275165928
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
 ##########
 @@ -378,14 +379,20 @@ public ByteBuffer getCurrentKey() {
 
   private void setTimer(WindowedValue<InputT> timerElement, 
TimerInternals.TimerData timerData) {
     try {
+      LOG.debug("Setting timer: {} {}", timerElement, timerData);
       // KvToByteBufferKeySelector returns the key encoded
       ByteBuffer encodedKey = (ByteBuffer) keySelector.getKey(timerElement);
       // We have to synchronize to ensure the state backend is not 
concurrently accessed by the
       // state requests
       try {
         stateBackendLock.lock();
         getKeyedStateBackend().setCurrentKey(encodedKey);
-        timerInternals.setTimer(timerData);
+        if 
(timerData.getTimestamp().isAfter(GlobalWindow.TIMESTAMP_MAX_VALUE)) {
 
 Review comment:
   Should this be `MAX_TIMESTAMP` (`Long.MAX_VALUE`)? There can be windows 
between `GlobalWindow.MAX_TIMEMSTAMP` and `MAX_TIMESTAMP` because the former is 
a standard day before the latter. Actually, `MAX_TIMESTAMP` would be good as a 
value because triggers using it as a timestamp can never fire because the 
Watermark cannot go past `MAX_TIMESTAMP`.
 
----------------------------------------------------------------
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: 227367)
    Time Spent: 1h 20m  (was: 1h 10m)

> 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 20m
>  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)

Reply via email to