wanglijie95 commented on code in PR #22413: URL: https://github.com/apache/flink/pull/22413#discussion_r1197529186
########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/RestartStreamingJobBenchmark.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.stream.StreamSupport; + +/** + * The benchmark of restarting tasks in a STREAMING/BATCH job. The related method is {@link + * DefaultScheduler#handleGlobalFailure}. + */ +public class RestartStreamingJobBenchmark extends SchedulingAndDeployingBenchmark { + private static final int SLOTS_PER_TASK_EXECUTOR = 4; + + @Override + protected 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, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + Collections.nCopies(SLOTS_PER_TASK_EXECUTOR, ResourceProfile.ANY)); + } + } + + public void startSchedulingAndFailover() throws Exception { + startScheduling(); Review Comment: Our goal is to verify the performance of `restart`, so I think the `startScheduling` should be moved to `setup`. The performance of `scheduling` has already been verfied in `SchedulingAndDeployingBenchmark` ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/JobConfiguration.java: ########## @@ -33,46 +33,85 @@ public enum JobConfiguration { ResultPartitionType.PIPELINED, JobType.STREAMING, ExecutionMode.PIPELINED, - 4000), + 4000, + false), BATCH( DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING, JobType.BATCH, ExecutionMode.BATCH, - 4000), + 4000, + false), STREAMING_TEST( DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED, JobType.STREAMING, ExecutionMode.PIPELINED, - 10), + 10, + false), BATCH_TEST( DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING, JobType.BATCH, ExecutionMode.BATCH, - 10); + 10, + false), + + STREAMING_EVENLY( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.PIPELINED, + JobType.STREAMING, + ExecutionMode.PIPELINED, + 4000, + true), + + BATCH_EVENLY( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.BLOCKING, + JobType.BATCH, + ExecutionMode.BATCH, + 4000, + true), + + STREAMING_EVENLY_TEST( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.PIPELINED, + JobType.STREAMING, + ExecutionMode.PIPELINED, + 10, + true), + + BATCH_EVENLY_TEST( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.BLOCKING, + JobType.BATCH, + ExecutionMode.BATCH, + 10, + true); private final int parallelism; private final DistributionPattern distributionPattern; private final ResultPartitionType resultPartitionType; private final JobType jobType; private final ExecutionMode executionMode; + private final boolean scheduleEvenly; JobConfiguration( DistributionPattern distributionPattern, ResultPartitionType resultPartitionType, JobType jobType, ExecutionMode executionMode, - int parallelism) { + int parallelism, + boolean scheduleEvenly) { Review Comment: I think it would be better to use `evenlySpreadOutSlots` here. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/SchedulerEndToEndBenchmarkBase.java: ########## @@ -81,6 +87,10 @@ static DefaultScheduler createScheduler( .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) + .setRestartBackoffTimeStrategy( Review Comment: I prefer to add a new `createScheduler`, because other tests are not expected any failover. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/RestartStreamingJobBenchmark.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.stream.StreamSupport; + +/** + * The benchmark of restarting tasks in a STREAMING/BATCH job. The related method is {@link + * DefaultScheduler#handleGlobalFailure}. + */ +public class RestartStreamingJobBenchmark extends SchedulingAndDeployingBenchmark { + private static final int SLOTS_PER_TASK_EXECUTOR = 4; + + @Override + protected 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, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + Collections.nCopies(SLOTS_PER_TASK_EXECUTOR, ResourceProfile.ANY)); + } + } + + public void startSchedulingAndFailover() throws Exception { + startScheduling(); + + CompletableFuture.runAsync( + () -> { + // trigger failover and force reset state to cancelled. + scheduler.handleGlobalFailure(new RuntimeException()); + scheduler + .getExecutionGraph() + .getAllExecutionVertices() + .forEach( + executionVertex -> + executionVertex.fail(new Throwable())); Review Comment: How about refering to the `ExecutionGraphTestUtils#completeCancellingForAllVertices` here? Because we want to `cancel` instread of `fail` ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/RestartStreamingJobBenchmarkTest.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; + +import org.junit.jupiter.api.Test; + +/** + * The benchmark of restarting tasks in a STREAMING/BATCH job. The related method is {@link + * DefaultScheduler#handleGlobalFailure}. + */ +class RestartStreamingJobBenchmarkTest { Review Comment: We can use `ParameterizedTestExtension` here. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/RestartStreamingJobBenchmark.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.stream.StreamSupport; + +/** + * The benchmark of restarting tasks in a STREAMING/BATCH job. The related method is {@link + * DefaultScheduler#handleGlobalFailure}. + */ +public class RestartStreamingJobBenchmark extends SchedulingAndDeployingBenchmark { Review Comment: `RestartStreamingJobBenchmark` -> I prefer `HandleGlobalFailureAndRestartAllTasksBenchmark` here, and this is not just for `streaming` jobs. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/RestartStreamingJobBenchmark.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.scheduler.DefaultScheduler; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.stream.StreamSupport; + +/** + * The benchmark of restarting tasks in a STREAMING/BATCH job. The related method is {@link + * DefaultScheduler#handleGlobalFailure}. + */ +public class RestartStreamingJobBenchmark extends SchedulingAndDeployingBenchmark { + private static final int SLOTS_PER_TASK_EXECUTOR = 4; + + @Override + protected 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, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + Collections.nCopies(SLOTS_PER_TASK_EXECUTOR, ResourceProfile.ANY)); + } + } + + public void startSchedulingAndFailover() throws Exception { + startScheduling(); + + CompletableFuture.runAsync( + () -> { + // trigger failover and force reset state to cancelled. + scheduler.handleGlobalFailure(new RuntimeException()); + scheduler + .getExecutionGraph() + .getAllExecutionVertices() + .forEach( + executionVertex -> + executionVertex.fail(new Throwable())); + }, + mainThreadExecutor) + .join(); + + // wait for task restart and state transition to DEPLOYING + while (true) { Review Comment: For more accurate and stable testing result, I don't recommend using a while loop to check whether the test completes. In the failover process, we will use the `mainThreadExecutor` and `delayExecutor`, we can do the following things to make all operations be done in one thread(`mainThreadExecutor`). 1. let `delayExecutor= new ManuallyTriggeredScheduledExecutor()` 2. trigger `delayExecutor.triggerScheduledTasks()` in `mainThreadExecutor` -- 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