[
https://issues.apache.org/jira/browse/FLINK-5420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15882677#comment-15882677
]
ASF GitHub Bot commented on FLINK-5420:
---------------------------------------
Github user aljoscha commented on a diff in the pull request:
https://github.com/apache/flink/pull/3412#discussion_r102940901
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyRegistry.java
---
@@ -0,0 +1,257 @@
+/*
+ * 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.TypeSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The key registry allows to register a {@link OnWatermarkCallback} and
multiple keys, for which
+ * the callback will be invoked periodically, upon reception of each
subsequent {@link Watermark},
+ * after the registration of the key.
+ * <p>
+ * <b>NOTE: </b> This service is only available to <b>keyed</b> operators.
+ *
+ * @param <K> The type of key returned by the {@code KeySelector}.
+ * */
+public class KeyRegistry<K> {
--- End diff --
To keep it in line with the other services this could be called
`InternalWatermarkCallbackService`.
> Make CEP operators rescalable
> -----------------------------
>
> Key: FLINK-5420
> URL: https://issues.apache.org/jira/browse/FLINK-5420
> Project: Flink
> Issue Type: Bug
> Components: CEP
> Affects Versions: 1.2.0
> Reporter: Kostas Kloudas
> Assignee: Kostas Kloudas
>
> This issue targets making the operators in the CEP library re-scalable. After
> this is implemented, the user will be able to take a savepoint and restart
> his job with a different parallelism.
> This issue depends on https://issues.apache.org/jira/browse/FLINK-5845.
> The way this is done is that we introduce the {{TimeServiceHandler}} in the
> {{AbstractStreamOperator}}, which keeps the registered
> {{InternalTimerService}} s (before this was in the
> {{AbstractStreamOperator}}) and a new service called {{KeyRegistry}}. The
> {{KeyRegistry}} will be fault tolerant and rescalable and will allow to
> register keys and a callback which will be invoked for each registered key
> upon reception of a watermark. This can be seen as keeping (recurring) timers
> for each of the registered keys that will fire "at the next watermark".
> After introducing this service, upon reception of a watermark, all the
> processing of the NFAs will be delegated to the callback.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)