reswqa commented on code in PR #21634: URL: https://github.com/apache/flink/pull/21634#discussion_r1065374786
########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/failover/RestartTasksInBatchJobBenchmarkTest.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.SchedulingStrategy; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; Review Comment: ```suggestion import org.junit.jupiter.api.Test; ``` Newly introduced tests should use Juint5. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/failover/RestartTasksInBatchJobBenchmarkTest.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.SchedulingStrategy; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +/** + * The benchmark of restarting tasks in a BATCH job. The related method is {@link + * SchedulingStrategy#restartTasks}. + */ +public class RestartTasksInBatchJobBenchmarkTest extends TestLogger { + + @Test + public void restartTasksInBatchJob() throws Exception { + RestartTasksInBatchJobBenchmark benchmark = new RestartTasksInBatchJobBenchmark(); + benchmark.setup(JobConfiguration.BATCH_TEST); + benchmark.restartTasks(); + benchmark.teardown(); + } + + @Test + public void restartTasksInAdaptiveBatchJob() throws Exception { Review Comment: ```suggestion void restartTasksInAdaptiveBatchJob() throws Exception { ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/failover/RestartTasksInBatchJobBenchmarkTest.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.SchedulingStrategy; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +/** + * The benchmark of restarting tasks in a BATCH job. The related method is {@link + * SchedulingStrategy#restartTasks}. + */ +public class RestartTasksInBatchJobBenchmarkTest extends TestLogger { + + @Test + public void restartTasksInBatchJob() throws Exception { Review Comment: ```suggestion void restartTasksInBatchJob() throws Exception { ``` package-private is suggested in Junit5. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/scheduling/InitSchedulingStrategyBenchmark.java: ########## @@ -20,20 +20,28 @@ import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; import org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.VertexwiseSchedulingStrategy; + +import static org.apache.flink.runtime.scheduler.benchmark.SchedulerBenchmarkUtils.createSchedulingStrategy; /** - * 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 InitSchedulingStrategyBenchmark extends SchedulingBenchmarkBase { + JobConfiguration jobConfiguration; Review Comment: Maybe private? ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/failover/RestartTasksInBatchJobBenchmarkTest.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.SchedulingStrategy; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +/** + * The benchmark of restarting tasks in a BATCH job. The related method is {@link + * SchedulingStrategy#restartTasks}. + */ +public class RestartTasksInBatchJobBenchmarkTest extends TestLogger { + + @Test + public void restartTasksInBatchJob() throws Exception { + RestartTasksInBatchJobBenchmark benchmark = new RestartTasksInBatchJobBenchmark(); + benchmark.setup(JobConfiguration.BATCH_TEST); + benchmark.restartTasks(); + benchmark.teardown(); + } + + @Test + public void restartTasksInAdaptiveBatchJob() throws Exception { + RestartTasksInBatchJobBenchmark benchmark = new RestartTasksInBatchJobBenchmark(); + benchmark.setup(JobConfiguration.ADAPTIVE_BATCH_TEST); + benchmark.restartTasks(); + benchmark.teardown(); + } + + @Test + public void restartTasksInAdaptiveBatchHybridJob() throws Exception { Review Comment: ```suggestion void restartTasksInAdaptiveBatchHybridJob() throws Exception { ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/scheduling/InitSchedulingStrategyBenchmarkTest.java: ########## @@ -44,4 +46,20 @@ public void initSchedulingStrategyBenchmarkInBatchJob() throws Exception { benchmark.initSchedulingStrategy(); benchmark.teardown(); } + + @Test + public void initSchedulingStrategyBenchmarkInAdaptiveBatchJob() throws Exception { Review Comment: I'm curious about whether the initialization of `VertexWiseSchedulingStrategy` really needs to be tested. After all, there is no time-consuming operation in it. But there are many things to be done in `PipelinedRegionSchedulingStrategy`'s constructor, such as `initCrossRegionConsumedPartitionGroups`. ########## flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/scheduling/StartSchedulingInBatchJobBenchmarkTest.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.scheduling; + +import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +/** + * The benchmark of starting scheduling in a BATCH job. The related method is {@link + * SchedulingStrategy#startScheduling}. + */ +public class StartSchedulingInBatchJobBenchmarkTest extends TestLogger { + + @Test + public void startSchedulingInBatchJobBenchmark() throws Exception { + StartSchedulingInBatchJobBenchmark benchmark = new StartSchedulingInBatchJobBenchmark(); + benchmark.setup(JobConfiguration.BATCH_TEST); + benchmark.startScheduling(); + benchmark.teardown(); + } + + @Test + public void startSchedulingInAdaptiveBatchJobBenchmark() throws Exception { + StartSchedulingInBatchJobBenchmark benchmark = new StartSchedulingInBatchJobBenchmark(); + benchmark.setup(JobConfiguration.ADAPTIVE_BATCH_TEST); + benchmark.startScheduling(); + benchmark.teardown(); + } + + @Test + public void startSchedulingInAdaptiveBatchHybridJobBenchmark() throws Exception { Review Comment: Maybe we also should add a test for hybrid corresponds to a non-dynamic graph, which means that the downstream has been initialized during upstream scheduling. -- 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