Thesharing commented on a change in pull request #7: URL: https://github.com/apache/flink-benchmarks/pull/7#discussion_r570777091
########## File path: src/main/java/org/apache/flink/scheduler/benchmark/SchedulerBenchmarkUtils.java ########## @@ -0,0 +1,245 @@ +/* + * 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.api.common.ExecutionMode; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.DummyJobInformation; +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.JobInformation; +import org.apache.flink.runtime.executiongraph.NoOpExecutionDeploymentListener; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.RegionPartitionReleaseStrategy; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +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.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testtasks.NoOpInvokable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; + +/** + * Utilities for runtime benchmarks. + */ +public class SchedulerBenchmarkUtils { + + public static List<JobVertex> createDefaultJobVertices( + int parallelism, + DistributionPattern distributionPattern, + ResultPartitionType resultPartitionType) { + + List<JobVertex> jobVertices = new ArrayList<>(); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + source.setParallelism(parallelism); + jobVertices.add(source); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + sink.setParallelism(parallelism); + jobVertices.add(sink); + + sink.connectNewDataSetAsInput(source, distributionPattern, resultPartitionType); + + return jobVertices; + } + + public static JobGraph createJobGraph( + List<JobVertex> jobVertices, + ScheduleMode scheduleMode, + ExecutionMode executionMode) throws IOException { + + final JobGraph jobGraph = new JobGraph(jobVertices.toArray(new JobVertex[0])); + + jobGraph.setScheduleMode(scheduleMode); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setExecutionMode(executionMode); + jobGraph.setExecutionConfig(executionConfig); + + return jobGraph; + } + + public static ExecutionGraph createExecutionGraph( + JobGraph jobGraph, + SlotProvider slotProvider) throws IOException { + + final JobInformation jobInformation = new DummyJobInformation( + jobGraph.getJobID(), + jobGraph.getName()); + + final ClassLoader classLoader = ExecutionGraph.class.getClassLoader(); + return new ExecutionGraph( + jobInformation, + TestingUtils.defaultExecutor(), + TestingUtils.defaultExecutor(), + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy(), + JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue(), + new RestartAllStrategy.Factory(), + slotProvider, + classLoader, + VoidBlobWriter.getInstance(), + Time.seconds(10L), + new RegionPartitionReleaseStrategy.Factory(), + NettyShuffleMaster.INSTANCE, + NoOpJobMasterPartitionTracker.INSTANCE, + jobGraph.getScheduleMode(), + NoOpExecutionDeploymentListener.INSTANCE, + (execution, newState) -> { + }, + System.currentTimeMillis()); + } + + public static void waitForListFulfilled( + Collection<?> list, + int length, + long maxWaitMillis) throws TimeoutException { + + final Deadline deadline = Deadline.fromNow(Duration.ofMillis(maxWaitMillis)); + final Predicate<Collection<?>> predicate = (Collection<?> l) -> l.size() == length; + boolean predicateResult; + + do { + predicateResult = predicate.test(list); + + if (!predicateResult) { + try { + Thread.sleep(2L); + } catch (InterruptedException ignored) { + Thread.currentThread().interrupt(); + } + } + } while (!predicateResult && deadline.hasTimeLeft()); + + if (!predicateResult) { + throw new TimeoutException(String.format( + "List no fulfilled in time, expected %d, actual %d.", + length, + list.size())); + } + } + + public static void verifyListSize( + Collection<?> list, + int length) { + if (list.size() < length) { + throw new RuntimeException(String.format( + "Size of the list mismatch, expected %d, actual %d.", + length, + list.size())); + } + } + + public static void deployTasks( + ExecutionGraph executionGraph, + JobVertexID jobVertexID, + TestingLogicalSlotBuilder slotBuilder, + boolean sendScheduleOrUpdateConsumersMessage) throws Exception { + + for (ExecutionVertex vertex : executionGraph.getJobVertex(jobVertexID).getTaskVertices()) { + LogicalSlot slot = slotBuilder.createTestingLogicalSlot(); + vertex.getCurrentExecutionAttempt() + .registerProducedPartitions( + slot.getTaskManagerLocation(), + sendScheduleOrUpdateConsumersMessage).get(); + vertex.deployToSlot(slot); Review comment: > `ExecutionVertex#deployToSlot()` is not used in production. What `DefaultExecutionVertexOperations#deploy()` invokes is `ExecutionVertex#deploy()`. Yes, I find that `deployToSlot` is only used in test cases. I've fixed it. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/JobConfiguration.java ########## @@ -0,0 +1,57 @@ +/* + * 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.ExecutionMode; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.ScheduleMode; + +public enum JobConfiguration { + + STREAMING(DistributionPattern.ALL_TO_ALL, + ResultPartitionType.PIPELINED, + ScheduleMode.EAGER, + ExecutionMode.PIPELINED), + + BATCH(DistributionPattern.ALL_TO_ALL, + ResultPartitionType.BLOCKING, + ScheduleMode.LAZY_FROM_SOURCES, + ExecutionMode.BATCH); + + JobConfiguration( + DistributionPattern distributionPattern, + ResultPartitionType resultPartitionType, + ScheduleMode scheduleMode, + ExecutionMode executionMode) { + this.distributionPattern = distributionPattern; + this.resultPartitionType = resultPartitionType; + this.scheduleMode = scheduleMode; + this.executionMode = executionMode; + } + + public final DistributionPattern distributionPattern; Review comment: Great idea. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksBenchmarkBase.java ########## @@ -0,0 +1,83 @@ +/* + * 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.deploying; + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.scheduler.benchmark.ColdStartSchedulerBenchmarkBase; +import org.apache.flink.scheduler.benchmark.JobConfiguration; + +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createAndInitExecutionGraph; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createDefaultJobVertices; + +public class DeployingTasksBenchmarkBase extends ColdStartSchedulerBenchmarkBase { + + List<JobVertex> jobVertices; + ExecutionGraph executionGraph; + BlockingQueue<TaskDeploymentDescriptor> taskDeploymentDescriptors; + + public void createAndSetupExecutionGraph(JobConfiguration jobConfiguration) throws Exception { + + jobVertices = createDefaultJobVertices( Review comment: Done. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksBenchmarkBase.java ########## @@ -0,0 +1,83 @@ +/* + * 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.deploying; + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.scheduler.benchmark.ColdStartSchedulerBenchmarkBase; +import org.apache.flink.scheduler.benchmark.JobConfiguration; + +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createAndInitExecutionGraph; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createDefaultJobVertices; + +public class DeployingTasksBenchmarkBase extends ColdStartSchedulerBenchmarkBase { + + List<JobVertex> jobVertices; + ExecutionGraph executionGraph; + BlockingQueue<TaskDeploymentDescriptor> taskDeploymentDescriptors; + + public void createAndSetupExecutionGraph(JobConfiguration jobConfiguration) throws Exception { + + jobVertices = createDefaultJobVertices( + PARALLELISM, + jobConfiguration.distributionPattern, + jobConfiguration.resultPartitionType); + + executionGraph = createAndInitExecutionGraph( Review comment: Done. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksInStreamingJobBenchmark.java ########## @@ -0,0 +1,77 @@ +/* + * 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.deploying; + +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.scheduler.benchmark.JobConfiguration; +import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils; + +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.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.VerboseMode; + + +public class DeployingTasksInStreamingJobBenchmark extends DeployingTasksBenchmarkBase { + + @Param("STREAMING") + private JobConfiguration jobConfiguration; + + public static void main(String[] args) throws RunnerException { + Options options = new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(".*" + DeployingTasksInStreamingJobBenchmark.class.getCanonicalName() + ".*") + .build(); + + new Runner(options).run(); + } + + @Setup(Level.Iteration) + public void setupIteration() throws Exception { + createAndSetupExecutionGraph(JobConfiguration.STREAMING); + } + + @TearDown(Level.Iteration) + public void teardownIteration() throws Exception { + SchedulerBenchmarkUtils.waitForListFulfilled(taskDeploymentDescriptors, PARALLELISM * 2, 1000L); + clearVariables(); + System.gc(); Review comment: No need. Removed. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/SchedulerBenchmarkUtils.java ########## @@ -0,0 +1,269 @@ +/* + * 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.api.common.ExecutionMode; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.DummyJobInformation; +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.JobInformation; +import org.apache.flink.runtime.executiongraph.NoOpExecutionDeploymentListener; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.RegionPartitionReleaseStrategy; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +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.jobgraph.ScheduleMode; +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.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testtasks.NoOpInvokable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; + +/** + * Utilities for runtime benchmarks. + */ +public class SchedulerBenchmarkUtils { + + public static List<JobVertex> createDefaultJobVertices( + int parallelism, + DistributionPattern distributionPattern, + ResultPartitionType resultPartitionType) { + + List<JobVertex> jobVertices = new ArrayList<>(); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + source.setParallelism(parallelism); + jobVertices.add(source); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + sink.setParallelism(parallelism); + jobVertices.add(sink); + + sink.connectNewDataSetAsInput(source, distributionPattern, resultPartitionType); + + return jobVertices; + } + + public static JobGraph createJobGraph( + List<JobVertex> jobVertices, + ScheduleMode scheduleMode, + ExecutionMode executionMode) throws IOException { + + final JobGraph jobGraph = new JobGraph(jobVertices.toArray(new JobVertex[0])); + + jobGraph.setScheduleMode(scheduleMode); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setExecutionMode(executionMode); + jobGraph.setExecutionConfig(executionConfig); + + return jobGraph; + } + + public static ExecutionGraph createExecutionGraph(JobGraph jobGraph) throws IOException { + + final JobInformation jobInformation = new DummyJobInformation( + jobGraph.getJobID(), + jobGraph.getName()); + + final ClassLoader classLoader = ExecutionGraph.class.getClassLoader(); + return new ExecutionGraph( + jobInformation, + TestingUtils.defaultExecutor(), + TestingUtils.defaultExecutor(), + AkkaUtils.getDefaultTimeout(), + JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue(), + classLoader, + VoidBlobWriter.getInstance(), + new RegionPartitionReleaseStrategy.Factory(), + NettyShuffleMaster.INSTANCE, + NoOpJobMasterPartitionTracker.INSTANCE, + jobGraph.getScheduleMode(), + NoOpExecutionDeploymentListener.INSTANCE, + (execution, newState) -> { + }, + System.currentTimeMillis()); + } + + public static ExecutionGraph createAndInitExecutionGraph( + List<JobVertex> jobVertices, + ScheduleMode scheduleMode, + ExecutionMode executionMode) throws IOException, JobException { + + final JobGraph jobGraph = createJobGraph( + jobVertices, + scheduleMode, + executionMode); + + ExecutionGraph executionGraph = createExecutionGraph(jobGraph); + + executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); + + return executionGraph; + } + + public static void waitForListFulfilled( + Collection<?> list, + int length, + long maxWaitMillis) throws TimeoutException { + + final Deadline deadline = Deadline.fromNow(Duration.ofMillis(maxWaitMillis)); + final Predicate<Collection<?>> predicate = (Collection<?> l) -> l.size() == length; + boolean predicateResult; + + do { + predicateResult = predicate.test(list); + + if (!predicateResult) { + try { + Thread.sleep(2L); + } catch (InterruptedException ignored) { + Thread.currentThread().interrupt(); + } + } + } while (!predicateResult && deadline.hasTimeLeft()); + + if (!predicateResult) { + throw new TimeoutException(String.format( + "List no fulfilled in time, expected %d, actual %d.", Review comment: Removed. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksInStreamingJobBenchmark.java ########## @@ -0,0 +1,77 @@ +/* + * 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.deploying; + +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.scheduler.benchmark.JobConfiguration; +import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils; + +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.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.VerboseMode; + + +public class DeployingTasksInStreamingJobBenchmark extends DeployingTasksBenchmarkBase { + + @Param("STREAMING") + private JobConfiguration jobConfiguration; + + public static void main(String[] args) throws RunnerException { + Options options = new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(".*" + DeployingTasksInStreamingJobBenchmark.class.getCanonicalName() + ".*") + .build(); + + new Runner(options).run(); Review comment: Great idea. Done. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksInStreamingJobBenchmark.java ########## @@ -0,0 +1,77 @@ +/* + * 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.deploying; + +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.scheduler.benchmark.JobConfiguration; +import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils; + +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.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.VerboseMode; + + +public class DeployingTasksInStreamingJobBenchmark extends DeployingTasksBenchmarkBase { + + @Param("STREAMING") + private JobConfiguration jobConfiguration; + + public static void main(String[] args) throws RunnerException { + Options options = new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(".*" + DeployingTasksInStreamingJobBenchmark.class.getCanonicalName() + ".*") + .build(); + + new Runner(options).run(); + } + + @Setup(Level.Iteration) + public void setupIteration() throws Exception { + createAndSetupExecutionGraph(JobConfiguration.STREAMING); + } + + @TearDown(Level.Iteration) + public void teardownIteration() throws Exception { + SchedulerBenchmarkUtils.waitForListFulfilled(taskDeploymentDescriptors, PARALLELISM * 2, 1000L); Review comment: Removed. ########## File path: src/main/java/org/apache/flink/scheduler/benchmark/topology/BuildExecutionGraphBenchmark.java ########## @@ -0,0 +1,90 @@ +/* + * 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.topology; + +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.scheduler.benchmark.ColdStartSchedulerBenchmarkBase; +import org.apache.flink.scheduler.benchmark.JobConfiguration; + +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.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.VerboseMode; + +import java.util.List; + +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createDefaultJobVertices; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createExecutionGraph; +import static org.apache.flink.scheduler.benchmark.SchedulerBenchmarkUtils.createJobGraph; + + +public class BuildExecutionGraphBenchmark extends ColdStartSchedulerBenchmarkBase { + + private JobGraph jobGraph; + private ExecutionGraph executionGraph; + + @Param({"BATCH", "STREAMING"}) + private JobConfiguration jobConfiguration; + + public static void main(String[] args) throws RunnerException { + Options options = new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(".*" + BuildExecutionGraphBenchmark.class.getCanonicalName() + ".*") + .build(); + + new Runner(options).run(); + } + + @Setup(Level.Iteration) + public void setupIteration() throws Exception { + final List<JobVertex> jobVertices = createDefaultJobVertices( + PARALLELISM, + jobConfiguration.distributionPattern, + jobConfiguration.resultPartitionType); + jobGraph = createJobGraph( + jobVertices, + jobConfiguration.scheduleMode, + jobConfiguration.executionMode); + + executionGraph = createExecutionGraph(jobGraph); + } + + @TearDown(Level.Iteration) + public void teardownIteration() { + jobGraph = null; + executionGraph = null; + System.gc(); + } + + @Benchmark + @BenchmarkMode(Mode.SingleShotTime) + public void buildTopology() throws Exception { + executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); Review comment: Yes. Done. ---------------------------------------------------------------- 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: us...@infra.apache.org