KarmaGYZ commented on a change in pull request #18303:
URL: https://github.com/apache/flink/pull/18303#discussion_r810820436



##########
File path: 
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/concurrent/ScheduledFutureAdapter.java
##########
@@ -94,17 +108,19 @@ public int compareTo(@Nonnull Delayed o) {
 
     @Override
     public boolean cancel(boolean mayInterruptIfRunning) {
-        return delegate.cancel(mayInterruptIfRunning);
+        return (scheduledFuture.isDone() || 
scheduledFuture.cancel(mayInterruptIfRunning))
+                && delegate.cancel(mayInterruptIfRunning);

Review comment:
       1. The `scheduledFuture` can be completed between 
`scheduledFuture.isDone()` and `scheduledFuture.cancel(mayInterruptIfRunning)`. 
So, we need to revert the order of these two methods.
   2. I think we'd better add a safenet here in case the first condition 
returns false and the `delegate` will not be canceled.

##########
File path: 
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/concurrent/ThrowingScheduledFuture.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.concurrent;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Completed {@link ScheduledFuture} implementation.

Review comment:
       ```suggestion
    * Completed {@link ScheduledFuture} implementation in which the scheduled 
task has been canceled.
   ```

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java
##########
@@ -20,40 +20,48 @@
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.runtime.concurrent.ThrowingScheduledFuture;
+import org.apache.flink.util.TestLoggerExtension;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.time.Duration;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.BiConsumer;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+
+import static 
org.apache.flink.runtime.rpc.EndpointCloseableRegistryTestUtils.validateRegisteredResourceCount;
+import static 
org.apache.flink.runtime.rpc.EndpointCloseableRegistryTestUtils.validateRegistryClosed;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /** Tests for the RpcEndpoint, its self gateways and MainThreadExecutor 
scheduling command. */
-public class RpcEndpointTest extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)

Review comment:
       The JUnit5 related refactor should belong to a separate hotfix commit.

##########
File path: 
flink-rpc/flink-rpc-core/src/test/java/org/apache/flink/runtime/concurrent/ScheduledFutureAdapterTest.java
##########
@@ -18,94 +18,138 @@
 
 package org.apache.flink.runtime.concurrent;
 
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import javax.annotation.Nonnull;
 
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /** Unit tests for {@link ScheduledFutureAdapter}. */
-public class ScheduledFutureAdapterTest extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)

Review comment:
       The JUnit5 related refactor should belong to a separate hotfix commit.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java
##########
@@ -309,22 +354,83 @@ public void testScheduleCallableWithDelayInSeconds() 
throws Exception {
                                 () -> 1, expectedDelay.toMillis() / 1000, 
TimeUnit.SECONDS));
     }
 
+    @Test
+    public void testScheduleCallableAfterClose() throws Exception {
+        testScheduleAfterClose(
+                (mainThreadExecutor, expectedDelay) ->
+                        mainThreadExecutor.schedule(
+                                () -> 1, expectedDelay.toMillis() / 1000, 
TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testCancelScheduledCallable() {
+        testCancelScheduledTask(
+                (mainThreadExecutor, timeDelay) ->
+                        mainThreadExecutor.schedule(
+                                () -> 1, timeDelay.toMillis(), 
TimeUnit.MILLISECONDS));
+    }
+
     private static void testScheduleWithDelay(
             BiConsumer<RpcEndpoint.MainThreadExecutor, Duration> scheduler) 
throws Exception {
-        final CompletableFuture<Long> actualDelayMsFuture = new 
CompletableFuture<>();
+        final CompletableFuture<Void> actualDelayMsFuture = new 
CompletableFuture<>();
+        final String endpointId = "foobar";
 
         final MainThreadExecutable mainThreadExecutable =
-                new TestMainThreadExecutable(
-                        (runnable, delay) -> 
actualDelayMsFuture.complete(delay));
+                new TestMainThreadExecutable((runnable) -> 
actualDelayMsFuture.complete(null));
 
         final RpcEndpoint.MainThreadExecutor mainThreadExecutor =
-                new RpcEndpoint.MainThreadExecutor(mainThreadExecutable, () -> 
{});
+                new RpcEndpoint.MainThreadExecutor(mainThreadExecutable, () -> 
{}, endpointId);
 
         final Duration expectedDelay = Duration.ofSeconds(1);
 
         scheduler.accept(mainThreadExecutor, expectedDelay);
 
-        assertThat(actualDelayMsFuture.get(), is(expectedDelay.toMillis()));
+        actualDelayMsFuture.get();
+        mainThreadExecutor.close();
+    }
+
+    private static void testScheduleAfterClose(
+            BiFunction<RpcEndpoint.MainThreadExecutor, Duration, 
ScheduledFuture<?>> scheduler) {
+        final CompletableFuture<Void> actualDelayMsFuture = new 
CompletableFuture<>();

Review comment:
       Should we rename it to taskCompletedFuture or something else?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java
##########
@@ -309,22 +354,83 @@ public void testScheduleCallableWithDelayInSeconds() 
throws Exception {
                                 () -> 1, expectedDelay.toMillis() / 1000, 
TimeUnit.SECONDS));
     }
 
+    @Test
+    public void testScheduleCallableAfterClose() throws Exception {
+        testScheduleAfterClose(
+                (mainThreadExecutor, expectedDelay) ->
+                        mainThreadExecutor.schedule(
+                                () -> 1, expectedDelay.toMillis() / 1000, 
TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testCancelScheduledCallable() {
+        testCancelScheduledTask(
+                (mainThreadExecutor, timeDelay) ->
+                        mainThreadExecutor.schedule(
+                                () -> 1, timeDelay.toMillis(), 
TimeUnit.MILLISECONDS));
+    }
+
     private static void testScheduleWithDelay(
             BiConsumer<RpcEndpoint.MainThreadExecutor, Duration> scheduler) 
throws Exception {
-        final CompletableFuture<Long> actualDelayMsFuture = new 
CompletableFuture<>();
+        final CompletableFuture<Void> actualDelayMsFuture = new 
CompletableFuture<>();

Review comment:
       Should we rename it to `taskCompletedFuture` or something else?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java
##########
@@ -38,24 +39,28 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static 
org.apache.flink.runtime.rpc.EndpointCloseableRegistryTestUtils.validateRegisteredResourceCount;
+import static 
org.apache.flink.runtime.rpc.EndpointCloseableRegistryTestUtils.validateRegistryClosed;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
-public class FencedRpcEndpointTest extends TestLogger {
+/** Tests for the FencedRpcEndpoint. */
+@ExtendWith(TestLoggerExtension.class)

Review comment:
       The JUnit5 related refactor should belong to a separate hotfix commit.

##########
File path: 
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
##########
@@ -453,5 +556,19 @@ public void execute(@Nonnull Runnable command) {
         public void assertRunningInMainThread() {
             mainThreadCheck.run();
         }
+
+        /** Shutdown the {@link ScheduledThreadPoolExecutor} and remove all 
the pending tasks. */
+        @Override
+        public void close() {
+            if (!mainScheduledExecutor.isShutdown()) {
+                List<Runnable> runnableList = 
mainScheduledExecutor.shutdownNow();
+                runnableList.clear();

Review comment:
       Why do we need to clear it?




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