KarmaGYZ commented on a change in pull request #18007: URL: https://github.com/apache/flink/pull/18007#discussion_r762739538
########## File path: flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java ########## @@ -40,8 +41,10 @@ public abstract class FencedRpcEndpoint<F extends Serializable> extends RpcEndpoint { private final UnfencedMainThreadExecutor unfencedMainThreadExecutor; + private final String endpointId; Review comment: It seems we do not need it atm. ########## File path: flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/ScheduledMainThreadExecutor.java ########## @@ -0,0 +1,97 @@ +/* + * 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.rpc; + +import org.apache.flink.runtime.concurrent.ScheduledFutureAdapter; +import org.apache.flink.runtime.concurrent.ThrowingScheduledFuture; +import org.apache.flink.util.concurrent.ClosableScheduledExecutor; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * Scheduled main thread executor uses {@link ScheduledExecutorService} to stores periodic tasks and + * send them to main thread executor when they needs to be executed. This scheduled executor will be + * used in JM, TM and RM to manager the periodic tasks such as resource timeout checker, heartbeat Review comment: ```suggestion * used in JM, TM and RM to manage the periodic tasks such as resource timeout checker, heartbeat ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java ########## @@ -312,19 +313,24 @@ public void testScheduleCallableWithDelayInSeconds() throws Exception { private static void testScheduleWithDelay( BiConsumer<RpcEndpoint.MainThreadExecutor, Duration> scheduler) throws Exception { final CompletableFuture<Long> actualDelayMsFuture = new CompletableFuture<>(); + final String foo = "foobar"; Review comment: It would be better to name it `endpointId`. ########## File path: flink-core/src/main/java/org/apache/flink/util/concurrent/ClosableScheduledExecutor.java ########## @@ -0,0 +1,25 @@ +/* + * 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.util.concurrent; + +/** Extension for the {@link ScheduledExecutor} interface which can be closed. */ +public interface ClosableScheduledExecutor extends ScheduledExecutor, AutoCloseable { + /** Close the scheduled executor and remove all the pending periodic tasks. */ + void close(); Review comment: Missing annotation `Override`. ########## File path: flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/ScheduledMainThreadExecutor.java ########## @@ -0,0 +1,97 @@ +/* + * 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.rpc; + +import org.apache.flink.runtime.concurrent.ScheduledFutureAdapter; +import org.apache.flink.runtime.concurrent.ThrowingScheduledFuture; +import org.apache.flink.util.concurrent.ClosableScheduledExecutor; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * Scheduled main thread executor uses {@link ScheduledExecutorService} to stores periodic tasks and + * send them to main thread executor when they needs to be executed. This scheduled executor will be + * used in JM, TM and RM to manager the periodic tasks such as resource timeout checker, heartbeat + * checker and some other periodic tasks. + */ +public class ScheduledMainThreadExecutor implements ClosableScheduledExecutor { + private final ScheduledExecutorService scheduledExecutorService; + private final MainThreadExecutable gateway; + + public ScheduledMainThreadExecutor(String endpointId, MainThreadExecutable gateway) { + this.scheduledExecutorService = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory(endpointId + "-scheduled-main-executor")); + this.gateway = gateway; + } + + @Override + public void execute(@Nonnull Runnable command) { + gateway.runAsync(command); + } Review comment: What if the `ScheduledMainThreadExecutor` has already been closed? -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org