[
https://issues.apache.org/jira/browse/FLINK-3674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15585562#comment-15585562
]
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_r83858860
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerService.java
---
@@ -0,0 +1,60 @@
+/*
+ * 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.annotation.Internal;
+
+/**
+ * Interface for working with time and timers.
+ *
+ * <p>This is the internal version of {@link
org.apache.flink.streaming.api.TimerService}
+ * that allows to specify a key and a namespace to which timers should be
scoped.
+ *
+ * @param <N> Type of the namespace to which timers are scoped.
+ */
+@Internal
+public interface InternalTimerService<N> {
+
+ /** Returns the current processing time. */
+ long currentProcessingTime();
+
+ /** Returns the current event time. */
+ long currentWatermark();
--- End diff --
The corresponding method in the public interface is called
`currentEventTime`. Does it makes sense to keep both method names synchronous?
> 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)