zhuzhurk commented on code in PR #21634: URL: https://github.com/apache/flink/pull/21634#discussion_r1217530388
########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/JobConfiguration.java: ########## @@ -89,27 +98,67 @@ public enum JobConfiguration { ResultPartitionType.BLOCKING, JobType.BATCH, ExecutionMode.BATCH, + SchedulerType.Default, 10, - true); + true), + + ADAPTIVE_BATCH( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.BLOCKING, + JobType.BATCH, + ExecutionMode.BATCH, + SchedulerType.AdaptiveBatch, + 4000, + false), + + ADAPTIVE_BATCH_HYBRID( + DistributionPattern.ALL_TO_ALL, + ResultPartitionType.HYBRID_FULL, Review Comment: Can we also add benchmarks for `HYBRID_SELECTIVE`? ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/scheduling/InitSchedulingStrategyBenchmarkTest.java: ########## @@ -20,28 +20,46 @@ import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; import org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.VertexwiseSchedulingStrategy; import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** - * The benchmark of initializing {@link PipelinedRegionSchedulingStrategy} in a STREAMING/BATCH job. + * The benchmark of initializing {@link PipelinedRegionSchedulingStrategy} or {@link + * VertexwiseSchedulingStrategy} in a STREAMING/BATCH job. */ public class InitSchedulingStrategyBenchmarkTest extends TestLogger { Review Comment: ```suggestion class InitSchedulingStrategyBenchmarkTest { ``` Can be package private. The `TestLogger` is no longer needed with JUnit5. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/failover/RestartTasksInBatchJobBenchmark.java: ########## @@ -0,0 +1,59 @@ +/* + * 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.failover; + +import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; +import org.apache.flink.runtime.topology.Vertex; +import org.apache.flink.util.IterableUtils; + +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.scheduler.benchmark.SchedulerBenchmarkUtils.createSchedulingStrategy; + +/** + * The benchmark of restarting tasks in a BATCH job. The related method is {@link + * SchedulingStrategy#restartTasks}. + */ +public class RestartTasksInBatchJobBenchmark extends FailoverBenchmarkBase { + + private SchedulingStrategy schedulingStrategy; + + private Set<ExecutionVertexID> toRestartExecutionIds; + + @Override + public void setup(JobConfiguration jobConfiguration) throws Exception { + super.setup(jobConfiguration); + schedulingStrategy = createSchedulingStrategy(jobConfiguration, schedulingTopology); + + toRestartExecutionIds = new HashSet<>(); + toRestartExecutionIds.addAll( + IterableUtils.toStream(schedulingTopology.getVertices()) Review Comment: Seems it only restarts sources, what's the purpose here? ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/scheduling/InitSchedulingStrategyBenchmarkTest.java: ########## @@ -20,27 +20,30 @@ import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; import org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy; -import org.apache.flink.util.TestLogger; +import org.apache.flink.runtime.scheduler.strategy.VertexwiseSchedulingStrategy; -import org.junit.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; /** - * The benchmark of initializing {@link PipelinedRegionSchedulingStrategy} in a STREAMING/BATCH job. + * The benchmark of initializing {@link PipelinedRegionSchedulingStrategy} or {@link + * VertexwiseSchedulingStrategy} in a STREAMING/BATCH job. */ -public class InitSchedulingStrategyBenchmarkTest extends TestLogger { +class InitSchedulingStrategyBenchmarkTest { - @Test - public void initSchedulingStrategyBenchmarkInStreamingJob() throws Exception { + @ParameterizedTest + @EnumSource( + value = JobConfiguration.class, + names = { + "STREAMING_TEST", + "BATCH_TEST", + "ADAPTIVE_BATCH_TEST", + "ADAPTIVE_BATCH_HYBRID_TEST" + }) + void initSchedulingStrategyBenchmarkInStreamingJob(JobConfiguration jobConfiguration) Review Comment: initSchedulingStrategyBenchmarkInStreamingJob -> initSchedulingStrategy ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java: ########## @@ -103,17 +115,58 @@ public static ExecutionGraph createAndInitExecutionGraph( final ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); - final DefaultScheduler scheduler = - new DefaultSchedulerBuilder(jobGraph, mainThreadExecutor, scheduledExecutorService) - .setIoExecutor(scheduledExecutorService) - .setFutureExecutor(scheduledExecutorService) - .setDelayExecutor( - new ScheduledExecutorServiceAdapter(scheduledExecutorService)) - .build(); + if (jobConfiguration.getSchedulerType() == SchedulerType.AdaptiveBatch) { + Configuration configuration = new Configuration(); + configuration.set(JobManagerOptions.SCHEDULER, SchedulerType.AdaptiveBatch); + + AdaptiveBatchScheduler scheduler = + new DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, scheduledExecutorService) + .setIoExecutor(scheduledExecutorService) + .setFutureExecutor(scheduledExecutorService) + .setDelayExecutor( + new ScheduledExecutorServiceAdapter(scheduledExecutorService)) + .setJobMasterConfiguration(configuration) + .setVertexParallelismAndInputInfosDecider( + createCustomParallelismDecider( + jobConfiguration.getParallelism())) + .buildAdaptiveBatchJobScheduler(); + scheduler.initializeVerticesIfPossible(); + return scheduler; + } else { + return new DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, scheduledExecutorService) + .setIoExecutor(scheduledExecutorService) + .setFutureExecutor(scheduledExecutorService) + .setDelayExecutor(new ScheduledExecutorServiceAdapter(scheduledExecutorService)) + .build(); + } + } + public static ExecutionGraph createAndInitExecutionGraph( + List<JobVertex> jobVertices, + JobConfiguration jobConfiguration, + ScheduledExecutorService scheduledExecutorService) + throws Exception { + DefaultScheduler scheduler = + createAndInitScheduler(jobVertices, jobConfiguration, scheduledExecutorService); return scheduler.getExecutionGraph(); } + public static SchedulingStrategy createSchedulingStrategy( + JobConfiguration jobConfiguration, SchedulingTopology schedulingTopology) { + TestingSchedulerOperations schedulerOperations = new TestingSchedulerOperations(); + + if (jobConfiguration.getSchedulerType() == SchedulerType.AdaptiveBatch) { + return new VertexwiseSchedulingStrategy( + schedulerOperations, + schedulingTopology, + DefaultInputConsumableDecider.Factory.INSTANCE); Review Comment: Hybrid shuffle may use other kinds of deciders. -- 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