zhuzhurk commented on a change in pull request #7: URL: https://github.com/apache/flink-benchmarks/pull/7#discussion_r571595430
########## File path: src/main/java/org/apache/flink/scheduler/benchmark/SchedulerBenchmarkUtils.java ########## @@ -0,0 +1,169 @@ +/* + * 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.scheduler.benchmark; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testtasks.NoOpInvokable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; + +/** + * Utilities for runtime benchmarks. + */ +public class SchedulerBenchmarkUtils { + + public static List<JobVertex> createDefaultJobVertices(JobConfiguration jobConfiguration) { + + List<JobVertex> jobVertices = new ArrayList<>(); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + source.setParallelism(jobConfiguration.getParallelism()); + jobVertices.add(source); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + sink.setParallelism(jobConfiguration.getParallelism()); + jobVertices.add(sink); + + sink.connectNewDataSetAsInput(source, Review comment: The format is broken. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/SchedulerBenchmarkUtils.java ########## @@ -0,0 +1,169 @@ +/* + * 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.scheduler.benchmark; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testtasks.NoOpInvokable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; + +/** + * Utilities for runtime benchmarks. + */ +public class SchedulerBenchmarkUtils { + + public static List<JobVertex> createDefaultJobVertices(JobConfiguration jobConfiguration) { + + List<JobVertex> jobVertices = new ArrayList<>(); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + source.setParallelism(jobConfiguration.getParallelism()); + jobVertices.add(source); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + sink.setParallelism(jobConfiguration.getParallelism()); + jobVertices.add(sink); + + sink.connectNewDataSetAsInput(source, + jobConfiguration.getDistributionPattern(), + jobConfiguration.getResultPartitionType()); + + return jobVertices; + } + + public static JobGraph createJobGraph(JobConfiguration jobConfiguration) throws IOException { + return createJobGraph(Collections.emptyList(), jobConfiguration); + } + + public static JobGraph createJobGraph( + List<JobVertex> jobVertices, + JobConfiguration jobConfiguration) throws IOException { + + final JobGraph jobGraph = new JobGraph(jobVertices.toArray(new JobVertex[0])); + + jobGraph.setScheduleMode(jobConfiguration.getScheduleMode()); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setExecutionMode(jobConfiguration.getExecutionMode()); + jobGraph.setExecutionConfig(executionConfig); + + return jobGraph; + } + + public static ExecutionGraph createAndInitExecutionGraph( + List<JobVertex> jobVertices, + JobConfiguration jobConfiguration) throws Exception { + + final JobGraph jobGraph = createJobGraph( + jobVertices, + jobConfiguration); + + ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); + + final DefaultScheduler scheduler = SchedulerTestingUtils.createScheduler(jobGraph, mainThreadExecutor); + + return scheduler.getExecutionGraph(); + } + + public static void deployTasks( + ExecutionGraph executionGraph, + JobVertexID jobVertexID, + TestingLogicalSlotBuilder slotBuilder, + boolean sendScheduleOrUpdateConsumersMessage) throws JobException, ExecutionException, InterruptedException { + + for (ExecutionVertex vertex : executionGraph.getJobVertex(jobVertexID).getTaskVertices()) { + LogicalSlot slot = slotBuilder.createTestingLogicalSlot(); + Execution execution = vertex.getCurrentExecutionAttempt(); + execution.registerProducedPartitions( + slot.getTaskManagerLocation(), + sendScheduleOrUpdateConsumersMessage).get(); + assignResourceAndDeploy(vertex, slot); + } + } + + public static void deployAllTasks( + ExecutionGraph executionGraph, + TestingLogicalSlotBuilder slotBuilder) throws JobException, ExecutionException, InterruptedException { + + for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { + LogicalSlot slot = slotBuilder.createTestingLogicalSlot(); + vertex.getCurrentExecutionAttempt().registerProducedPartitions(slot.getTaskManagerLocation(), true).get(); + assignResourceAndDeploy(vertex, slot); + } + } + + private static void assignResourceAndDeploy(ExecutionVertex vertex, LogicalSlot slot) throws JobException { + vertex.tryAssignResource(slot); + vertex.deploy(); + } + + public static void transitionTaskStatus( + ExecutionGraph executionGraph, + JobVertexID jobVertexID, + ExecutionState state) { + + for (ExecutionVertex vertex : executionGraph + .getJobVertex(jobVertexID) + .getTaskVertices()) { + executionGraph.updateState(new TaskExecutionStateTransition(new TaskExecutionState( Review comment: NIT: new lines for each nested `new` ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/partitionrelease/PartitionReleaseInBatchJobBenchmark.java ########## @@ -0,0 +1,91 @@ +/* + * 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.scheduler.benchmark.partitionrelease; + +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.scheduler.benchmark.JobConfiguration; +import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkBase; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.runner.RunnerException; + +import java.util.List; + +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createAndInitExecutionGraph; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createDefaultJobVertices; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.deployTasks; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.transitionTaskStatus; + + +public class PartitionReleaseInBatchJobBenchmark extends SchedulerBenchmarkBase { + + @Param("BATCH") + private JobConfiguration jobConfiguration; + + private ExecutionGraph executionGraph; + private JobVertex sink; + + public static void main(String[] args) throws RunnerException { + runBenchmark(PartitionReleaseInBatchJobBenchmark.class); + } + + @Setup(Level.Trial) + public void setup() throws Exception { + final List<JobVertex> jobVertices = createDefaultJobVertices(jobConfiguration); + + executionGraph = createAndInitExecutionGraph(jobVertices, jobConfiguration); + + final JobVertex source = jobVertices.get(0); + sink = jobVertices.get(1); + + final TestingLogicalSlotBuilder slotBuilder = new TestingLogicalSlotBuilder(); + + deployTasks(executionGraph, source.getID(), slotBuilder, true); + + transitionTaskStatus(executionGraph, source.getID(), ExecutionState.FINISHED); + + deployTasks(executionGraph, sink.getID(), slotBuilder, true); + } + + private void clearVariables() { + executionGraph = null; + sink = null; + } + + @TearDown(Level.Iteration) + public void teardownIteration() { Review comment: I guess this `tearDown()` has missed removing? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
