You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/08 13:22:50 UTC

[GitHub] [flink-benchmarks] dawidwys commented on a change in pull request #35: [FLINK-24231] Benchmark for buffer debloating multiple gates

dawidwys commented on a change in pull request #35:
URL: https://github.com/apache/flink-benchmarks/pull/35#discussion_r725006103



##########
File path: src/main/java/org/apache/flink/benchmark/MultiInputCheckpointingTimeBenchmark.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.benchmark.operators.RecordSource;
+import org.apache.flink.benchmark.operators.RecordSource.Record;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.execution.JobClient;
+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.testutils.CommonTestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+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.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.flink.api.common.eventtime.WatermarkStrategy.noWatermarks;
+
+/**
+ * The test verifies that the debloating kicks in and properly downsizes buffers in case of multi input gates with
+ * different throughput. In the end the checkpoint should take ~1(number of rebalance) * DEBLOATING_TARGET.
+ */
+@OutputTimeUnit(SECONDS)
+public class MultiInputCheckpointingTimeBenchmark extends BenchmarkBase {
+
+    public static final int JOB_PARALLELISM = 4;
+    public static final MemorySize START_MEMORY_SEGMENT_SIZE = MemorySize.parse("8 kb");
+    public static final MemorySize MIN_MEMORY_SEGMENT_SIZE = MemorySize.parse("64 b");
+    public static final Duration DEBLOATING_TARGET = Duration.of(300, ChronoUnit.MILLIS);
+    public static final MemorySize SMALL_RECORD_SIZE = MemorySize.parse("1b");
+    public static final MemorySize BIG_RECORD_SIZE = MemorySize.parse("1kb");
+    public static final int DEBLOATING_STABILIZATION_PERIOD = 2_000;
+
+    public static void main(String[] args) throws RunnerException {
+        Options options =
+                new OptionsBuilder()
+                        .verbosity(VerboseMode.NORMAL)
+                        .include(MultiInputCheckpointingTimeBenchmark.class.getCanonicalName())
+                        .build();
+
+        new Runner(options).run();
+    }
+
+    @Benchmark
+    public void checkpointMultiInput(DebloatedCheckpointEnvironmentContext context)
+            throws Exception {
+        final CompletableFuture<String> checkpoint =
+                context.miniCluster.triggerCheckpoint(context.jobID);
+        checkpoint.get();
+    }
+
+    @State(Scope.Thread)
+    public static class DebloatedCheckpointEnvironmentContext extends FlinkEnvironmentContext {
+        public JobID jobID;
+
+        @Override
+        public void setUp() throws Exception {
+            super.setUp();
+            env.setParallelism(JOB_PARALLELISM);
+            env.enableCheckpointing(Long.MAX_VALUE);
+
+            DataStream<Record> source1 =
+                    env.fromSource(
+                                    new RecordSource(
+                                            Integer.MAX_VALUE, (int) SMALL_RECORD_SIZE.getBytes()),
+                                    noWatermarks(),
+                                    RecordSource.class.getName())
+                            .slotSharingGroup("source-small-records")
+                            .rebalance();
+
+            DataStream<Record> source2 =
+                    env.fromSource(
+                                    new RecordSource(
+                                            Integer.MAX_VALUE, (int) BIG_RECORD_SIZE.getBytes()),
+                                    noWatermarks(),
+                                    RecordSource.class.getName())
+                            .slotSharingGroup("source-big-records")
+                            .rebalance();
+
+            source1.connect(source2)
+                    .map(
+                            new CoMapFunction<Record, Record, Record>() {
+                                @Override
+                                public Record map1(Record record) throws Exception {
+                                    return record;
+                                }
+
+                                @Override
+                                public Record map2(Record record) throws Exception {
+                                    return record;
+                                }
+                            })
+                    .name("co-map")
+                    .slotSharingGroup("map-and-sink")
+                    .addSink(new SlowDiscardSink<>())
+                    .slotSharingGroup("map-and-sink");
+
+            final StreamGraph streamGraph = env.getStreamGraph(false);
+            final JobGraph jobGraph = streamGraph.getJobGraph();
+            final List<JobVertex> vertices = jobGraph.getVerticesSortedTopologicallyFromSources();
+
+            final JobClient jobClient = env.executeAsync();
+            jobID = jobClient.getJobID();
+            CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false);
+            List<JobVertexID> sourceIds =
+                    Arrays.asList(vertices.get(0).getID(), vertices.get(1).getID());
+            BackpressureUtils.waitForBackpressure(
+                    jobID, sourceIds, miniCluster.getRestAddress().get());
+            Thread.sleep(DEBLOATING_STABILIZATION_PERIOD);

Review comment:
       The deduplication of code was something I struggled with, I put some thinking into. I agree we could duplicate it. I started with just another parameter in the other test. The reason why I split it in the end, was to be able to tweak individual parameters more easily.
   
   If you really think its better to unify it further I can try, but personally I think easier modifications (that won't break the single input gate test) are worth a little bit of duplication.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org