wanglijie95 commented on code in PR #22413: URL: https://github.com/apache/flink/pull/22413#discussion_r1201634745
########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/HandleGlobalFailureAndRestartAllTasksBenchmark.java: ########## @@ -0,0 +1,119 @@ +/* + * 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.scheduler.benchmark.e2e; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.executiongraph.failover.flip1.FixedDelayRestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; +import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutor; + +import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.StreamSupport; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.completeCancellingForAllVertices; + +/** + * The benchmark of handling global failure and restarting tasks in a STREAMING/BATCH job. The + * related method is {@link DefaultScheduler#handleGlobalFailure}. + */ +public class HandleGlobalFailureAndRestartAllTasksBenchmark extends SchedulerEndToEndBenchmarkBase { + private static final int SLOTS_PER_TASK_EXECUTOR = 4; + private DefaultScheduler scheduler; + private ManuallyTriggeredScheduledExecutor taskRestartExecutor; + + public void setup(JobConfiguration jobConfiguration) throws Exception { + taskRestartExecutor = new ManuallyTriggeredScheduledExecutor(); + // Use DirectScheduledExecutorService to ensure that we can run + // DefaultScheduler#restartTasks in the current thread synchronously when tasks restart is + // triggered. + scheduledExecutorService = new DirectScheduledExecutorService(); + + super.setup(jobConfiguration); + + scheduler = + createScheduler( + jobGraph, + physicalSlotProvider, + mainThreadExecutor, + scheduledExecutorService, + taskRestartExecutor, + new FixedDelayRestartBackoffTimeStrategy + .FixedDelayRestartBackoffTimeStrategyFactory(1, 1) + .create()); + + scheduler.startScheduling(); + offerSlots(); + } + + public void startSchedulingAndFailover() throws Exception { + // trigger failover, force reset state to canceled. + scheduler.handleGlobalFailure(new RuntimeException("For test.")); + completeCancellingForAllVertices(scheduler.getExecutionGraph()); + + taskRestartExecutor.triggerScheduledTasks(); + } + + private static DefaultScheduler createScheduler( + JobGraph jobGraph, + PhysicalSlotProvider physicalSlotProvider, + ComponentMainThreadExecutor mainThreadExecutor, + ScheduledExecutorService executorService, + ScheduledExecutor taskRestartExecutor, + RestartBackoffTimeStrategy restartBackoffTimeStrategy) + throws Exception { + return new DefaultSchedulerBuilder( + jobGraph, + mainThreadExecutor, + executorService, + executorService, + taskRestartExecutor) + .setExecutionSlotAllocatorFactory( + SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( + physicalSlotProvider)) + .setRestartBackoffTimeStrategy(restartBackoffTimeStrategy) + .build(); + } + + private void offerSlots() { + final int numberSlots = + StreamSupport.stream(jobGraph.getVertices().spliterator(), false) + .mapToInt(JobVertex::getParallelism) + .sum(); + + for (int i = 0; i < Math.ceil((double) numberSlots / SLOTS_PER_TASK_EXECUTOR); i++) { + SlotPoolUtils.tryOfferSlots( + slotPool, Review Comment: We can directly use the `mainThreadExecutor` here ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkBase.java: ########## @@ -28,9 +28,11 @@ public class SchedulerBenchmarkBase { public ScheduledExecutorService scheduledExecutorService; public void setup() { - scheduledExecutorService = - Executors.newSingleThreadScheduledExecutor( - new ExecutorThreadFactory("flink-benchmarks")); + if (scheduledExecutorService == null) { Review Comment: Add document to tell the `scheduledExecutorService` may have been set in subclass. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/HandleGlobalFailureAndRestartAllTasksBenchmark.java: ########## @@ -0,0 +1,119 @@ +/* + * 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.scheduler.benchmark.e2e; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.executiongraph.failover.flip1.FixedDelayRestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; +import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutor; + +import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.StreamSupport; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.completeCancellingForAllVertices; + +/** + * The benchmark of handling global failure and restarting tasks in a STREAMING/BATCH job. The + * related method is {@link DefaultScheduler#handleGlobalFailure}. + */ +public class HandleGlobalFailureAndRestartAllTasksBenchmark extends SchedulerEndToEndBenchmarkBase { + private static final int SLOTS_PER_TASK_EXECUTOR = 4; + private DefaultScheduler scheduler; + private ManuallyTriggeredScheduledExecutor taskRestartExecutor; + + public void setup(JobConfiguration jobConfiguration) throws Exception { + taskRestartExecutor = new ManuallyTriggeredScheduledExecutor(); + // Use DirectScheduledExecutorService to ensure that we can run + // DefaultScheduler#restartTasks in the current thread synchronously when tasks restart is + // triggered. + scheduledExecutorService = new DirectScheduledExecutorService(); + + super.setup(jobConfiguration); + + scheduler = + createScheduler( + jobGraph, + physicalSlotProvider, + mainThreadExecutor, + scheduledExecutorService, + taskRestartExecutor, + new FixedDelayRestartBackoffTimeStrategy + .FixedDelayRestartBackoffTimeStrategyFactory(1, 1) + .create()); + + scheduler.startScheduling(); + offerSlots(); + } + + public void startSchedulingAndFailover() throws Exception { Review Comment: `startSchedulingAndFailover` -> `handleGlobalFailureAndRestartAllTasks` -- 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]
