[ 
https://issues.apache.org/jira/browse/FLINK-3674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15585563#comment-15585563
 ] 

ASF GitHub Bot commented on FLINK-3674:
---------------------------------------

Github user StefanRRichter commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2570#discussion_r83860732
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/HeapInternalTimerServiceTest.java
 ---
    @@ -0,0 +1,509 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.flink.streaming.api.operators;
    +
    +import org.apache.flink.api.common.typeutils.base.IntSerializer;
    +import org.apache.flink.api.common.typeutils.base.StringSerializer;
    +import org.apache.flink.streaming.api.windowing.triggers.Trigger;
    +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
    +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
    +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
    +import org.junit.Test;
    +import org.mockito.Matchers;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.stubbing.Answer;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.InputStream;
    +
    +import static org.hamcrest.Matchers.containsInAnyOrder;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertThat;
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.contains;
    +import static org.mockito.Matchers.eq;
    +import static org.mockito.Mockito.*;
    +
    +/**
    + * Tests for {@link HeapInternalTimerService}.
    + */
    +public class HeapInternalTimerServiceTest {
    +
    +   private static InternalTimer<Integer, String> anyInternalTimer() {
    +           return any();
    +   }
    +
    +   /**
    +    * Verify that we only ever have one processing-time task registered at 
the
    +    * {@link ProcessingTimeService}.
    +    */
    +   @Test
    +   public void testOnlySetsOnePhysicalProcessingTimeTimer() throws 
Exception {
    +           @SuppressWarnings("unchecked")
    +           Triggerable<Integer, String> mockTriggerable = 
mock(Triggerable.class);
    +
    +           TestKeyContext keyContext = new TestKeyContext();
    +
    +           TestProcessingTimeService processingTimeService = new 
TestProcessingTimeService();
    +
    +           HeapInternalTimerService<Integer, String> timerService =
    +                           createTimerService(mockTriggerable, keyContext, 
processingTimeService);
    +
    +           keyContext.setCurrentKey(0);
    +
    +           timerService.registerProcessingTimeTimer("ciao", 10);
    +           timerService.registerProcessingTimeTimer("ciao", 20);
    +           timerService.registerProcessingTimeTimer("ciao", 30);
    +           timerService.registerProcessingTimeTimer("hello", 10);
    +           timerService.registerProcessingTimeTimer("hello", 20);
    +
    +           assertEquals(5, timerService.numProcessingTimeTimers());
    +           assertEquals(2, timerService.numProcessingTimeTimers("hello"));
    +           assertEquals(3, timerService.numProcessingTimeTimers("ciao"));
    +
    +           assertEquals(1, processingTimeService.getNumRegisteredTimers());
    +           
assertThat(processingTimeService.getRegisteredTimerTimestamps(), 
containsInAnyOrder(10L));
    +
    +           processingTimeService.setCurrentTime(10);
    +
    +           assertEquals(3, timerService.numProcessingTimeTimers());
    +           assertEquals(1, timerService.numProcessingTimeTimers("hello"));
    +           assertEquals(2, timerService.numProcessingTimeTimers("ciao"));
    +
    +           assertEquals(1, processingTimeService.getNumRegisteredTimers());
    +           
assertThat(processingTimeService.getRegisteredTimerTimestamps(), 
containsInAnyOrder(20L));
    +
    +           processingTimeService.setCurrentTime(20);
    +
    +           assertEquals(1, timerService.numProcessingTimeTimers());
    +           assertEquals(0, timerService.numProcessingTimeTimers("hello"));
    +           assertEquals(1, timerService.numProcessingTimeTimers("ciao"));
    +
    +           assertEquals(1, processingTimeService.getNumRegisteredTimers());
    +           
assertThat(processingTimeService.getRegisteredTimerTimestamps(), 
containsInAnyOrder(30L));
    +
    +           processingTimeService.setCurrentTime(30);
    +
    +           assertEquals(0, timerService.numProcessingTimeTimers());
    +
    +           assertEquals(0, processingTimeService.getNumRegisteredTimers());
    +   }
    --- End diff --
    
    Could have an additional check that it `getNumRegisteredTimers()` goes 
again back up to 1 if we start registering a timer again.


> Add an interface for Time aware User Functions
> ----------------------------------------------
>
>                 Key: FLINK-3674
>                 URL: https://issues.apache.org/jira/browse/FLINK-3674
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming
>    Affects Versions: 1.0.0
>            Reporter: Stephan Ewen
>            Assignee: Aljoscha Krettek
>
> I suggest to add an interface that UDFs can implement, which will let them be 
> notified upon watermark updates.
> Example usage:
> {code}
> public interface EventTimeFunction {
>     void onWatermark(Watermark watermark);
> }
> public class MyMapper implements MapFunction<String, String>, 
> EventTimeFunction {
>     private long currentEventTime = Long.MIN_VALUE;
>     public String map(String value) {
>         return value + " @ " + currentEventTime;
>     }
>     public void onWatermark(Watermark watermark) {
>         currentEventTime = watermark.getTimestamp();
>     }
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to