[
https://issues.apache.org/jira/browse/FLINK-7245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16131401#comment-16131401
]
ASF GitHub Bot commented on FLINK-7245:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/4530#discussion_r133842744
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala
---
@@ -0,0 +1,45 @@
+/*
+ * 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.table.runtime.operators
+
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.api.watermark.Watermark
+
+/**
+ * A {@link
org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator} that
supports
+ * holding back watermarks with a static delays.
+ */
+class KeyedCoProcessOperatorWithWatermarkDelay[KEY, IN1, IN2, OUT](
+ private val flatMapper: CoProcessFunction[IN1, IN2, OUT],
+ private val watermarkDelay1: Long = 0L,
+ // The watermarkDelay2 is useless now
+ private var watermarkDelay2: Long = 0L)
--- End diff --
Since this is an internal class, we don't need to be concerned about
changing the interface later. I'd suggest to remove `watermarkDelay2` and add
it later when we need it.
> Enhance the operators to support holding back watermarks
> --------------------------------------------------------
>
> Key: FLINK-7245
> URL: https://issues.apache.org/jira/browse/FLINK-7245
> Project: Flink
> Issue Type: New Feature
> Components: DataStream API
> Reporter: Xingcan Cui
> Assignee: Xingcan Cui
>
> Currently the watermarks are applied and emitted by the
> {{AbstractStreamOperator}} instantly.
> {code:java}
> public void processWatermark(Watermark mark) throws Exception {
> if (timeServiceManager != null) {
> timeServiceManager.advanceWatermark(mark);
> }
> output.emitWatermark(mark);
> }
> {code}
> Some calculation results (with timestamp fields) triggered by these
> watermarks (e.g., join or aggregate results) may be regarded as delayed by
> the downstream operators since their timestamps must be less than or equal to
> the corresponding triggers.
> This issue aims to add another "working mode", which supports holding back
> watermarks, to current operators. These watermarks should be blocked and
> stored by the operators until all the corresponding new generated results are
> emitted.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)