pnowojski commented on a change in pull request #32: URL: https://github.com/apache/flink-benchmarks/pull/32#discussion_r716465154
########## File path: src/main/java/org/apache/flink/benchmark/BufferDebloatedCheckpointTimeBenchmark.java ########## @@ -0,0 +1,261 @@ +/* + * 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.benchmark; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.benchmark.operators.RecordSource; +import org.apache.flink.benchmark.operators.RecordSource.Record; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo; +import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +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.net.URI; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.api.common.eventtime.WatermarkStrategy.noWatermarks; + +/** + * The test verifies that the debloating kicks in and properly downsizes buffers. In the end the checkpoint should + * take ~2(number of rebalance) * DEBLOATING_TARGET. + * + * <p>Some info about the chosen numbers: + * <ul> + * <li>The minimal memory segment size is decreased (256b) so that the scaling possibility is higher. Memory segments + * start with 4kb</li> + * <li>A memory segment of the minimal size fits ~3 records (of size 64b), each record takes ~1ms to be processed by + * the sink</li> + * <li>We have 2 (exclusive buffers) * 4 (parallelism) + 8 floating = 64 buffers per gate, with 300 ms debloating + * target and ~1ms/record processing speed, we can buffer 300/64 = ~4.5 records in a buffer after debloating which + * means the size of a buffer is slightly above the minimal memory segment size.</li> + * <li>The buffer debloating target of 300ms means a checkpoint should take ~2(number of exchanges)*300ms=~600ms</li> + * </ul> + */ +@State(Scope.Thread) +@OutputTimeUnit(MILLISECONDS) +@BenchmarkMode(Mode.SingleShotTime) +@Fork( + value = 3, + jvmArgsAppend = { + "-Djava.rmi.server.hostname=127.0.0.1", + "-Dcom.sun.management.jmxremote.authenticate=false", + "-Dcom.sun.management.jmxremote.ssl=false", + "-Dcom.sun.management.jmxremote.ssl" + }) +@Warmup(iterations = 4) +@Measurement(iterations = 10) +public class BufferDebloatedCheckpointTimeBenchmark { + public static final int JOB_PARALLELISM = 4; + public static final MemorySize START_MEMORY_SEGMENT_SIZE = MemorySize.parse("4 kb"); + public static final MemorySize MIN_MEMORY_SEGMENT_SIZE = MemorySize.parse("256 b"); + public static final Duration DEBLOATING_TARGET = Duration.of(300, ChronoUnit.MILLIS); + public static final int RECORD_SIZE = (int) MemorySize.parse("64b").getBytes(); + + public static void main(String[] args) throws RunnerException { + Options options = + new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(BufferDebloatedCheckpointTimeBenchmark.class.getCanonicalName()) + .build(); + + new Runner(options).run(); + } + + @Benchmark + public void unalignedCheckpoint(DebloatedCheckpointEnvironmentContext context) Review comment: benchmark name is incorrect. `debloatedCheckpoint`? `debloatedCheckpointSingleInput`? Or `alignedCheckpointSingleInput`? `alignedCheckpoint.SINGLE` (`SINGLE` - being a parameter), to keep it in sync with `unalignedCheckpoint` benchmarks? ########## File path: src/main/java/org/apache/flink/benchmark/BufferDebloatedCheckpointTimeBenchmark.java ########## @@ -0,0 +1,261 @@ +/* + * 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.benchmark; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.benchmark.operators.RecordSource; +import org.apache.flink.benchmark.operators.RecordSource.Record; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo; +import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +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.net.URI; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.api.common.eventtime.WatermarkStrategy.noWatermarks; + +/** + * The test verifies that the debloating kicks in and properly downsizes buffers. In the end the checkpoint should + * take ~2(number of rebalance) * DEBLOATING_TARGET. + * + * <p>Some info about the chosen numbers: + * <ul> + * <li>The minimal memory segment size is decreased (256b) so that the scaling possibility is higher. Memory segments + * start with 4kb</li> + * <li>A memory segment of the minimal size fits ~3 records (of size 64b), each record takes ~1ms to be processed by + * the sink</li> + * <li>We have 2 (exclusive buffers) * 4 (parallelism) + 8 floating = 64 buffers per gate, with 300 ms debloating + * target and ~1ms/record processing speed, we can buffer 300/64 = ~4.5 records in a buffer after debloating which + * means the size of a buffer is slightly above the minimal memory segment size.</li> + * <li>The buffer debloating target of 300ms means a checkpoint should take ~2(number of exchanges)*300ms=~600ms</li> + * </ul> + */ +@State(Scope.Thread) +@OutputTimeUnit(MILLISECONDS) +@BenchmarkMode(Mode.SingleShotTime) +@Fork( + value = 3, + jvmArgsAppend = { + "-Djava.rmi.server.hostname=127.0.0.1", + "-Dcom.sun.management.jmxremote.authenticate=false", + "-Dcom.sun.management.jmxremote.ssl=false", + "-Dcom.sun.management.jmxremote.ssl" + }) +@Warmup(iterations = 4) +@Measurement(iterations = 10) +public class BufferDebloatedCheckpointTimeBenchmark { + public static final int JOB_PARALLELISM = 4; + public static final MemorySize START_MEMORY_SEGMENT_SIZE = MemorySize.parse("4 kb"); + public static final MemorySize MIN_MEMORY_SEGMENT_SIZE = MemorySize.parse("256 b"); + public static final Duration DEBLOATING_TARGET = Duration.of(300, ChronoUnit.MILLIS); + public static final int RECORD_SIZE = (int) MemorySize.parse("64b").getBytes(); + + public static void main(String[] args) throws RunnerException { + Options options = + new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(BufferDebloatedCheckpointTimeBenchmark.class.getCanonicalName()) + .build(); + + new Runner(options).run(); + } + + @Benchmark + public void unalignedCheckpoint(DebloatedCheckpointEnvironmentContext context) + throws Exception { + final CompletableFuture<String> checkpoint = context.miniCluster.triggerCheckpoint(context.jobID); + checkpoint.get(); + } + + @State(Scope.Thread) + public static class DebloatedCheckpointEnvironmentContext { + public JobID jobID; + public StreamExecutionEnvironment env; + public MiniCluster miniCluster; + + @Setup(Level.Invocation) Review comment: Generally speaking it's best to avoid `Level.Invocation`. Here it might be irrelevant (as single invocation is quite long and is mostly sleeping), but... Why do we need to setup per each invocation? Why not use `Trial`? We do not need to start up Flink cluster per each invocation or even iteration. We should be able to re-use the cluster. This would also help to hide `Thread.sleep(2_000);` cost. ########## File path: src/main/java/org/apache/flink/benchmark/BufferDebloatedCheckpointTimeBenchmark.java ########## @@ -0,0 +1,261 @@ +/* + * 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.benchmark; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.benchmark.operators.RecordSource; +import org.apache.flink.benchmark.operators.RecordSource.Record; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo; +import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +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.net.URI; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.api.common.eventtime.WatermarkStrategy.noWatermarks; + +/** + * The test verifies that the debloating kicks in and properly downsizes buffers. In the end the checkpoint should + * take ~2(number of rebalance) * DEBLOATING_TARGET. + * + * <p>Some info about the chosen numbers: + * <ul> + * <li>The minimal memory segment size is decreased (256b) so that the scaling possibility is higher. Memory segments + * start with 4kb</li> + * <li>A memory segment of the minimal size fits ~3 records (of size 64b), each record takes ~1ms to be processed by + * the sink</li> + * <li>We have 2 (exclusive buffers) * 4 (parallelism) + 8 floating = 64 buffers per gate, with 300 ms debloating + * target and ~1ms/record processing speed, we can buffer 300/64 = ~4.5 records in a buffer after debloating which + * means the size of a buffer is slightly above the minimal memory segment size.</li> + * <li>The buffer debloating target of 300ms means a checkpoint should take ~2(number of exchanges)*300ms=~600ms</li> + * </ul> + */ +@State(Scope.Thread) +@OutputTimeUnit(MILLISECONDS) +@BenchmarkMode(Mode.SingleShotTime) +@Fork( + value = 3, + jvmArgsAppend = { + "-Djava.rmi.server.hostname=127.0.0.1", + "-Dcom.sun.management.jmxremote.authenticate=false", + "-Dcom.sun.management.jmxremote.ssl=false", + "-Dcom.sun.management.jmxremote.ssl" + }) +@Warmup(iterations = 4) +@Measurement(iterations = 10) +public class BufferDebloatedCheckpointTimeBenchmark { Review comment: Could you replace the existing `unalignedCheckpoint.0` and `unalignedCheckpoint.1` benchmarks with this one? ########## File path: src/main/java/org/apache/flink/benchmark/BufferDebloatedCheckpointTimeBenchmark.java ########## @@ -0,0 +1,261 @@ +/* + * 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.benchmark; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.benchmark.operators.RecordSource; +import org.apache.flink.benchmark.operators.RecordSource.Record; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo; +import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +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.net.URI; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.api.common.eventtime.WatermarkStrategy.noWatermarks; + +/** + * The test verifies that the debloating kicks in and properly downsizes buffers. In the end the checkpoint should + * take ~2(number of rebalance) * DEBLOATING_TARGET. + * + * <p>Some info about the chosen numbers: + * <ul> + * <li>The minimal memory segment size is decreased (256b) so that the scaling possibility is higher. Memory segments + * start with 4kb</li> + * <li>A memory segment of the minimal size fits ~3 records (of size 64b), each record takes ~1ms to be processed by + * the sink</li> + * <li>We have 2 (exclusive buffers) * 4 (parallelism) + 8 floating = 64 buffers per gate, with 300 ms debloating + * target and ~1ms/record processing speed, we can buffer 300/64 = ~4.5 records in a buffer after debloating which + * means the size of a buffer is slightly above the minimal memory segment size.</li> + * <li>The buffer debloating target of 300ms means a checkpoint should take ~2(number of exchanges)*300ms=~600ms</li> + * </ul> + */ +@State(Scope.Thread) +@OutputTimeUnit(MILLISECONDS) +@BenchmarkMode(Mode.SingleShotTime) +@Fork( + value = 3, + jvmArgsAppend = { + "-Djava.rmi.server.hostname=127.0.0.1", + "-Dcom.sun.management.jmxremote.authenticate=false", + "-Dcom.sun.management.jmxremote.ssl=false", + "-Dcom.sun.management.jmxremote.ssl" + }) +@Warmup(iterations = 4) +@Measurement(iterations = 10) +public class BufferDebloatedCheckpointTimeBenchmark { + public static final int JOB_PARALLELISM = 4; + public static final MemorySize START_MEMORY_SEGMENT_SIZE = MemorySize.parse("4 kb"); + public static final MemorySize MIN_MEMORY_SEGMENT_SIZE = MemorySize.parse("256 b"); + public static final Duration DEBLOATING_TARGET = Duration.of(300, ChronoUnit.MILLIS); + public static final int RECORD_SIZE = (int) MemorySize.parse("64b").getBytes(); + + public static void main(String[] args) throws RunnerException { + Options options = + new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .include(BufferDebloatedCheckpointTimeBenchmark.class.getCanonicalName()) + .build(); + + new Runner(options).run(); + } + + @Benchmark + public void unalignedCheckpoint(DebloatedCheckpointEnvironmentContext context) + throws Exception { + final CompletableFuture<String> checkpoint = context.miniCluster.triggerCheckpoint(context.jobID); + checkpoint.get(); + } + + @State(Scope.Thread) + public static class DebloatedCheckpointEnvironmentContext { + public JobID jobID; + public StreamExecutionEnvironment env; + public MiniCluster miniCluster; + + @Setup(Level.Invocation) + public void setUp() throws Exception { + setUpEnvironment(); + + DataStreamSource<Record> source = + env.fromSource( + new RecordSource(Integer.MAX_VALUE, RECORD_SIZE), + noWatermarks(), + RecordSource.class.getName()); + + source.slotSharingGroup("source") + .rebalance() + .map((MapFunction<Record, Record>) value -> value) + .slotSharingGroup("map") + .rebalance() + .addSink(new SlowDiscardSink<>()) + .slotSharingGroup("sink"); + + final JobVertexID sourceId = extractSourceId(); + final JobClient jobClient = env.executeAsync(); + jobID = jobClient.getJobID(); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); + waitForBackpressure(jobID, sourceId); + // wait a bit for debloating to stabilize + Thread.sleep(2_000); + } + + private void setUpEnvironment() throws Exception { + if (miniCluster != null) { + throw new RuntimeException("setUp was called multiple times!"); + } + final Configuration clusterConfig = createConfiguration(); + miniCluster = new MiniCluster(new MiniClusterConfiguration.Builder() + .setNumSlotsPerTaskManager(getNumberOfSlotsPerTaskManager()) + .setNumTaskManagers(getNumberOfTaskManagers()) + .setConfiguration(clusterConfig) + .build()); + miniCluster.start(); + + // set up the execution environment + env = new StreamExecutionEnvironment( + new MiniClusterPipelineExecutorServiceLoader(miniCluster), + clusterConfig, + null); + + env.setRestartStrategy(RestartStrategies.noRestart()); + } + + private JobVertexID extractSourceId() { + return env.getStreamGraph(false) + .getJobGraph() + .getVerticesSortedTopologicallyFromSources() + .get(0) + .getID(); + } + + private void waitForBackpressure( + JobID jobID, + JobVertexID sourceId) throws Exception { + final RestClient restClient = new RestClient( + new UnmodifiableConfiguration(new Configuration()), + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-RestClient-IO"))); + final URI restAddress = miniCluster.getRestAddress().get(); + final JobVertexMessageParameters metricsParameters = new JobVertexMessageParameters(); + metricsParameters.jobPathParameter.resolve(jobID); + metricsParameters.jobVertexIdPathParameter.resolve(sourceId); + JobVertexBackPressureInfo responseBody; + do { + responseBody = restClient.sendRequest( + restAddress.getHost(), + restAddress.getPort(), + JobVertexBackPressureHeaders.getInstance(), + metricsParameters, + EmptyRequestBody.getInstance()).get(); + } while (responseBody.getBackpressureLevel() != JobVertexBackPressureInfo.VertexBackPressureLevel.HIGH); + } Review comment: Maybe add a timeout? I don't think we have any watchdog in benchmarks. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
