This is an automated email from the ASF dual-hosted git repository.
RocMarshal pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-benchmarks.git
The following commit(s) were added to refs/heads/master by this push:
new 414ae78 [FLINK-38944][benchmark] Introduce a benchmark on adaptive
partition selection for RescalePartitioner and RebalancePartitioner
414ae78 is described below
commit 414ae7860d5a5da809465c87ae78e83d3e71cc39
Author: panyuepeng <[email protected]>
AuthorDate: Mon Dec 15 20:29:41 2025 +0800
[FLINK-38944][benchmark] Introduce a benchmark on adaptive partition
selection for RescalePartitioner and RebalancePartitioner
---
.../AdaptivePartitionThroughputBenchmark.java | 107 +++++++++++++++++++++
.../functions/SkewableHigherMultiply.java | 52 ++++++++++
2 files changed, 159 insertions(+)
diff --git
a/src/main/java/org/apache/flink/benchmark/AdaptivePartitionThroughputBenchmark.java
b/src/main/java/org/apache/flink/benchmark/AdaptivePartitionThroughputBenchmark.java
new file mode 100644
index 0000000..7426032
--- /dev/null
+++
b/src/main/java/org/apache/flink/benchmark/AdaptivePartitionThroughputBenchmark.java
@@ -0,0 +1,107 @@
+/*
+ * 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.benchmark.functions.LongSourceType;
+import org.apache.flink.benchmark.functions.MultiplyByTwo;
+import org.apache.flink.benchmark.functions.SkewableHigherMultiply;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.OperationsPerInvocation;
+import org.openjdk.jmh.annotations.Param;
+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;
+
+@OperationsPerInvocation(value =
AdaptivePartitionThroughputBenchmark.RECORDS_PER_INVOCATION)
+public class AdaptivePartitionThroughputBenchmark extends BenchmarkBase {
+ public static final int RECORDS_PER_INVOCATION = 150_000;
+ private static final long CHECKPOINT_INTERVAL_MS = 100;
+
+ @Param({"F27_UNBOUNDED"})
+ public LongSourceType sourceType;
+
+ @Param({"true", "false"})
+ public boolean adaptivePartitioner;
+
+ @Param({"2", "6"})
+ public int adaptivePartitionerMaxTraverseSize;
+
+ public static void main(String[] args) throws RunnerException {
+ Options options =
+ new OptionsBuilder()
+ .verbosity(VerboseMode.NORMAL)
+ .include(
+ ".*"
+ +
AdaptivePartitionThroughputBenchmark.class
+ .getCanonicalName()
+ + ".*")
+ .build();
+
+ new Runner(options).run();
+ }
+
+ @Benchmark
+ public void
mapRebalanceMapRescaleMapSink(InputBenchmarkFlinkEnvironmentContext context)
+ throws Exception {
+
+ StreamExecutionEnvironment env = context.env;
+ env.enableCheckpointing(CHECKPOINT_INTERVAL_MS);
+ env.setParallelism(8);
+ configAdaptivePartitioner(env);
+
+ DataStreamSource<Long> source = sourceType.source(env,
RECORDS_PER_INVOCATION);
+ source.map(new MultiplyByTwo())
+ .rebalance()
+ .map(new SkewableHigherMultiply(1, 0, 5))
+ .rescale()
+ .map(new SkewableHigherMultiply(1, 0, 5))
+ .sinkTo(new DiscardingSink<>());
+
+ env.execute();
+ }
+
+ private void configAdaptivePartitioner(StreamExecutionEnvironment env) {
+ Configuration config = new Configuration();
+ config.set(
+ NettyShuffleEnvironmentOptions.ADAPTIVE_PARTITIONER_ENABLED,
adaptivePartitioner);
+ config.set(
+
NettyShuffleEnvironmentOptions.ADAPTIVE_PARTITIONER_MAX_TRAVERSE_SIZE,
+ adaptivePartitionerMaxTraverseSize);
+ config.setString("restart-strategy", "fixed-delay");
+ config.setString("restart-strategy.fixed-delay.attempts", "15000000");
+ config.setString("restart-strategy.fixed-delay.delay", "3s");
+ env.configure(config);
+ }
+
+ public static class InputBenchmarkFlinkEnvironmentContext extends
FlinkEnvironmentContext {
+
+ @Override
+ protected int getNumberOfSlotsPerTaskManager() {
+ return 16;
+ }
+ }
+}
diff --git
a/src/main/java/org/apache/flink/benchmark/functions/SkewableHigherMultiply.java
b/src/main/java/org/apache/flink/benchmark/functions/SkewableHigherMultiply.java
new file mode 100644
index 0000000..f975694
--- /dev/null
+++
b/src/main/java/org/apache/flink/benchmark/functions/SkewableHigherMultiply.java
@@ -0,0 +1,52 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+public class SkewableHigherMultiply extends RichMapFunction<Long, Long> {
+
+ private final int additionalCostMillisPerRecord;
+ private final Set<Integer> expectedSkewedSubTasksIndexes;
+
+ public SkewableHigherMultiply(
+ int additionalCostMillisPerRecord, Integer...
expectedSkewedSubTasksIndexes) {
+ this.additionalCostMillisPerRecord = additionalCostMillisPerRecord;
+ this.expectedSkewedSubTasksIndexes =
+ new HashSet<>(Arrays.asList(expectedSkewedSubTasksIndexes));
+ }
+
+ @Override
+ public Long map(Long value) throws Exception {
+ double base = value * 2.0d;
+ if (additionalCostMillisPerRecord <= 0) {
+ return (long) base;
+ } else {
+ if (expectedSkewedSubTasksIndexes.contains(
+
getRuntimeContext().getTaskInfo().getIndexOfThisSubtask())) {
+ Thread.sleep(additionalCostMillisPerRecord);
+ }
+ return (long) base;
+ }
+ }
+}