kl0u commented on a change in pull request #13443:
URL: https://github.com/apache/flink/pull/13443#discussion_r493652985



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
##########
@@ -556,11 +558,13 @@ protected void reportOrForwardLatencyMarker(LatencyMarker 
marker) {
                }
                @SuppressWarnings("unchecked")
                InternalTimeServiceManager<K> keyedTimeServiceHandler = 
(InternalTimeServiceManager<K>) timeServiceManager;
+               KeyedStateBackend<K> keyedStateBackend = getKeyedStateBackend();
+               checkNotNull(keyedStateBackend, "Timers can only be used on 
keyed operators.");

Review comment:
       I think this should be a `checkState()`. An `IllegalStateException` 
sounds more appropriate than a NPE.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceSerializationProxy.java
##########
@@ -38,7 +38,7 @@
        public static final int VERSION = 2;
 
        /** The key-group timer services to write / read. */
-       private final InternalTimeServiceManager<K> timerServicesManager;
+       private final InternalTimeServiceManagerImpl<K> timerServicesManager;

Review comment:
       It would be nice if here we could use the interface rather than the 
concrete implementation. Maybe by adding a new interface that include the 
missing methods and let the implementation implement both the 
`InternalTimeServiceManager` and the "new" interface?
   
   Although I have not tried it and it may look even uglier the solution I 
propose.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.PriorityQueueSetFactory;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An entity keeping all the time-related services available to all operators 
extending the
+ * {@link AbstractStreamOperator}. Right now, this is only a
+ * {@link InternalTimerServiceImpl timer services}.
+ *
+ * <b>NOTE:</b> These services are only available to keyed operators.
+ *
+ * @param <K> The type of keys used for the timers and the registry.
+ */
+@Internal
+public class InternalTimeServiceManagerImpl<K> implements 
InternalTimeServiceManager<K> {
+       protected static final Logger LOG = 
LoggerFactory.getLogger(InternalTimeServiceManagerImpl.class);
+
+       @VisibleForTesting
+       static final String TIMER_STATE_PREFIX = "_timer_state";
+       @VisibleForTesting
+       static final String PROCESSING_TIMER_PREFIX = TIMER_STATE_PREFIX + 
"/processing_";
+       @VisibleForTesting
+       static final String EVENT_TIMER_PREFIX = TIMER_STATE_PREFIX + "/event_";
+
+       private final KeyGroupRange localKeyGroupRange;
+       private final KeyContext keyContext;
+
+       private final PriorityQueueSetFactory priorityQueueSetFactory;
+       private final ProcessingTimeService processingTimeService;
+
+       private final Map<String, InternalTimerServiceImpl<K, ?>> timerServices;
+
+       private final boolean useLegacySynchronousSnapshots;
+
+       private InternalTimeServiceManagerImpl(
+               KeyGroupRange localKeyGroupRange,
+               KeyContext keyContext,
+               PriorityQueueSetFactory priorityQueueSetFactory,
+               ProcessingTimeService processingTimeService, boolean 
useLegacySynchronousSnapshots) {
+
+               this.localKeyGroupRange = 
Preconditions.checkNotNull(localKeyGroupRange);
+               this.priorityQueueSetFactory = 
Preconditions.checkNotNull(priorityQueueSetFactory);
+               this.keyContext = Preconditions.checkNotNull(keyContext);
+               this.processingTimeService = 
Preconditions.checkNotNull(processingTimeService);
+               this.useLegacySynchronousSnapshots = 
useLegacySynchronousSnapshots;
+
+               this.timerServices = new HashMap<>();
+       }
+
+       /**
+        * A factory method for creating the {@link 
InternalTimeServiceManagerImpl}.
+        *
+        * <p><b>IMPORTANT:</b> Keep in sync with {@link 
InternalTimeServiceManager.Provider}.
+        */
+       public static <K> InternalTimeServiceManagerImpl<K> create(
+                       CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+                       ClassLoader userClassloader,
+                       KeyContext keyContext, //the operator
+                       ProcessingTimeService processingTimeService,
+                       Iterable<KeyGroupStatePartitionStreamProvider> 
rawKeyedStates) throws Exception {
+
+               if (keyedStatedBackend == null) {
+                       return null;
+               }
+
+               final KeyGroupRange keyGroupRange = 
keyedStatedBackend.getKeyGroupRange();
+               final boolean requiresSnapshotLegacyTimers = keyedStatedBackend 
instanceof AbstractKeyedStateBackend &&
+                       ((AbstractKeyedStateBackend<K>) 
keyedStatedBackend).requiresLegacySynchronousTimerSnapshots();
+
+               final InternalTimeServiceManagerImpl<K> timeServiceManager = 
new InternalTimeServiceManagerImpl<>(
+                       keyGroupRange,
+                       keyContext,
+                       keyedStatedBackend,
+                       processingTimeService,
+                       requiresSnapshotLegacyTimers);
+
+               // and then initialize the timer services
+               for (KeyGroupStatePartitionStreamProvider streamProvider : 
rawKeyedStates) {
+                       int keyGroupIdx = streamProvider.getKeyGroupId();
+
+                       
Preconditions.checkArgument(keyGroupRange.contains(keyGroupIdx),
+                               "Key Group " + keyGroupIdx + " does not belong 
to the local range.");
+
+                       timeServiceManager.restoreStateForKeyGroup(
+                               streamProvider.getStream(),
+                               keyGroupIdx,
+                               userClassloader);
+               }
+
+               return timeServiceManager;
+       }
+
+       @Override
+       public <N> InternalTimerService<N> getInternalTimerService(
+                       String name,
+                       TypeSerializer<K> keySerializer,
+                       TypeSerializer<N> namespaceSerializer,
+                       Triggerable<K, N> triggerable) {
+               checkNotNull(keySerializer, "Timers can only be used on keyed 
operators.");
+
+               // the following casting is to overcome type restrictions.
+               TimerSerializer<K, N> timerSerializer = new 
TimerSerializer<>(keySerializer, namespaceSerializer);
+
+               InternalTimerServiceImpl<K, N> timerService = 
registerOrGetTimerService(name, timerSerializer);
+
+               timerService.startTimerService(
+                       timerSerializer.getKeySerializer(),
+                       timerSerializer.getNamespaceSerializer(),
+                       triggerable);
+
+               return timerService;
+       }
+
+       @SuppressWarnings("unchecked")
+       <N> InternalTimerServiceImpl<K, N> registerOrGetTimerService(String 
name, TimerSerializer<K, N> timerSerializer) {
+               InternalTimerServiceImpl<K, N> timerService = 
(InternalTimerServiceImpl<K, N>) timerServices.get(name);
+               if (timerService == null) {
+
+                       timerService = new InternalTimerServiceImpl<>(
+                               localKeyGroupRange,
+                               keyContext,
+                               processingTimeService,
+                               
createTimerPriorityQueue(PROCESSING_TIMER_PREFIX + name, timerSerializer),
+                               createTimerPriorityQueue(EVENT_TIMER_PREFIX + 
name, timerSerializer));
+
+                       timerServices.put(name, timerService);
+               }
+               return timerService;
+       }
+
+       Map<String, InternalTimerServiceImpl<K, ?>> 
getRegisteredTimerServices() {
+               return Collections.unmodifiableMap(timerServices);
+       }
+
+       private <N> KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, 
N>> createTimerPriorityQueue(
+               String name,
+               TimerSerializer<K, N> timerSerializer) {
+               return priorityQueueSetFactory.create(
+                       name,

Review comment:
       Same here.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java
##########
@@ -455,12 +457,16 @@ protected void reportOrForwardLatencyMarker(LatencyMarker 
marker) {
                if (timeServiceManager == null) {
                        throw new RuntimeException("The timer service has not 
been initialized.");
                }
+
+               @SuppressWarnings("unchecked")
                InternalTimeServiceManager<K> keyedTimeServiceHandler = 
(InternalTimeServiceManager<K>) timeServiceManager;
+               KeyedStateBackend<K> keyedStateBackend = getKeyedStateBackend();
+               checkNotNull(keyedStateBackend, "Timers can only be used on 
keyed operators.");

Review comment:
       Same as in previous comment.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java
##########
@@ -19,214 +19,62 @@
 package org.apache.flink.streaming.api.operators;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
-import org.apache.flink.runtime.state.KeyGroupsList;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
-import org.apache.flink.runtime.state.PriorityQueueSetFactory;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
 import org.apache.flink.runtime.state.StateSnapshotContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * An entity keeping all the time-related services available to all operators 
extending the
- * {@link AbstractStreamOperator}. Right now, this is only a
- * {@link InternalTimerServiceImpl timer services}.
+ * {@link AbstractStreamOperator} or {@link AbstractStreamOperatorV2}.
  *
  * <b>NOTE:</b> These services are only available to keyed operators.
  *
  * @param <K> The type of keys used for the timers and the registry.
  */
 @Internal
-public class InternalTimeServiceManager<K> {
-       protected static final Logger LOG = 
LoggerFactory.getLogger(InternalTimeServiceManager.class);
-
-       @VisibleForTesting
-       static final String TIMER_STATE_PREFIX = "_timer_state";
-       @VisibleForTesting
-       static final String PROCESSING_TIMER_PREFIX = TIMER_STATE_PREFIX + 
"/processing_";
-       @VisibleForTesting
-       static final String EVENT_TIMER_PREFIX = TIMER_STATE_PREFIX + "/event_";
-
-       private final KeyGroupRange localKeyGroupRange;
-       private final KeyContext keyContext;
-
-       private final PriorityQueueSetFactory priorityQueueSetFactory;
-       private final ProcessingTimeService processingTimeService;
-
-       private final Map<String, InternalTimerServiceImpl<K, ?>> timerServices;
-
-       private final boolean useLegacySynchronousSnapshots;
-
-       InternalTimeServiceManager(
-                       KeyGroupRange localKeyGroupRange,
-                       KeyContext keyContext,
-                       PriorityQueueSetFactory priorityQueueSetFactory,
-                       ProcessingTimeService processingTimeService,
-                       boolean useLegacySynchronousSnapshots) {
-
-               this.localKeyGroupRange = 
Preconditions.checkNotNull(localKeyGroupRange);
-               this.priorityQueueSetFactory = 
Preconditions.checkNotNull(priorityQueueSetFactory);
-               this.keyContext = Preconditions.checkNotNull(keyContext);
-               this.processingTimeService = 
Preconditions.checkNotNull(processingTimeService);
-               this.useLegacySynchronousSnapshots = 
useLegacySynchronousSnapshots;
-
-               this.timerServices = new HashMap<>();
-       }
-
-       public <N> InternalTimerService<N> getInternalTimerService(
-                       String name,
-                       TypeSerializer<N> namespaceSerializer,
-                       Triggerable<K, N> triggerable,
-                       KeyedStateBackend<K> keyedStateBackend) {
-               checkNotNull(keyedStateBackend, "Timers can only be used on 
keyed operators.");
-
-               TypeSerializer<K> keySerializer = 
keyedStateBackend.getKeySerializer();
-               // the following casting is to overcome type restrictions.
-               TimerSerializer<K, N> timerSerializer = new 
TimerSerializer<>(keySerializer, namespaceSerializer);
-               return getInternalTimerService(name, timerSerializer, 
triggerable);
-       }
-
-       public <N> InternalTimerService<N> getInternalTimerService(
+public interface InternalTimeServiceManager<K> {
+       /**
+        * Creates an {@link InternalTimerService} for handling a group of 
timers identified by
+        * the given {@code name}. The timers are scoped to a key and namespace.
+        *
+        * <p>When the timer is due it will call the given {@link Triggerable}.
+        */
+       <N> InternalTimerService<N> getInternalTimerService(
                String name,
-               TimerSerializer<K, N> timerSerializer,
-               Triggerable<K, N> triggerable) {
-
-               InternalTimerServiceImpl<K, N> timerService = 
registerOrGetTimerService(name, timerSerializer);
-
-               timerService.startTimerService(
-                       timerSerializer.getKeySerializer(),
-                       timerSerializer.getNamespaceSerializer(),
-                       triggerable);
-
-               return timerService;
-       }
-
-       @SuppressWarnings("unchecked")
-       <N> InternalTimerServiceImpl<K, N> registerOrGetTimerService(String 
name, TimerSerializer<K, N> timerSerializer) {
-               InternalTimerServiceImpl<K, N> timerService = 
(InternalTimerServiceImpl<K, N>) timerServices.get(name);
-               if (timerService == null) {
-
-                       timerService = new InternalTimerServiceImpl<>(
-                               localKeyGroupRange,
-                               keyContext,
-                               processingTimeService,
-                               
createTimerPriorityQueue(PROCESSING_TIMER_PREFIX + name, timerSerializer),
-                               createTimerPriorityQueue(EVENT_TIMER_PREFIX + 
name, timerSerializer));
-
-                       timerServices.put(name, timerService);
-               }
-               return timerService;
-       }
-
-       Map<String, InternalTimerServiceImpl<K, ?>> 
getRegisteredTimerServices() {
-               return Collections.unmodifiableMap(timerServices);
-       }
-
-       private <N> KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, 
N>> createTimerPriorityQueue(
-               String name,
-               TimerSerializer<K, N> timerSerializer) {
-               return priorityQueueSetFactory.create(
-                       name,
-                       timerSerializer);
-       }
-
-       public void advanceWatermark(Watermark watermark) throws Exception {
-               for (InternalTimerServiceImpl<?, ?> service : 
timerServices.values()) {
-                       service.advanceWatermark(watermark.getTimestamp());
-               }
-       }
-
-       //////////////////                              Fault Tolerance Methods 
                        ///////////////////
-
-       public void snapshotState(StateSnapshotContext context, String 
operatorName) throws Exception {
-               //TODO all of this can be removed once heap-based timers are 
integrated with RocksDB incremental snapshots
-               if (useLegacySynchronousSnapshots) {
-                       KeyedStateCheckpointOutputStream out;
-                       try {
-                               out = context.getRawKeyedOperatorStateOutput();
-                       } catch (Exception exception) {
-                               throw new Exception("Could not open raw keyed 
operator state stream for " +
-                                       operatorName + '.', exception);
-                       }
-
-                       try {
-                               KeyGroupsList allKeyGroups = 
out.getKeyGroupList();
-                               for (int keyGroupIdx : allKeyGroups) {
-                                       out.startNewKeyGroup(keyGroupIdx);
-
-                                       snapshotStateForKeyGroup(
-                                               new 
DataOutputViewStreamWrapper(out), keyGroupIdx);
-                               }
-                       } catch (Exception exception) {
-                               throw new Exception("Could not write timer 
service of " + operatorName +
-                                       " to checkpoint state stream.", 
exception);
-                       } finally {
-                               try {
-                                       out.close();
-                               } catch (Exception closeException) {
-                                       LOG.warn("Could not close raw keyed 
operator state stream for {}. This " +
-                                               "might have prevented deleting 
some state data.", operatorName, closeException);
-                               }
-                       }
-               }
-       }
-
-       private void snapshotStateForKeyGroup(DataOutputView stream, int 
keyGroupIdx) throws IOException {
-               InternalTimerServiceSerializationProxy<K> serializationProxy =
-                       new InternalTimerServiceSerializationProxy<>(this, 
keyGroupIdx);
-
-               serializationProxy.write(stream);
-       }
-
-       public void restoreStateForKeyGroup(
-                       InputStream stream,
-                       int keyGroupIdx,
-                       ClassLoader userCodeClassLoader) throws IOException {
-
-               InternalTimerServiceSerializationProxy<K> serializationProxy =
-                       new InternalTimerServiceSerializationProxy<>(
-                               this,
-                               userCodeClassLoader,
-                               keyGroupIdx);
-
-               serializationProxy.read(stream);
-       }
-
-       ////////////////////                    Methods used ONLY IN TESTS      
                        ////////////////////
-
-       @VisibleForTesting
-       public int numProcessingTimeTimers() {
-               int count = 0;
-               for (InternalTimerServiceImpl<?, ?> timerService : 
timerServices.values()) {
-                       count += timerService.numProcessingTimeTimers();
-               }
-               return count;
-       }
-
-       @VisibleForTesting
-       public int numEventTimeTimers() {
-               int count = 0;
-               for (InternalTimerServiceImpl<?, ?> timerService : 
timerServices.values()) {
-                       count += timerService.numEventTimeTimers();
-               }
-               return count;
+               TypeSerializer<K> keySerializer,
+               TypeSerializer<N> namespaceSerializer,
+               Triggerable<K, N> triggerable);
+
+       /**
+        * It advances the Watermark of all managed {@link InternalTimerService 
timer services},
+        * potentially firing the event time timers.
+        */
+       void advanceWatermark(Watermark watermark) throws Exception;
+
+       /**
+        * Snapshots the timers in the keyed state.
+        *
+        * <p><b>TODO:</b> This can be removed once heap-based timers are 
integrated with RocksDB
+        * incremental snapshots.
+        */
+       void snapshotState(
+               StateSnapshotContext context,
+               String operatorName) throws Exception;
+
+       /**
+        * A provider pattern for creating an instance of a {@link 
InternalTimeServiceManager}.
+        * Allows substituting the manager that will be used in the runtime.
+        */
+       @FunctionalInterface
+       interface Provider {
+               <K> InternalTimeServiceManager<K> create(
+                       CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+                       ClassLoader userClassloader,
+                       KeyContext keyContext, //the operator

Review comment:
       Is this comment correct ?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.PriorityQueueSetFactory;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An entity keeping all the time-related services available to all operators 
extending the
+ * {@link AbstractStreamOperator}. Right now, this is only a
+ * {@link InternalTimerServiceImpl timer services}.
+ *
+ * <b>NOTE:</b> These services are only available to keyed operators.
+ *
+ * @param <K> The type of keys used for the timers and the registry.
+ */
+@Internal
+public class InternalTimeServiceManagerImpl<K> implements 
InternalTimeServiceManager<K> {
+       protected static final Logger LOG = 
LoggerFactory.getLogger(InternalTimeServiceManagerImpl.class);
+
+       @VisibleForTesting
+       static final String TIMER_STATE_PREFIX = "_timer_state";
+       @VisibleForTesting
+       static final String PROCESSING_TIMER_PREFIX = TIMER_STATE_PREFIX + 
"/processing_";
+       @VisibleForTesting
+       static final String EVENT_TIMER_PREFIX = TIMER_STATE_PREFIX + "/event_";
+
+       private final KeyGroupRange localKeyGroupRange;
+       private final KeyContext keyContext;
+
+       private final PriorityQueueSetFactory priorityQueueSetFactory;
+       private final ProcessingTimeService processingTimeService;
+
+       private final Map<String, InternalTimerServiceImpl<K, ?>> timerServices;
+
+       private final boolean useLegacySynchronousSnapshots;
+
+       private InternalTimeServiceManagerImpl(
+               KeyGroupRange localKeyGroupRange,
+               KeyContext keyContext,
+               PriorityQueueSetFactory priorityQueueSetFactory,
+               ProcessingTimeService processingTimeService, boolean 
useLegacySynchronousSnapshots) {
+
+               this.localKeyGroupRange = 
Preconditions.checkNotNull(localKeyGroupRange);
+               this.priorityQueueSetFactory = 
Preconditions.checkNotNull(priorityQueueSetFactory);
+               this.keyContext = Preconditions.checkNotNull(keyContext);
+               this.processingTimeService = 
Preconditions.checkNotNull(processingTimeService);
+               this.useLegacySynchronousSnapshots = 
useLegacySynchronousSnapshots;
+
+               this.timerServices = new HashMap<>();
+       }
+
+       /**
+        * A factory method for creating the {@link 
InternalTimeServiceManagerImpl}.
+        *
+        * <p><b>IMPORTANT:</b> Keep in sync with {@link 
InternalTimeServiceManager.Provider}.
+        */
+       public static <K> InternalTimeServiceManagerImpl<K> create(
+                       CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+                       ClassLoader userClassloader,
+                       KeyContext keyContext, //the operator
+                       ProcessingTimeService processingTimeService,
+                       Iterable<KeyGroupStatePartitionStreamProvider> 
rawKeyedStates) throws Exception {
+
+               if (keyedStatedBackend == null) {
+                       return null;
+               }
+
+               final KeyGroupRange keyGroupRange = 
keyedStatedBackend.getKeyGroupRange();
+               final boolean requiresSnapshotLegacyTimers = keyedStatedBackend 
instanceof AbstractKeyedStateBackend &&
+                       ((AbstractKeyedStateBackend<K>) 
keyedStatedBackend).requiresLegacySynchronousTimerSnapshots();
+
+               final InternalTimeServiceManagerImpl<K> timeServiceManager = 
new InternalTimeServiceManagerImpl<>(
+                       keyGroupRange,
+                       keyContext,
+                       keyedStatedBackend,
+                       processingTimeService,
+                       requiresSnapshotLegacyTimers);
+
+               // and then initialize the timer services
+               for (KeyGroupStatePartitionStreamProvider streamProvider : 
rawKeyedStates) {
+                       int keyGroupIdx = streamProvider.getKeyGroupId();
+
+                       
Preconditions.checkArgument(keyGroupRange.contains(keyGroupIdx),
+                               "Key Group " + keyGroupIdx + " does not belong 
to the local range.");
+
+                       timeServiceManager.restoreStateForKeyGroup(
+                               streamProvider.getStream(),
+                               keyGroupIdx,
+                               userClassloader);
+               }
+
+               return timeServiceManager;
+       }
+
+       @Override
+       public <N> InternalTimerService<N> getInternalTimerService(
+                       String name,
+                       TypeSerializer<K> keySerializer,
+                       TypeSerializer<N> namespaceSerializer,
+                       Triggerable<K, N> triggerable) {
+               checkNotNull(keySerializer, "Timers can only be used on keyed 
operators.");
+
+               // the following casting is to overcome type restrictions.
+               TimerSerializer<K, N> timerSerializer = new 
TimerSerializer<>(keySerializer, namespaceSerializer);
+
+               InternalTimerServiceImpl<K, N> timerService = 
registerOrGetTimerService(name, timerSerializer);
+
+               timerService.startTimerService(
+                       timerSerializer.getKeySerializer(),
+                       timerSerializer.getNamespaceSerializer(),
+                       triggerable);
+
+               return timerService;
+       }
+
+       @SuppressWarnings("unchecked")
+       <N> InternalTimerServiceImpl<K, N> registerOrGetTimerService(String 
name, TimerSerializer<K, N> timerSerializer) {
+               InternalTimerServiceImpl<K, N> timerService = 
(InternalTimerServiceImpl<K, N>) timerServices.get(name);
+               if (timerService == null) {
+
+                       timerService = new InternalTimerServiceImpl<>(
+                               localKeyGroupRange,
+                               keyContext,
+                               processingTimeService,
+                               
createTimerPriorityQueue(PROCESSING_TIMER_PREFIX + name, timerSerializer),
+                               createTimerPriorityQueue(EVENT_TIMER_PREFIX + 
name, timerSerializer));
+
+                       timerServices.put(name, timerService);
+               }
+               return timerService;
+       }
+
+       Map<String, InternalTimerServiceImpl<K, ?>> 
getRegisteredTimerServices() {
+               return Collections.unmodifiableMap(timerServices);
+       }
+
+       private <N> KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, 
N>> createTimerPriorityQueue(
+               String name,

Review comment:
       All the args can fit in the same line here.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.PriorityQueueSetFactory;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An entity keeping all the time-related services available to all operators 
extending the
+ * {@link AbstractStreamOperator}. Right now, this is only a
+ * {@link InternalTimerServiceImpl timer services}.
+ *
+ * <b>NOTE:</b> These services are only available to keyed operators.
+ *
+ * @param <K> The type of keys used for the timers and the registry.
+ */
+@Internal
+public class InternalTimeServiceManagerImpl<K> implements 
InternalTimeServiceManager<K> {
+       protected static final Logger LOG = 
LoggerFactory.getLogger(InternalTimeServiceManagerImpl.class);
+
+       @VisibleForTesting
+       static final String TIMER_STATE_PREFIX = "_timer_state";
+       @VisibleForTesting
+       static final String PROCESSING_TIMER_PREFIX = TIMER_STATE_PREFIX + 
"/processing_";
+       @VisibleForTesting
+       static final String EVENT_TIMER_PREFIX = TIMER_STATE_PREFIX + "/event_";
+
+       private final KeyGroupRange localKeyGroupRange;
+       private final KeyContext keyContext;
+
+       private final PriorityQueueSetFactory priorityQueueSetFactory;
+       private final ProcessingTimeService processingTimeService;
+
+       private final Map<String, InternalTimerServiceImpl<K, ?>> timerServices;
+
+       private final boolean useLegacySynchronousSnapshots;
+
+       private InternalTimeServiceManagerImpl(
+               KeyGroupRange localKeyGroupRange,
+               KeyContext keyContext,
+               PriorityQueueSetFactory priorityQueueSetFactory,
+               ProcessingTimeService processingTimeService, boolean 
useLegacySynchronousSnapshots) {
+
+               this.localKeyGroupRange = 
Preconditions.checkNotNull(localKeyGroupRange);
+               this.priorityQueueSetFactory = 
Preconditions.checkNotNull(priorityQueueSetFactory);
+               this.keyContext = Preconditions.checkNotNull(keyContext);
+               this.processingTimeService = 
Preconditions.checkNotNull(processingTimeService);
+               this.useLegacySynchronousSnapshots = 
useLegacySynchronousSnapshots;
+
+               this.timerServices = new HashMap<>();
+       }
+
+       /**
+        * A factory method for creating the {@link 
InternalTimeServiceManagerImpl}.
+        *
+        * <p><b>IMPORTANT:</b> Keep in sync with {@link 
InternalTimeServiceManager.Provider}.
+        */
+       public static <K> InternalTimeServiceManagerImpl<K> create(
+                       CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+                       ClassLoader userClassloader,
+                       KeyContext keyContext, //the operator

Review comment:
       Is this comment correct?




----------------------------------------------------------------
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]


Reply via email to