tillrohrmann commented on a change in pull request #17073:
URL: https://github.com/apache/flink/pull/17073#discussion_r699973686



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/DefaultTimerService.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.runtime.taskexecutor.slot;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Service to register timeouts for a given key. The timeouts are identified 
by a ticket so that
+ * newly registered timeouts for the same key can be distinguished from older 
timeouts.
+ *
+ * @param <K> Type of the key
+ */
+public class DefaultTimerService<K> implements TimerService<K> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(DefaultTimerService.class);
+
+    /** Executor service for the scheduled timeouts. */
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** Timeout for the shutdown of the service. */
+    private final long shutdownTimeout;
+
+    /** Map of currently active timeouts. */
+    private final Map<K, Timeout<K>> timeouts;
+
+    /** Listener which is notified about occurring timeouts. */
+    private TimeoutListener<K> timeoutListener;
+
+    public DefaultTimerService(
+            final ScheduledExecutorService scheduledExecutorService, final 
long shutdownTimeout) {
+        this.scheduledExecutorService = 
Preconditions.checkNotNull(scheduledExecutorService);
+
+        Preconditions.checkArgument(
+                shutdownTimeout >= 0L,
+                "The shut down timeout must be larger than or equal than 0.");
+        this.shutdownTimeout = shutdownTimeout;
+
+        this.timeouts = new HashMap<>(16);
+        this.timeoutListener = null;
+    }
+
+    @Override
+    public void start(TimeoutListener<K> initialTimeoutListener) {
+        // sanity check; We only allow to assign a timeout listener once
+        Preconditions.checkState(!scheduledExecutorService.isShutdown());
+        Preconditions.checkState(timeoutListener == null);
+
+        this.timeoutListener = 
Preconditions.checkNotNull(initialTimeoutListener);
+    }
+
+    @Override
+    public void stop() {
+        unregisterAllTimeouts();
+
+        timeoutListener = null;
+
+        scheduledExecutorService.shutdown();
+
+        try {
+            if (!scheduledExecutorService.awaitTermination(
+                    shutdownTimeout, TimeUnit.MILLISECONDS)) {
+                LOG.debug(
+                        "The scheduled executor service did not properly 
terminate. Shutting "
+                                + "it down now.");
+                scheduledExecutorService.shutdownNow();
+            }
+        } catch (InterruptedException e) {
+            LOG.debug(
+                    "Could not properly await the termination of the scheduled 
executor service.",
+                    e);
+            scheduledExecutorService.shutdownNow();
+        }

Review comment:
       I'd suggest to use `ExecutorUtils.gracefulShutdown`.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java
##########
@@ -76,6 +76,6 @@ public static ResourceProfile createTotalResourceProfile(int 
numberOfSlots) {
     }
 
     public static TimerService<AllocationID> createDefaultTimerService(long 
shutdownTimeout) {
-        return new TimerService<>(TestingUtils.defaultExecutor(), 
shutdownTimeout);
+        return new DefaultTimerService<>(TestingUtils.defaultExecutor(), 
shutdownTimeout);

Review comment:
       I think the previous commits won't build because of this.




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to