├── .asf.yaml ├── .github └── workflows │ └── ci.yml ├── .gitignore ├── .mvn └── wrapper │ └── maven-wrapper.properties ├── LICENSE ├── README.md ├── benchmark.sh ├── jenkinsfiles ├── benchmark.jenkinsfile └── regression-check.jenkinsfile ├── mvnw ├── mvnw.cmd ├── pom.xml ├── regression_report.py ├── regression_report_v2.py ├── save_jmh_result.py └── src ├── main ├── java │ └── org │ │ └── apache │ │ └── flink │ │ ├── benchmark │ │ ├── AsyncWaitOperatorBenchmark.java │ │ ├── BackpressureUtils.java │ │ ├── BenchmarkBase.java │ │ ├── BlockingPartitionBenchmark.java │ │ ├── BlockingPartitionRemoteChannelBenchmark.java │ │ ├── CheckpointEnvironmentContext.java │ │ ├── CheckpointingTimeBenchmark.java │ │ ├── CollectSink.java │ │ ├── ContinuousFileReaderOperatorBenchmark.java │ │ ├── FlinkEnvironmentContext.java │ │ ├── HighAvailabilityServiceBenchmark.java │ │ ├── InputBenchmark.java │ │ ├── KeyByBenchmarks.java │ │ ├── MemoryStateBackendBenchmark.java │ │ ├── MultiInputCheckpointingTimeBenchmark.java │ │ ├── MultipleInputBenchmark.java │ │ ├── ProcessingTimerBenchmark.java │ │ ├── RemoteBenchmarkBase.java │ │ ├── RemoteChannelThroughputBenchmark.java │ │ ├── RocksStateBackendBenchmark.java │ │ ├── SerializationFrameworkMiniBenchmarks.java │ │ ├── SortingBoundedInputBenchmarks.java │ │ ├── StateBackendBenchmarkBase.java │ │ ├── StreamGraphUtils.java │ │ ├── TwoInputBenchmark.java │ │ ├── WatermarkAggregationBenchmark.java │ │ ├── WindowBenchmarks.java │ │ ├── full │ │ │ ├── PojoSerializationBenchmark.java │ │ │ ├── SerializationFrameworkAllBenchmarks.java │ │ │ ├── StringSerializationBenchmark.java │ │ │ └── package-info.java │ │ ├── functions │ │ │ ├── BaseSourceWithKeyRange.java │ │ │ ├── IntLongApplications.java │ │ │ ├── IntegerLongSource.java │ │ │ ├── LongNewSource.java │ │ │ ├── LongSource.java │ │ │ ├── LongSourceType.java │ │ │ ├── MultiplyByTwo.java │ │ │ ├── MultiplyIntLongByTwo.java │ │ │ ├── QueuingLongSource.java │ │ │ ├── SuccessException.java │ │ │ ├── SumReduce.java │ │ │ ├── SumReduceIntLong.java │ │ │ ├── TestUtils.java │ │ │ └── ValidatingCounter.java │ │ ├── operators │ │ │ ├── MultiplyByTwoCoStreamMap.java │ │ │ ├── MultiplyByTwoOperatorFactory.java │ │ │ └── RecordSource.java │ │ └── thrift │ │ │ ├── MyOperation.java │ │ │ └── MyPojo.java │ │ ├── config │ │ ├── ConfigUtil.java │ │ └── StateBenchmarkOptions.java │ │ ├── scheduler │ │ └── benchmark │ │ │ ├── SchedulerBenchmarkExecutorBase.java │ │ │ ├── deploying │ │ │ ├── DeployingDownstreamTasksInBatchJobBenchmarkExecutor.java │ │ │ └── DeployingTasksInStreamingJobBenchmarkExecutor.java │ │ │ ├── e2e │ │ │ ├── CreateSchedulerBenchmarkExecutor.java │ │ │ ├── HandleGlobalFailureAndRestartAllTasksBenchmarkExecutor.java │ │ │ └── SchedulingAndDeployingBenchmarkExecutor.java │ │ │ ├── failover │ │ │ ├── RegionToRestartInBatchJobBenchmarkExecutor.java │ │ │ └── RegionToRestartInStreamingJobBenchmarkExecutor.java │ │ │ ├── partitionrelease │ │ │ └── PartitionReleaseInBatchJobBenchmarkExecutor.java │ │ │ ├── scheduling │ │ │ ├── InitSchedulingStrategyBenchmarkExecutor.java │ │ │ └── SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java │ │ │ └── topology │ │ │ └── BuildExecutionGraphBenchmarkExecutor.java │ │ └── state │ │ └── benchmark │ │ ├── HashMapStateBackendRescalingBenchmarkExecutor.java │ │ ├── ListStateBenchmark.java │ │ ├── MapStateBenchmark.java │ │ ├── RescalingBenchmarkBase.java │ │ ├── RocksdbStateBackendRescalingBenchmarkExecutor.java │ │ ├── StateBenchmarkBase.java │ │ ├── StateBenchmarkConstants.java │ │ ├── ValueStateBenchmark.java │ │ └── ttl │ │ ├── TtlListStateBenchmark.java │ │ ├── TtlMapStateBenchmark.java │ │ ├── TtlStateBenchmarkBase.java │ │ └── TtlValueStateBenchmark.java └── resources │ ├── avro │ └── mypojo.avsc │ ├── benchmark-conf.yaml │ ├── log4j2-test.properties │ ├── protobuf │ └── MyPojo.proto │ └── thrift │ └── mypojo.thrift └── test ├── java └── org │ └── apache │ └── flink │ ├── benchmark │ ├── DataSkewStreamNetworkThroughputBenchmarkExecutor.java │ ├── StreamNetworkBroadcastThroughputBenchmarkExecutor.java │ ├── StreamNetworkLatencyBenchmarkExecutor.java │ └── StreamNetworkThroughputBenchmarkExecutor.java │ └── config │ └── ConfigUtilTest.java └── resources ├── benchmark-conf.yaml ├── local127.keystore └── local127.truststore /.asf.yaml: -------------------------------------------------------------------------------- 1 | github: 2 | enabled_merge_buttons: 3 | squash: true 4 | merge: false 5 | rebase: true 6 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | name: CI 17 | 18 | on: [push, pull_request] 19 | 20 | jobs: 21 | build: 22 | runs-on: ubuntu-latest 23 | 24 | steps: 25 | - uses: actions/checkout@v3 26 | 27 | - name: Set JDK 28 | uses: actions/setup-java@v3 29 | with: 30 | java-version: 17 31 | distribution: 'temurin' 32 | cache: 'maven' 33 | 34 | - name: Set Maven 3.8.6 35 | uses: stCarolas/setup-maven@v4.5 36 | with: 37 | maven-version: 3.8.6 38 | 39 | - name: Install prerequisites 40 | run: | 41 | wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.13_amd64.deb 42 | sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.13_amd64.deb 43 | rm libssl1.0.0_1.0.2n-1ubuntu5.13_amd64.deb 44 | 45 | - name: Compile 46 | # https.protocols is a workaround for https://bugs.openjdk.java.net/browse/JDK-8213202 47 | run: mvn install -P test-java17 -DskipTests=true -Dmaven.javadoc.skip=true -Dhttps.protocols=TLSv1,TLSv1.1,TLSv1.2 -B -V 48 | 49 | - name: Test 50 | run: mvn test -P test-java17 -B 51 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .cache 2 | .classpath 3 | .idea 4 | .metadata 5 | .settings 6 | .project 7 | .version.properties 8 | filter.properties 9 | target 10 | tmp 11 | *.class 12 | *.iml 13 | *.swp 14 | *.jar 15 | *.log 16 | .DS_Store 17 | atlassian-ide-plugin.xml 18 | out/ 19 | *.ipr 20 | *.iws 21 | jmh-result*.csv 22 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.3.9/apache-maven-3.3.9-bin.zip 18 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar 19 | -------------------------------------------------------------------------------- /benchmark.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | ################################################################################ 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | ################################################################################ 19 | 20 | JAVA_ARGS=() 21 | JMH_ARGS=() 22 | BINARY="java" 23 | BENCHMARK_PATTERN= 24 | 25 | while getopts ":j:c:b:e:p:a:m:h" opt; do 26 | case $opt in 27 | j) JAVA_ARGS+=("${OPTARG}") 28 | ;; 29 | c) CLASSPATH_ARG="${OPTARG}" 30 | ;; 31 | b) BINARY="${OPTARG}" 32 | ;; 33 | p) PROFILER_ARG="${OPTARG:+-prof ${OPTARG}}" 34 | # conditional prefixing inspired by https://stackoverflow.com/a/40771884/1389220 35 | ;; 36 | a) JMH_ARGS+=("${OPTARG}") 37 | ;; 38 | e) BENCHMARK_EXCLUDES="${OPTARG:+-e ${OPTARG}}" 39 | ;; 40 | m) BENCHMARK_PATTERN="${OPTARG}" 41 | ;; 42 | h) 43 | 1>&2 cat << EOF 44 | usage: $0 -c ${CLASSPATH} [-j JVM_ARG] [-b /path/to/java] [-a JMH_ARG] [-p :&2 56 | exit 1 57 | ;; 58 | esac 59 | done 60 | shift "$(($OPTIND -1))" 61 | 62 | # shellcheck disable=SC2086 63 | ${BINARY} "${JAVA_ARGS[@]}" \ 64 | -classpath "${CLASSPATH_ARG}" \ 65 | org.openjdk.jmh.Main \ 66 | -foe true \ 67 | -rf csv \ 68 | "${JMH_ARGS[@]}" \ 69 | ${PROFILER_ARG:-} \ 70 | ${BENCHMARK_EXCLUDES:-} \ 71 | "${BENCHMARK_PATTERN:-.*}" 72 | -------------------------------------------------------------------------------- /jenkinsfiles/regression-check.jenkinsfile: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | timestamps { 19 | try { 20 | timeout(time: 3, unit: 'HOURS') { // includes waiting for a machine 21 | node('Aliyun') { 22 | dir('flink-benchmarks') { 23 | git url: 'https://github.com/apache/flink-benchmarks.git', branch: 'master' 24 | sh 'python2 ./regression_report_v2.py > regression-report' 25 | def alerts = readFile "regression-report" 26 | if (alerts) { 27 | def attachments = [ 28 | [ 29 | text: "Performance regression\n$alerts", 30 | fallback: "Performance regression", 31 | ] 32 | ] 33 | slackSend (attachments: attachments, channel: "#flink-dev-benchmarks") 34 | } else { 35 | slackSend (channel: "#flink-dev-benchmarks", message: "No regression found.") 36 | } 37 | } 38 | } 39 | } 40 | } 41 | catch (org.jenkinsci.plugins.workflow.steps.FlowInterruptedException err) { 42 | slackSend ( 43 | channel: "#flink-dev-benchmarks", 44 | message: "Build interrupted ${env.BUILD_NUMBER} of ${env.JOB_NAME} (<${env.BUILD_URL}|Open>): $err" 45 | ) 46 | } 47 | catch (err) { 48 | slackSend (message: "Failed build ${env.BUILD_NUMBER} of ${env.JOB_NAME} (<${env.BUILD_URL}|Open>): $err", channel: "#flink-dev-benchmarks") 49 | throw err 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /save_jmh_result.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python 2 | # -*- coding: utf-8 -*- 3 | ################################################################################ 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | ################################################################################ 20 | 21 | #################################################### 22 | # Sample script that shows how to save result data # 23 | #################################################### 24 | import datetime 25 | import os 26 | import urllib 27 | import urllib2 28 | import argparse 29 | import csv 30 | import json 31 | import random 32 | 33 | # You need to enter the real URL and have the server running 34 | DEFAULT_CODESPEED_URL = 'http://localhost:8000/' 35 | 36 | current_date = datetime.datetime.today() 37 | 38 | 39 | parser = argparse.ArgumentParser(description='Upload jmh benchmark csv results') 40 | parser.add_argument('--commit', dest='commit', required=True, 41 | help='md5') 42 | parser.add_argument('--branch', dest='branch', required=True) 43 | parser.add_argument('--input', dest='input', required=False, 44 | help='input csv file') 45 | parser.add_argument('--environment', dest='environment', required=True) 46 | parser.add_argument('--dry', dest='dry', action='store_true') 47 | parser.add_argument('--codespeed', dest='codespeed', default=DEFAULT_CODESPEED_URL, 48 | help='codespeed url, default: %s' % DEFAULT_CODESPEED_URL) 49 | parser.add_argument('--project', dest='project', default="Flink") 50 | parser.add_argument('--exec', dest='executable', default="Flink") 51 | 52 | def readData(args): 53 | results = [] 54 | if args.input: 55 | path = args.input 56 | else: 57 | path = "jmh-result.csv" 58 | modificationDate = datetime.datetime.fromtimestamp(os.path.getmtime(path)) 59 | #modificationDate = datetime.date(2016, 8, int(args.commit)) 60 | 61 | with open(path) as csvFile: 62 | reader = csv.reader(csvFile, delimiter=",") 63 | lines = [line for line in reader] 64 | header = lines[0] 65 | params = sorted(filter(lambda s : s.startswith("Param"), header)) 66 | paramIndexes = map(lambda param : header.index(param), params) 67 | benchmarkIndex = header.index("Benchmark") 68 | scoreIndex = header.index("Score") 69 | modeIndex = header.index("Mode") 70 | unitIndex = header.index("Unit") 71 | errorIndex = scoreIndex + 1 72 | 73 | for line in lines[1:]: 74 | name = line[benchmarkIndex].split(".")[-1] 75 | if len(paramIndexes) > 0: 76 | for paramIndex in paramIndexes: 77 | if len(line[paramIndex]) > 0: 78 | name += "." + line[paramIndex] 79 | lessIsBetter = (line[modeIndex] == "avgt" or line[modeIndex] == "ss") 80 | # unitsTitle is used to distinguish different groups of benchmarks when getting changes 81 | # see https://github.com/tobami/codespeed/blob/263860bc298fd970c8466b3161de386582e4f801/codespeed/models.py#L444 82 | unitsTitle = "Time" 83 | if lessIsBetter: 84 | unitsTitle = "Times" 85 | results.append({ 86 | 'commitid': args.commit, 87 | 'branch': args.branch, 88 | 'project': args.project, 89 | 'executable': args.executable, 90 | 'benchmark': name, 91 | 'environment': args.environment, 92 | 'lessisbetter': lessIsBetter, 93 | 'units': line[unitIndex], 94 | 'units_title': unitsTitle, 95 | 'result_value': float(line[scoreIndex]), 96 | 97 | 'revision_date': str(modificationDate), 98 | 'result_date': str(modificationDate), 99 | 'std_dev': line[errorIndex], # Optional. Default is blank 100 | }) 101 | return results 102 | 103 | def add(data, codespeedUrl): 104 | #params = urllib.urlencode(data) 105 | response = "None" 106 | try: 107 | f = urllib2.urlopen( 108 | codespeedUrl + 'result/add/json/', urllib.urlencode(data)) 109 | except urllib2.HTTPError as e: 110 | print str(e) 111 | print e.read() 112 | return 113 | response = f.read() 114 | f.close() 115 | print "Server (%s) response: %s\n" % (codespeedUrl, response) 116 | 117 | if __name__ == "__main__": 118 | args = parser.parse_args() 119 | 120 | data = json.dumps(readData(args), indent=4, sort_keys=True) 121 | if args.dry: 122 | print data 123 | else: 124 | add({'json': data}, args.codespeed) 125 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/AsyncWaitOperatorBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.benchmark; 19 | 20 | import org.apache.flink.benchmark.functions.LongSource; 21 | import org.apache.flink.streaming.api.datastream.AsyncDataStream; 22 | import org.apache.flink.streaming.api.datastream.DataStream; 23 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.streaming.api.functions.async.ResultFuture; 26 | import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; 27 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 28 | 29 | import org.openjdk.jmh.annotations.Benchmark; 30 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 31 | import org.openjdk.jmh.annotations.Param; 32 | import org.openjdk.jmh.annotations.Setup; 33 | import org.openjdk.jmh.annotations.TearDown; 34 | import org.openjdk.jmh.runner.Runner; 35 | import org.openjdk.jmh.runner.RunnerException; 36 | import org.openjdk.jmh.runner.options.Options; 37 | import org.openjdk.jmh.runner.options.OptionsBuilder; 38 | import org.openjdk.jmh.runner.options.VerboseMode; 39 | 40 | import java.util.Collections; 41 | import java.util.concurrent.ExecutorService; 42 | import java.util.concurrent.Executors; 43 | import java.util.concurrent.TimeUnit; 44 | 45 | @OperationsPerInvocation(value = AsyncWaitOperatorBenchmark.RECORDS_PER_INVOCATION) 46 | public class AsyncWaitOperatorBenchmark extends BenchmarkBase { 47 | public static final int RECORDS_PER_INVOCATION = 1_000_000; 48 | 49 | private static final long CHECKPOINT_INTERVAL_MS = 100; 50 | 51 | private static ExecutorService executor; 52 | 53 | @Param public AsyncDataStream.OutputMode outputMode; 54 | 55 | public static void main(String[] args) throws RunnerException { 56 | Options options = 57 | new OptionsBuilder() 58 | .verbosity(VerboseMode.NORMAL) 59 | .include(".*" + AsyncWaitOperatorBenchmark.class.getCanonicalName() + ".*") 60 | .build(); 61 | 62 | new Runner(options).run(); 63 | } 64 | 65 | @Setup 66 | public void setUp() { 67 | executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); 68 | } 69 | 70 | @TearDown 71 | public void tearDown() { 72 | executor.shutdown(); 73 | } 74 | 75 | @Benchmark 76 | public void asyncWait(FlinkEnvironmentContext context) throws Exception { 77 | 78 | StreamExecutionEnvironment env = context.env; 79 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 80 | env.setParallelism(1); 81 | 82 | DataStreamSource source = env.addSource(new LongSource(RECORDS_PER_INVOCATION)); 83 | DataStream result = createAsyncOperator(source); 84 | result.addSink(new DiscardingSink<>()); 85 | 86 | env.execute(); 87 | } 88 | 89 | private DataStream createAsyncOperator(DataStreamSource source) { 90 | switch (outputMode) { 91 | case ORDERED: 92 | return AsyncDataStream.orderedWait( 93 | source, new BenchmarkAsyncFunctionExecutor(), 0, TimeUnit.MILLISECONDS); 94 | case UNORDERED: 95 | return AsyncDataStream.unorderedWait( 96 | source, new BenchmarkAsyncFunctionExecutor(), 0, TimeUnit.MILLISECONDS); 97 | default: 98 | throw new UnsupportedOperationException("Unknown mode"); 99 | } 100 | } 101 | 102 | private static class BenchmarkAsyncFunctionExecutor extends RichAsyncFunction { 103 | @Override 104 | public void asyncInvoke(Long input, ResultFuture resultFuture) { 105 | executor.execute(() -> resultFuture.complete(Collections.singleton(input * 2))); 106 | } 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/BackpressureUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.api.common.JobID; 22 | import org.apache.flink.api.common.time.Deadline; 23 | import org.apache.flink.client.deployment.StandaloneClusterId; 24 | import org.apache.flink.client.program.rest.RestClusterClient; 25 | import org.apache.flink.configuration.Configuration; 26 | import org.apache.flink.configuration.RestOptions; 27 | import org.apache.flink.runtime.jobgraph.JobVertexID; 28 | import org.apache.flink.runtime.rest.messages.EmptyRequestBody; 29 | import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; 30 | import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo; 31 | import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; 32 | import org.apache.flink.util.FlinkRuntimeException; 33 | 34 | import java.net.URI; 35 | import java.time.Duration; 36 | import java.util.List; 37 | 38 | /** Utility class for querying a backpressure status. */ 39 | public class BackpressureUtils { 40 | 41 | static void waitForBackpressure( 42 | JobID jobID, 43 | List sourceId, 44 | URI restAddress, 45 | Configuration clientConfiguration) 46 | throws Exception { 47 | RestClusterClient restClient = 48 | createClient(restAddress.getPort(), clientConfiguration); 49 | Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); 50 | boolean allBackpressured; 51 | // There seems to be a race condition in some setups, between setting up REST server 52 | // and client being able to connect. This is handled by the retrying mechanism in 53 | // the RestClusterClient, but time out takes a lot of time to trigger, so we are 54 | // doing a little bit of sleep here in an attempt to avoid waiting for that timeout. 55 | Thread.sleep(100); 56 | do { 57 | allBackpressured = 58 | sourceId.stream() 59 | .map(id -> queryBackpressure(jobID, id, restClient, restAddress)) 60 | .allMatch( 61 | level -> 62 | level 63 | == JobVertexBackPressureInfo 64 | .VertexBackPressureLevel.HIGH); 65 | } while (!allBackpressured && deadline.hasTimeLeft()); 66 | if (!allBackpressured) { 67 | throw new FlinkRuntimeException( 68 | "Could not trigger backpressure for the job in given time."); 69 | } 70 | } 71 | 72 | private static RestClusterClient createClient( 73 | int port, Configuration clientConfiguration) throws Exception { 74 | final Configuration clientConfig = new Configuration(); 75 | clientConfig.addAll(clientConfiguration); 76 | clientConfig.set(RestOptions.PORT, port); 77 | return new RestClusterClient<>(clientConfig, StandaloneClusterId.getInstance()); 78 | } 79 | 80 | private static JobVertexBackPressureInfo.VertexBackPressureLevel queryBackpressure( 81 | JobID jobID, JobVertexID vertexID, RestClusterClient restClient, URI restAddress) { 82 | try { 83 | final JobVertexMessageParameters metricsParameters = new JobVertexMessageParameters(); 84 | metricsParameters.jobPathParameter.resolve(jobID); 85 | metricsParameters.jobVertexIdPathParameter.resolve(vertexID); 86 | return ((JobVertexBackPressureInfo) 87 | restClient 88 | .sendRequest( 89 | JobVertexBackPressureHeaders.getInstance(), 90 | metricsParameters, 91 | EmptyRequestBody.getInstance()) 92 | .get()) 93 | .getBackpressureLevel(); 94 | } catch (Exception e) { 95 | throw new FlinkRuntimeException(e); 96 | } 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/BenchmarkBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.openjdk.jmh.annotations.BenchmarkMode; 22 | import org.openjdk.jmh.annotations.Fork; 23 | import org.openjdk.jmh.annotations.Measurement; 24 | import org.openjdk.jmh.annotations.OutputTimeUnit; 25 | import org.openjdk.jmh.annotations.State; 26 | import org.openjdk.jmh.annotations.Warmup; 27 | 28 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 29 | import static org.openjdk.jmh.annotations.Mode.Throughput; 30 | import static org.openjdk.jmh.annotations.Scope.Thread; 31 | 32 | @SuppressWarnings("MethodMayBeStatic") 33 | @State(Thread) 34 | @OutputTimeUnit(MILLISECONDS) 35 | @BenchmarkMode(Throughput) 36 | @Fork( 37 | value = 3, 38 | jvmArgsAppend = { 39 | "-Djava.rmi.server.hostname=127.0.0.1", 40 | "-Dcom.sun.management.jmxremote.authenticate=false", 41 | "-Dcom.sun.management.jmxremote.ssl=false", 42 | "-Dcom.sun.management.jmxremote.ssl" 43 | }) 44 | @Warmup(iterations = 10) 45 | @Measurement(iterations = 10) 46 | public class BenchmarkBase {} 47 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/BlockingPartitionBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.configuration.CoreOptions; 23 | import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.streaming.api.graph.StreamGraph; 26 | import org.apache.flink.util.FileUtils; 27 | 28 | import org.openjdk.jmh.annotations.Benchmark; 29 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 30 | import org.openjdk.jmh.runner.Runner; 31 | import org.openjdk.jmh.runner.RunnerException; 32 | import org.openjdk.jmh.runner.options.Options; 33 | import org.openjdk.jmh.runner.options.OptionsBuilder; 34 | import org.openjdk.jmh.runner.options.VerboseMode; 35 | 36 | /** JMH throughput benchmark runner. */ 37 | @OperationsPerInvocation(value = BlockingPartitionBenchmark.RECORDS_PER_INVOCATION) 38 | public class BlockingPartitionBenchmark extends BenchmarkBase { 39 | 40 | public static final int RECORDS_PER_INVOCATION = 15_000_000; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options options = 44 | new OptionsBuilder() 45 | .verbosity(VerboseMode.NORMAL) 46 | .include(".*" + BlockingPartitionBenchmark.class.getCanonicalName() + ".*") 47 | .build(); 48 | 49 | new Runner(options).run(); 50 | } 51 | 52 | @Benchmark 53 | public void uncompressedFilePartition(UncompressedFileEnvironmentContext context) 54 | throws Exception { 55 | executeBenchmark(context.env); 56 | } 57 | 58 | @Benchmark 59 | public void compressedFilePartition(CompressedFileEnvironmentContext context) throws Exception { 60 | executeBenchmark(context.env); 61 | } 62 | 63 | private void executeBenchmark(StreamExecutionEnvironment env) throws Exception { 64 | StreamGraph streamGraph = 65 | StreamGraphUtils.buildGraphForBatchJob(env, RECORDS_PER_INVOCATION); 66 | env.execute(streamGraph); 67 | } 68 | 69 | /** Setup for the benchmark(s). */ 70 | public static class BlockingPartitionEnvironmentContext extends FlinkEnvironmentContext { 71 | 72 | /** 73 | * Parallelism of 1 causes the reads/writes to be always sequential and only covers the case 74 | * of one reader. More parallelism should be more suitable for finding performance 75 | * regressions of the code. Considering that the benchmarking machine has 4 CPU cores, we 76 | * set the parallelism to 4. 77 | */ 78 | private final int parallelism = 4; 79 | 80 | @Override 81 | public void setUp() throws Exception { 82 | super.setUp(); 83 | 84 | env.setParallelism(parallelism); 85 | env.setBufferTimeout(-1); 86 | } 87 | 88 | protected Configuration createConfiguration( 89 | boolean compressionEnabled) { 90 | Configuration configuration = super.createConfiguration(); 91 | 92 | configuration.set( 93 | NettyShuffleEnvironmentOptions.SHUFFLE_COMPRESSION_CODEC, 94 | compressionEnabled ? 95 | NettyShuffleEnvironmentOptions.CompressionCodec.LZ4 96 | : NettyShuffleEnvironmentOptions.CompressionCodec.NONE); 97 | configuration.set( 98 | CoreOptions.TMP_DIRS, 99 | FileUtils.getCurrentWorkingDirectory().toAbsolutePath().toString()); 100 | return configuration; 101 | } 102 | } 103 | 104 | public static class UncompressedFileEnvironmentContext 105 | extends BlockingPartitionEnvironmentContext { 106 | @Override 107 | protected Configuration createConfiguration() { 108 | return createConfiguration(false); 109 | } 110 | } 111 | 112 | public static class CompressedFileEnvironmentContext 113 | extends BlockingPartitionEnvironmentContext { 114 | @Override 115 | protected Configuration createConfiguration() { 116 | return createConfiguration(true); 117 | } 118 | } 119 | } 120 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/BlockingPartitionRemoteChannelBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.configuration.CoreOptions; 23 | import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; 24 | import org.apache.flink.streaming.api.graph.StreamGraph; 25 | import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; 26 | import org.apache.flink.util.FileUtils; 27 | 28 | import org.openjdk.jmh.annotations.Benchmark; 29 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 30 | import org.openjdk.jmh.runner.Runner; 31 | import org.openjdk.jmh.runner.RunnerException; 32 | import org.openjdk.jmh.runner.options.Options; 33 | import org.openjdk.jmh.runner.options.OptionsBuilder; 34 | import org.openjdk.jmh.runner.options.VerboseMode; 35 | 36 | @OperationsPerInvocation(value = BlockingPartitionRemoteChannelBenchmark.RECORDS_PER_INVOCATION) 37 | public class BlockingPartitionRemoteChannelBenchmark extends RemoteBenchmarkBase { 38 | 39 | private static final int NUM_VERTICES = 2; 40 | 41 | public static void main(String[] args) throws RunnerException { 42 | Options options = 43 | new OptionsBuilder() 44 | .verbosity(VerboseMode.NORMAL) 45 | .include(BlockingPartitionRemoteChannelBenchmark.class.getCanonicalName()) 46 | .build(); 47 | 48 | new Runner(options).run(); 49 | } 50 | 51 | @Benchmark 52 | public void remoteSortPartition(RemoteSortEnvironmentContext context) throws Exception { 53 | StreamGraph streamGraph = 54 | StreamGraphUtils.buildGraphForBatchJob(context.env, RECORDS_PER_INVOCATION); 55 | context.miniCluster.executeJobBlocking( 56 | StreamingJobGraphGenerator.createJobGraph(streamGraph)); 57 | } 58 | 59 | /** Environment context for specific file based bounded blocking partition. */ 60 | public static class BlockingPartitionEnvironmentContext extends RemoteBenchmarkContext { 61 | 62 | @Override 63 | public void setUp() throws Exception { 64 | super.setUp(); 65 | 66 | env.setParallelism(PARALLELISM); 67 | env.setBufferTimeout(-1); 68 | } 69 | 70 | protected Configuration createConfiguration() { 71 | Configuration configuration = super.createConfiguration(); 72 | 73 | configuration.set( 74 | CoreOptions.TMP_DIRS, 75 | FileUtils.getCurrentWorkingDirectory().toAbsolutePath().toString()); 76 | return configuration; 77 | } 78 | 79 | @Override 80 | protected int getNumberOfVertices() { 81 | return NUM_VERTICES; 82 | } 83 | } 84 | 85 | public static class RemoteSortEnvironmentContext extends BlockingPartitionEnvironmentContext { 86 | @Override 87 | protected Configuration createConfiguration() { 88 | return super.createConfiguration(); 89 | } 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/CollectSink.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; 22 | 23 | import java.util.ArrayList; 24 | import java.util.List; 25 | 26 | /** Created by pnowojski on 7/5/17. */ 27 | public class CollectSink implements SinkFunction { 28 | public final List result = new ArrayList<>(); 29 | 30 | @Override 31 | public void invoke(T value) throws Exception { 32 | result.add(value); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/FlinkEnvironmentContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.configuration.CheckpointingOptions; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.configuration.DeploymentOptions; 24 | import org.apache.flink.configuration.RestOptions; 25 | import org.apache.flink.configuration.RestartStrategyOptions; 26 | import org.apache.flink.configuration.StateBackendOptions; 27 | import org.apache.flink.runtime.minicluster.MiniCluster; 28 | import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; 29 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 30 | import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader; 31 | 32 | import org.openjdk.jmh.annotations.Setup; 33 | import org.openjdk.jmh.annotations.State; 34 | import org.openjdk.jmh.annotations.TearDown; 35 | 36 | import java.time.Duration; 37 | 38 | import static org.apache.flink.configuration.ResourceManagerOptions.REQUIREMENTS_CHECK_DELAY; 39 | import static org.openjdk.jmh.annotations.Scope.Thread; 40 | 41 | @State(Thread) 42 | public class FlinkEnvironmentContext { 43 | 44 | public static final int NUM_NETWORK_BUFFERS = 1000; 45 | protected final int parallelism = 1; 46 | protected final boolean objectReuse = true; 47 | public StreamExecutionEnvironment env; 48 | public MiniCluster miniCluster; 49 | 50 | @Setup 51 | public void setUp() throws Exception { 52 | if (miniCluster != null) { 53 | throw new RuntimeException("setUp was called multiple times!"); 54 | } 55 | final Configuration clusterConfig = createConfiguration(); 56 | clusterConfig.set(RestartStrategyOptions.RESTART_STRATEGY, "none"); 57 | clusterConfig.set(StateBackendOptions.STATE_BACKEND, "hashmap"); 58 | miniCluster = 59 | new MiniCluster( 60 | new MiniClusterConfiguration.Builder() 61 | .setNumSlotsPerTaskManager(getNumberOfSlotsPerTaskManager()) 62 | .setNumTaskManagers(getNumberOfTaskManagers()) 63 | .setConfiguration(clusterConfig) 64 | .build()); 65 | 66 | try { 67 | miniCluster.start(); 68 | } catch (Exception e) { 69 | throw new RuntimeException(e); 70 | } 71 | // set up the execution environment 72 | env = 73 | new StreamExecutionEnvironment( 74 | new MiniClusterPipelineExecutorServiceLoader(miniCluster), 75 | clusterConfig, 76 | null); 77 | 78 | env.setParallelism(parallelism); 79 | if (objectReuse) { 80 | env.getConfig().enableObjectReuse(); 81 | } 82 | } 83 | 84 | @TearDown 85 | public void tearDown() throws Exception { 86 | miniCluster.close(); 87 | miniCluster = null; 88 | } 89 | 90 | protected int getNumberOfTaskManagers() { 91 | return 1; 92 | } 93 | 94 | protected int getNumberOfSlotsPerTaskManager() { 95 | return 4; 96 | } 97 | 98 | public void execute() throws Exception { 99 | env.execute(); 100 | } 101 | 102 | protected Configuration createConfiguration() { 103 | final Configuration configuration = new Configuration(); 104 | configuration.set(RestOptions.BIND_PORT, "0"); 105 | // no equivalent config available. 106 | //configuration.setInteger( 107 | // NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, NUM_NETWORK_BUFFERS); 108 | configuration.set(DeploymentOptions.TARGET, MiniClusterPipelineExecutorServiceLoader.NAME); 109 | configuration.set(DeploymentOptions.ATTACHED, true); 110 | // It doesn't make sense to wait for the final checkpoint in benchmarks since it only prolongs 111 | // the test but doesn't give any advantages. 112 | configuration.set(CheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, false); 113 | // TODO: remove this line after FLINK-28243 will be done 114 | configuration.set(REQUIREMENTS_CHECK_DELAY, Duration.ZERO); 115 | return configuration; 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/HighAvailabilityServiceBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.curator.test.TestingServer; 22 | import org.apache.flink.api.common.JobID; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.configuration.HighAvailabilityOptions; 25 | import org.apache.flink.runtime.jobgraph.JobGraph; 26 | import org.apache.flink.runtime.jobgraph.JobVertex; 27 | import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; 28 | import org.apache.flink.runtime.testtasks.NoOpInvokable; 29 | import org.apache.flink.util.FileUtils; 30 | import org.openjdk.jmh.annotations.Benchmark; 31 | import org.openjdk.jmh.annotations.OutputTimeUnit; 32 | import org.openjdk.jmh.annotations.Param; 33 | import org.openjdk.jmh.annotations.State; 34 | import org.openjdk.jmh.runner.Runner; 35 | import org.openjdk.jmh.runner.RunnerException; 36 | import org.openjdk.jmh.runner.options.Options; 37 | import org.openjdk.jmh.runner.options.OptionsBuilder; 38 | import org.openjdk.jmh.runner.options.VerboseMode; 39 | 40 | import java.io.File; 41 | import java.io.IOException; 42 | import java.nio.file.Files; 43 | import java.util.UUID; 44 | 45 | import static java.util.concurrent.TimeUnit.SECONDS; 46 | import static org.openjdk.jmh.annotations.Scope.Thread; 47 | 48 | /** 49 | * When Flink session cluster supports shorted-lived jobs(OLAP), HA service needs to be enabled. 50 | * However, after HA service is enabled, Flink needs to interact with HA service frequently when 51 | * processing short-lived jobs, resulting in increased latency and decreased QPS. This benchmark 52 | * mainly statist the QPS of Flink Session cluster with and without HA service for shorted-lived 53 | * jobs and HA service optimization. 54 | */ 55 | @OutputTimeUnit(SECONDS) 56 | public class HighAvailabilityServiceBenchmark extends BenchmarkBase { 57 | public static void main(String[] args) throws RunnerException { 58 | Options options = 59 | new OptionsBuilder() 60 | .verbosity(VerboseMode.NORMAL) 61 | .include(".*" + HighAvailabilityServiceBenchmark.class.getCanonicalName() + ".*") 62 | .build(); 63 | 64 | new Runner(options).run(); 65 | } 66 | 67 | @Benchmark 68 | public void submitJobThroughput(HighAvailabilityContext context) throws Exception { 69 | context.miniCluster.executeJobBlocking(buildNoOpJob()); 70 | } 71 | 72 | private static JobGraph buildNoOpJob() { 73 | JobGraph jobGraph = new JobGraph(JobID.generate(), UUID.randomUUID().toString()); 74 | jobGraph.addVertex(createNoOpVertex()); 75 | return jobGraph; 76 | } 77 | 78 | private static JobVertex createNoOpVertex() { 79 | JobVertex vertex = new JobVertex("v"); 80 | vertex.setInvokableClass(NoOpInvokable.class); 81 | vertex.setParallelism(1); 82 | vertex.setMaxParallelism(1); 83 | return vertex; 84 | } 85 | 86 | @State(Thread) 87 | public static class HighAvailabilityContext extends FlinkEnvironmentContext { 88 | private TestingServer testingServer; 89 | public final File haDir; 90 | 91 | @Param({"ZOOKEEPER", "NONE"}) 92 | public HighAvailabilityMode highAvailabilityMode; 93 | 94 | public HighAvailabilityContext() { 95 | try { 96 | haDir = Files.createTempDirectory("bench-ha-").toFile(); 97 | } catch (IOException e) { 98 | throw new RuntimeException(e); 99 | } 100 | } 101 | 102 | @Override 103 | public void setUp() throws Exception { 104 | if (isZookeeperHighAvailability()) { 105 | testingServer = new TestingServer(); 106 | testingServer.start(); 107 | } 108 | 109 | // The method `super.setUp()` will call `createConfiguration()` to get Configuration and 110 | // create a `MiniCluster`. We need to start TestingServer before `createConfiguration()`, 111 | // then we can add zookeeper quorum in the configuration. So we can only start 112 | // `TestingServer` before `super.setUp()`. 113 | super.setUp(); 114 | } 115 | 116 | private boolean isZookeeperHighAvailability() { 117 | return highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER; 118 | } 119 | 120 | @Override 121 | protected Configuration createConfiguration() { 122 | Configuration configuration = super.createConfiguration(); 123 | configuration.set(HighAvailabilityOptions.HA_MODE, highAvailabilityMode.name()); 124 | configuration.set(HighAvailabilityOptions.HA_STORAGE_PATH, haDir.toURI().toString()); 125 | if (isZookeeperHighAvailability()) { 126 | configuration.set(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString()); 127 | } 128 | return configuration; 129 | } 130 | 131 | @Override 132 | public void tearDown() throws Exception { 133 | super.tearDown(); 134 | if (isZookeeperHighAvailability()) { 135 | testingServer.stop(); 136 | testingServer.close(); 137 | } 138 | FileUtils.deleteDirectory(haDir); 139 | } 140 | } 141 | } 142 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/InputBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.LongSourceType; 22 | import org.apache.flink.benchmark.functions.MultiplyByTwo; 23 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 26 | 27 | import org.openjdk.jmh.annotations.Benchmark; 28 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.runner.Runner; 31 | import org.openjdk.jmh.runner.RunnerException; 32 | import org.openjdk.jmh.runner.options.Options; 33 | import org.openjdk.jmh.runner.options.OptionsBuilder; 34 | import org.openjdk.jmh.runner.options.VerboseMode; 35 | 36 | @OperationsPerInvocation(value = InputBenchmark.RECORDS_PER_INVOCATION) 37 | public class InputBenchmark extends BenchmarkBase { 38 | public static final int RECORDS_PER_INVOCATION = 15_000_000; 39 | private static final long CHECKPOINT_INTERVAL_MS = 100; 40 | 41 | @Param({"LEGACY", "F27_UNBOUNDED"}) 42 | public LongSourceType sourceType; 43 | 44 | public static void main(String[] args) throws RunnerException { 45 | Options options = 46 | new OptionsBuilder() 47 | .verbosity(VerboseMode.NORMAL) 48 | .include(".*" + InputBenchmark.class.getCanonicalName() + ".*") 49 | .build(); 50 | 51 | new Runner(options).run(); 52 | } 53 | 54 | @Benchmark 55 | public void mapSink(FlinkEnvironmentContext context) throws Exception { 56 | 57 | StreamExecutionEnvironment env = context.env; 58 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 59 | env.setParallelism(1); 60 | 61 | DataStreamSource source = sourceType.source(env, RECORDS_PER_INVOCATION); 62 | source.map(new MultiplyByTwo()).addSink(new DiscardingSink<>()); 63 | 64 | env.execute(); 65 | } 66 | 67 | @Benchmark 68 | public void mapRebalanceMapSink(FlinkEnvironmentContext context) throws Exception { 69 | 70 | StreamExecutionEnvironment env = context.env; 71 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 72 | env.setParallelism(1); 73 | 74 | DataStreamSource source = sourceType.source(env, RECORDS_PER_INVOCATION); 75 | source.map(new MultiplyByTwo()) 76 | .rebalance() 77 | .map((Long in) -> in) 78 | .addSink(new DiscardingSink<>()); 79 | 80 | env.execute(); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/KeyByBenchmarks.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.api.java.tuple.Tuple2; 22 | import org.apache.flink.benchmark.functions.BaseSourceWithKeyRange; 23 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 26 | 27 | import org.apache.flink.streaming.util.keys.KeySelectorUtil; 28 | import org.openjdk.jmh.annotations.Benchmark; 29 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 30 | import org.openjdk.jmh.runner.Runner; 31 | import org.openjdk.jmh.runner.RunnerException; 32 | import org.openjdk.jmh.runner.options.Options; 33 | import org.openjdk.jmh.runner.options.OptionsBuilder; 34 | import org.openjdk.jmh.runner.options.VerboseMode; 35 | 36 | /** Benchmark for keyBy() on tuples and arrays. */ 37 | public class KeyByBenchmarks extends BenchmarkBase { 38 | 39 | private static final int TUPLE_RECORDS_PER_INVOCATION = 15_000_000; 40 | private static final int ARRAY_RECORDS_PER_INVOCATION = 7_000_000; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options options = 44 | new OptionsBuilder() 45 | .verbosity(VerboseMode.NORMAL) 46 | .include(".*" + KeyByBenchmarks.class.getCanonicalName() + ".*") 47 | .build(); 48 | 49 | new Runner(options).run(); 50 | } 51 | 52 | @Benchmark 53 | @OperationsPerInvocation(value = KeyByBenchmarks.TUPLE_RECORDS_PER_INVOCATION) 54 | public void tupleKeyBy(FlinkEnvironmentContext context) throws Exception { 55 | StreamExecutionEnvironment env = context.env; 56 | env.setParallelism(4); 57 | 58 | env.addSource(new IncreasingTupleSource(TUPLE_RECORDS_PER_INVOCATION, 10)) 59 | .keyBy(e -> e.f0) 60 | .addSink(new DiscardingSink<>()); 61 | 62 | env.execute(); 63 | } 64 | 65 | @Benchmark 66 | @OperationsPerInvocation(value = KeyByBenchmarks.ARRAY_RECORDS_PER_INVOCATION) 67 | public void arrayKeyBy(FlinkEnvironmentContext context) throws Exception { 68 | StreamExecutionEnvironment env = context.env; 69 | env.setParallelism(4); 70 | 71 | DataStreamSource source = env.addSource(new IncreasingArraySource(ARRAY_RECORDS_PER_INVOCATION, 10)); 72 | source.keyBy(KeySelectorUtil.getSelectorForArray(new int[]{0}, source.getType())) 73 | .addSink(new DiscardingSink<>()); 74 | 75 | env.execute(); 76 | } 77 | 78 | private static class IncreasingTupleSource 79 | extends BaseSourceWithKeyRange> { 80 | private static final long serialVersionUID = 2941333602938145526L; 81 | 82 | IncreasingTupleSource(int numEvents, int numKeys) { 83 | super(numEvents, numKeys); 84 | } 85 | 86 | @Override 87 | protected Tuple2 getElement(int keyId) { 88 | return new Tuple2<>(keyId, 1); 89 | } 90 | } 91 | 92 | private static class IncreasingArraySource extends BaseSourceWithKeyRange { 93 | private static final long serialVersionUID = -7883758559005221998L; 94 | 95 | IncreasingArraySource(int numEvents, int numKeys) { 96 | super(numEvents, numKeys); 97 | } 98 | 99 | @Override 100 | protected int[] getElement(int keyId) { 101 | return new int[] {keyId, 1}; 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/MemoryStateBackendBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.IntLongApplications; 22 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; 23 | 24 | import org.openjdk.jmh.annotations.Benchmark; 25 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 26 | import org.openjdk.jmh.annotations.Param; 27 | import org.openjdk.jmh.annotations.State; 28 | import org.openjdk.jmh.runner.Runner; 29 | import org.openjdk.jmh.runner.RunnerException; 30 | import org.openjdk.jmh.runner.options.Options; 31 | import org.openjdk.jmh.runner.options.OptionsBuilder; 32 | import org.openjdk.jmh.runner.options.VerboseMode; 33 | 34 | import java.time.Duration; 35 | 36 | import static org.openjdk.jmh.annotations.Scope.Thread; 37 | 38 | @OperationsPerInvocation(value = MemoryStateBackendBenchmark.RECORDS_PER_INVOCATION) 39 | public class MemoryStateBackendBenchmark extends StateBackendBenchmarkBase { 40 | public static final int RECORDS_PER_INVOCATION = 7_000_000; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options options = 44 | new OptionsBuilder() 45 | .verbosity(VerboseMode.NORMAL) 46 | .include(".*" + MemoryStateBackendBenchmark.class.getCanonicalName() + ".*") 47 | .build(); 48 | 49 | new Runner(options).run(); 50 | } 51 | 52 | @Benchmark 53 | public void stateBackends(MemoryStateBackendContext context) throws Exception { 54 | IntLongApplications.reduceWithWindow( 55 | context.source, TumblingEventTimeWindows.of(Duration.ofSeconds(10_000))); 56 | context.execute(); 57 | } 58 | 59 | @State(Thread) 60 | public static class MemoryStateBackendContext extends StateBackendContext { 61 | @Param({"MEMORY", "FS"}) 62 | public StateBackend stateBackend = StateBackend.MEMORY; 63 | 64 | @Override 65 | public void setUp() throws Exception { 66 | super.setUp(stateBackend, RECORDS_PER_INVOCATION); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/ProcessingTimerBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.benchmark; 19 | 20 | import org.apache.flink.api.common.functions.OpenContext; 21 | import org.apache.flink.core.testutils.OneShotLatch; 22 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 23 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction; 24 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 25 | import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction; 26 | import org.apache.flink.util.Collector; 27 | import org.openjdk.jmh.annotations.Benchmark; 28 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 29 | import org.openjdk.jmh.runner.Runner; 30 | import org.openjdk.jmh.runner.RunnerException; 31 | import org.openjdk.jmh.runner.options.Options; 32 | import org.openjdk.jmh.runner.options.OptionsBuilder; 33 | import org.openjdk.jmh.runner.options.VerboseMode; 34 | 35 | import java.util.Random; 36 | 37 | @OperationsPerInvocation(value = ProcessingTimerBenchmark.PROCESSING_TIMERS_PER_INVOCATION) 38 | public class ProcessingTimerBenchmark extends BenchmarkBase { 39 | 40 | public static final int PROCESSING_TIMERS_PER_INVOCATION = 150_000; 41 | 42 | private static final int PARALLELISM = 1; 43 | 44 | private static OneShotLatch LATCH = new OneShotLatch(); 45 | 46 | public static void main(String[] args) throws RunnerException { 47 | Options options = 48 | new OptionsBuilder() 49 | .verbosity(VerboseMode.NORMAL) 50 | .include(".*" + ProcessingTimerBenchmark.class.getCanonicalName() + ".*") 51 | .build(); 52 | 53 | new Runner(options).run(); 54 | } 55 | 56 | @Benchmark 57 | public void fireProcessingTimers(FlinkEnvironmentContext context) throws Exception { 58 | LATCH.reset(); 59 | StreamExecutionEnvironment env = context.env; 60 | env.setParallelism(PARALLELISM); 61 | 62 | env.addSource(new SingleRecordSource()) 63 | .keyBy(String::hashCode) 64 | .process(new ProcessingTimerKeyedProcessFunction(PROCESSING_TIMERS_PER_INVOCATION)) 65 | .addSink(new DiscardingSink<>()); 66 | 67 | env.execute(); 68 | } 69 | 70 | private static class SingleRecordSource extends RichParallelSourceFunction { 71 | 72 | private Random random; 73 | 74 | public SingleRecordSource() {} 75 | 76 | @Override 77 | public void open(OpenContext openContext) throws Exception { 78 | this.random = new Random(); 79 | } 80 | 81 | @Override 82 | public void run(SourceContext sourceContext) throws Exception { 83 | synchronized (sourceContext.getCheckpointLock()) { 84 | sourceContext.collect(String.valueOf(random.nextLong())); 85 | } 86 | 87 | LATCH.await(); 88 | } 89 | 90 | @Override 91 | public void cancel() {} 92 | } 93 | 94 | private static class ProcessingTimerKeyedProcessFunction 95 | extends KeyedProcessFunction { 96 | 97 | private final long timersPerRecord; 98 | private long firedTimesCount; 99 | 100 | public ProcessingTimerKeyedProcessFunction(long timersPerRecord) { 101 | this.timersPerRecord = timersPerRecord; 102 | } 103 | 104 | @Override 105 | public void open(OpenContext context) throws Exception { 106 | this.firedTimesCount = 0; 107 | } 108 | 109 | @Override 110 | public void processElement(String s, Context context, Collector collector) 111 | throws Exception { 112 | final long currTimestamp = System.currentTimeMillis(); 113 | for (int i = 0; i < timersPerRecord; i++) { 114 | context.timerService().registerProcessingTimeTimer(currTimestamp - timersPerRecord + i); 115 | } 116 | } 117 | 118 | @Override 119 | public void onTimer(long timestamp, OnTimerContext ctx, Collector out) 120 | throws Exception { 121 | if (++firedTimesCount == timersPerRecord) { 122 | LATCH.trigger(); 123 | } 124 | } 125 | } 126 | } 127 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/RemoteBenchmarkBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | /** Benchmark base for setting up the cluster to perform remote network shuffle. */ 22 | public abstract class RemoteBenchmarkBase extends BenchmarkBase { 23 | 24 | protected static final int PARALLELISM = 4; 25 | protected static final int RECORDS_PER_SUBTASK = 10_000_000; 26 | protected static final int RECORDS_PER_INVOCATION = RECORDS_PER_SUBTASK * PARALLELISM; 27 | 28 | public abstract static class RemoteBenchmarkContext extends FlinkEnvironmentContext { 29 | @Override 30 | protected int getNumberOfTaskManagers() { 31 | return getNumberOfVertices() * PARALLELISM; 32 | } 33 | 34 | @Override 35 | protected int getNumberOfSlotsPerTaskManager() { 36 | return 1; 37 | } 38 | 39 | /** @return the number of vertices the respective job graph contains. */ 40 | abstract int getNumberOfVertices(); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/RemoteChannelThroughputBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.api.common.functions.MapFunction; 22 | import org.apache.flink.benchmark.functions.LongSource; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.configuration.TaskManagerOptions; 25 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 26 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 27 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 28 | import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; 29 | 30 | import org.openjdk.jmh.annotations.Benchmark; 31 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 32 | import org.openjdk.jmh.annotations.Param; 33 | import org.openjdk.jmh.annotations.Scope; 34 | import org.openjdk.jmh.annotations.State; 35 | import org.openjdk.jmh.runner.Runner; 36 | import org.openjdk.jmh.runner.RunnerException; 37 | import org.openjdk.jmh.runner.options.Options; 38 | import org.openjdk.jmh.runner.options.OptionsBuilder; 39 | import org.openjdk.jmh.runner.options.VerboseMode; 40 | 41 | @OperationsPerInvocation(value = RemoteChannelThroughputBenchmark.RECORDS_PER_INVOCATION) 42 | public class RemoteChannelThroughputBenchmark extends RemoteBenchmarkBase { 43 | private static final String ALIGNED = "ALIGNED"; 44 | private static final String DEBLOAT = "DEBLOAT"; 45 | private static final String UNALIGNED = "UNALIGNED"; 46 | 47 | private static final int NUM_VERTICES = 3; 48 | private static final long CHECKPOINT_INTERVAL_MS = 100; 49 | 50 | public static void main(String[] args) throws RunnerException { 51 | Options options = 52 | new OptionsBuilder() 53 | .verbosity(VerboseMode.NORMAL) 54 | .include(RemoteChannelThroughputBenchmark.class.getCanonicalName()) 55 | .build(); 56 | 57 | new Runner(options).run(); 58 | } 59 | 60 | @Benchmark 61 | public void remoteRebalance(RemoteChannelThroughputBenchmarkContext context) throws Exception { 62 | StreamExecutionEnvironment env = context.env; 63 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 64 | env.setParallelism(PARALLELISM); 65 | env.getCheckpointConfig().enableUnalignedCheckpoints(context.mode.equals(UNALIGNED)); 66 | 67 | DataStreamSource source = env.addSource(new LongSource(RECORDS_PER_SUBTASK)); 68 | source.slotSharingGroup("source") 69 | .rebalance() 70 | .map((MapFunction) value -> value) 71 | .slotSharingGroup("map") 72 | .rebalance() 73 | .addSink(new DiscardingSink<>()) 74 | .slotSharingGroup("sink"); 75 | 76 | context.miniCluster.executeJobBlocking( 77 | StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph())); 78 | } 79 | 80 | @State(Scope.Thread) 81 | public static class RemoteChannelThroughputBenchmarkContext extends RemoteBenchmarkContext { 82 | @Param({ALIGNED, UNALIGNED, DEBLOAT}) 83 | public String mode = ALIGNED; 84 | 85 | @Override 86 | protected Configuration createConfiguration() { 87 | Configuration configuration = super.createConfiguration(); 88 | if (mode.equals(DEBLOAT)) { 89 | configuration.set(TaskManagerOptions.BUFFER_DEBLOAT_ENABLED, true); 90 | } 91 | return configuration; 92 | } 93 | 94 | @Override 95 | protected int getNumberOfVertices() { 96 | return NUM_VERTICES; 97 | } 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/RocksStateBackendBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.IntLongApplications; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.configuration.MemorySize; 24 | import org.apache.flink.contrib.streaming.state.RocksDBOptions; 25 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; 26 | 27 | import org.openjdk.jmh.annotations.Benchmark; 28 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.State; 31 | import org.openjdk.jmh.runner.Runner; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | import org.openjdk.jmh.runner.options.Options; 34 | import org.openjdk.jmh.runner.options.OptionsBuilder; 35 | import org.openjdk.jmh.runner.options.VerboseMode; 36 | 37 | import java.time.Duration; 38 | 39 | import static org.openjdk.jmh.annotations.Scope.Thread; 40 | 41 | @OperationsPerInvocation(value = RocksStateBackendBenchmark.RECORDS_PER_INVOCATION) 42 | public class RocksStateBackendBenchmark extends StateBackendBenchmarkBase { 43 | public static final int RECORDS_PER_INVOCATION = 2_000_000; 44 | 45 | public static void main(String[] args) throws RunnerException { 46 | Options options = 47 | new OptionsBuilder() 48 | .verbosity(VerboseMode.NORMAL) 49 | .include(".*" + RocksStateBackendBenchmark.class.getCanonicalName() + ".*") 50 | .build(); 51 | 52 | new Runner(options).run(); 53 | } 54 | 55 | @Benchmark 56 | public void stateBackends(RocksStateBackendContext context) throws Exception { 57 | IntLongApplications.reduceWithWindow( 58 | context.source, TumblingEventTimeWindows.of(Duration.ofSeconds(10_000))); 59 | context.execute(); 60 | } 61 | 62 | @State(Thread) 63 | public static class RocksStateBackendContext extends StateBackendContext { 64 | @Param({"ROCKS", "ROCKS_INC"}) 65 | public StateBackend stateBackend = StateBackend.MEMORY; 66 | 67 | @Override 68 | public void setUp() throws Exception { 69 | super.setUp(stateBackend, RECORDS_PER_INVOCATION); 70 | } 71 | 72 | @Override 73 | protected Configuration createConfiguration() { 74 | Configuration configuration = super.createConfiguration(); 75 | // explicit set the managed memory as 322122552 bytes, which is the default managed 76 | // memory of 1GB TM with 1 slot. 77 | configuration.set( 78 | RocksDBOptions.FIX_PER_SLOT_MEMORY_SIZE, MemorySize.parse("322122552b")); 79 | return configuration; 80 | } 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/StateBackendBenchmarkBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.IntegerLongSource; 22 | import org.apache.flink.configuration.CheckpointingOptions; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.configuration.ReadableConfig; 25 | import org.apache.flink.configuration.StateBackendOptions; 26 | import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; 27 | import org.apache.flink.runtime.state.AbstractStateBackend; 28 | import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; 29 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 30 | import org.apache.flink.util.FileUtils; 31 | 32 | import java.io.File; 33 | import java.io.IOException; 34 | import java.nio.file.Files; 35 | 36 | public class StateBackendBenchmarkBase extends BenchmarkBase { 37 | public enum StateBackend { 38 | MEMORY, 39 | FS, 40 | FS_ASYNC, 41 | ROCKS, 42 | ROCKS_INC 43 | } 44 | 45 | public static class StateBackendContext extends FlinkEnvironmentContext { 46 | 47 | public final File checkpointDir; 48 | 49 | public final int numberOfElements = 1000; 50 | 51 | public DataStreamSource source; 52 | 53 | public StateBackendContext() { 54 | try { 55 | checkpointDir = Files.createTempDirectory("bench-").toFile(); 56 | } catch (IOException e) { 57 | throw new RuntimeException(e); 58 | } 59 | } 60 | 61 | public void setUp(StateBackend stateBackend, long recordsPerInvocation) throws IOException { 62 | try { 63 | super.setUp(); 64 | } catch (Exception e) { 65 | e.printStackTrace(); 66 | } 67 | 68 | 69 | Configuration configuration = Configuration.fromMap(env.getConfiguration().toMap()); 70 | String checkpointDataUri = "file://" + checkpointDir.getAbsolutePath(); 71 | switch (stateBackend) { 72 | case MEMORY: 73 | configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap"); 74 | configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "jobmanager"); 75 | break; 76 | case FS: 77 | configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap"); 78 | configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); 79 | break; 80 | case ROCKS: 81 | configuration.set(StateBackendOptions.STATE_BACKEND, "rocksdb"); 82 | configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); 83 | configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, false); 84 | break; 85 | case ROCKS_INC: 86 | configuration.set(StateBackendOptions.STATE_BACKEND, "rocksdb"); 87 | configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); 88 | configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true); 89 | break; 90 | default: 91 | throw new UnsupportedOperationException( 92 | "Unknown state backend: " + stateBackend); 93 | } 94 | 95 | // default character 96 | //env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 97 | source = env.addSource(new IntegerLongSource(numberOfElements, recordsPerInvocation)); 98 | } 99 | 100 | @Override 101 | public void tearDown() throws Exception { 102 | super.tearDown(); 103 | FileUtils.deleteDirectory(checkpointDir); 104 | } 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/StreamGraphUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.LongSource; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.configuration.PipelineOptions; 24 | import org.apache.flink.runtime.jobgraph.JobType; 25 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 26 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 27 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 28 | import org.apache.flink.streaming.api.graph.GlobalStreamExchangeMode; 29 | import org.apache.flink.streaming.api.graph.StreamGraph; 30 | 31 | /** Utilities for building respective graph for performing in benchmark. */ 32 | public class StreamGraphUtils { 33 | 34 | public static StreamGraph buildGraphForBatchJob( 35 | StreamExecutionEnvironment env, int numRecords) { 36 | DataStreamSource source = env.addSource(new LongSource(numRecords)); 37 | source.addSink(new DiscardingSink<>()); 38 | 39 | Configuration config = new Configuration(); 40 | config.set(PipelineOptions.OPERATOR_CHAINING, false); 41 | env.configure(config); 42 | 43 | StreamGraph streamGraph = env.getStreamGraph(); 44 | streamGraph.setGlobalStreamExchangeMode(GlobalStreamExchangeMode.ALL_EDGES_BLOCKING); 45 | streamGraph.setJobType(JobType.BATCH); 46 | 47 | return streamGraph; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/TwoInputBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.benchmark.functions.LongSource; 23 | import org.apache.flink.benchmark.functions.QueuingLongSource; 24 | import org.apache.flink.benchmark.operators.MultiplyByTwoCoStreamMap; 25 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 26 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 27 | import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink; 28 | 29 | import org.openjdk.jmh.annotations.Benchmark; 30 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 31 | import org.openjdk.jmh.runner.Runner; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | import org.openjdk.jmh.runner.options.Options; 34 | import org.openjdk.jmh.runner.options.OptionsBuilder; 35 | import org.openjdk.jmh.runner.options.VerboseMode; 36 | 37 | public class TwoInputBenchmark extends BenchmarkBase { 38 | 39 | public static final int RECORDS_PER_INVOCATION = 25_000_000; 40 | public static final int ONE_IDLE_RECORDS_PER_INVOCATION = 15_000_000; 41 | public static final long CHECKPOINT_INTERVAL_MS = 100; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | Options options = 45 | new OptionsBuilder() 46 | .verbosity(VerboseMode.NORMAL) 47 | .include(".*" + TwoInputBenchmark.class.getCanonicalName() + ".*") 48 | .build(); 49 | 50 | new Runner(options).run(); 51 | } 52 | 53 | @Benchmark 54 | @OperationsPerInvocation(value = TwoInputBenchmark.RECORDS_PER_INVOCATION) 55 | public void twoInputMapSink(FlinkEnvironmentContext context) throws Exception { 56 | 57 | StreamExecutionEnvironment env = context.env; 58 | 59 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 60 | env.setParallelism(1); 61 | 62 | // Setting buffer timeout to 1 is an attempt to improve twoInputMapSink benchmark stability. 63 | // Without 1ms buffer timeout, some JVM forks are much slower then others, making results 64 | // unstable and unreliable. 65 | env.setBufferTimeout(1); 66 | 67 | long numRecordsPerInput = RECORDS_PER_INVOCATION / 2; 68 | DataStreamSource source1 = env.addSource(new LongSource(numRecordsPerInput)); 69 | DataStreamSource source2 = env.addSource(new LongSource(numRecordsPerInput)); 70 | 71 | source1.connect(source2) 72 | .transform( 73 | "custom operator", 74 | TypeInformation.of(Long.class), 75 | new MultiplyByTwoCoStreamMap()) 76 | .addSink(new DiscardingSink<>()); 77 | 78 | env.execute(); 79 | } 80 | 81 | @Benchmark 82 | @OperationsPerInvocation(value = TwoInputBenchmark.ONE_IDLE_RECORDS_PER_INVOCATION) 83 | public void twoInputOneIdleMapSink(FlinkEnvironmentContext context) throws Exception { 84 | 85 | StreamExecutionEnvironment env = context.env; 86 | env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); 87 | env.setParallelism(1); 88 | 89 | QueuingLongSource.reset(); 90 | DataStreamSource source1 = 91 | env.addSource(new QueuingLongSource(1, ONE_IDLE_RECORDS_PER_INVOCATION - 1)); 92 | DataStreamSource source2 = env.addSource(new QueuingLongSource(2, 1)); 93 | 94 | source1.connect(source2) 95 | .transform( 96 | "custom operator", 97 | TypeInformation.of(Long.class), 98 | new MultiplyByTwoCoStreamMap()) 99 | .addSink(new DiscardingSink<>()); 100 | 101 | env.execute(); 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/WatermarkAggregationBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | 20 | package org.apache.flink.benchmark; 21 | 22 | import org.apache.flink.runtime.source.coordinator.SourceCoordinatorAlignmentBenchmark; 23 | 24 | import org.openjdk.jmh.annotations.Benchmark; 25 | import org.openjdk.jmh.annotations.BenchmarkMode; 26 | import org.openjdk.jmh.annotations.Level; 27 | import org.openjdk.jmh.annotations.Mode; 28 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 29 | import org.openjdk.jmh.annotations.Setup; 30 | import org.openjdk.jmh.annotations.TearDown; 31 | import org.openjdk.jmh.runner.Runner; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | import org.openjdk.jmh.runner.options.Options; 34 | import org.openjdk.jmh.runner.options.OptionsBuilder; 35 | import org.openjdk.jmh.runner.options.VerboseMode; 36 | 37 | /** The watermark aggregation benchmark for source coordinator when enabling the watermark alignment. */ 38 | public class WatermarkAggregationBenchmark extends BenchmarkBase { 39 | 40 | private static final int NUM_SUBTASKS = 5000; 41 | 42 | private static final int ROUND_PER_INVOCATION = 10; 43 | 44 | private SourceCoordinatorAlignmentBenchmark benchmark; 45 | 46 | public static void main(String[] args) throws RunnerException { 47 | Options options = 48 | new OptionsBuilder() 49 | .verbosity(VerboseMode.NORMAL) 50 | .include(".*" + WatermarkAggregationBenchmark.class.getCanonicalName() + ".*") 51 | .build(); 52 | 53 | new Runner(options).run(); 54 | } 55 | 56 | @Setup(Level.Trial) 57 | public void setup() throws Exception { 58 | benchmark = new SourceCoordinatorAlignmentBenchmark(); 59 | benchmark.setup(NUM_SUBTASKS); 60 | } 61 | 62 | @Benchmark 63 | @OperationsPerInvocation(NUM_SUBTASKS * ROUND_PER_INVOCATION) 64 | public void aggregateWatermark() { 65 | for (int round = 0; round < ROUND_PER_INVOCATION; round++) { 66 | benchmark.sendReportedWatermarkToAllSubtasks(); 67 | } 68 | } 69 | 70 | @TearDown(Level.Trial) 71 | public void teardown() throws Exception { 72 | benchmark.teardown(); 73 | } 74 | 75 | } 76 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/WindowBenchmarks.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.benchmark.functions.IntLongApplications; 22 | import org.apache.flink.benchmark.functions.IntegerLongSource; 23 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 24 | import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; 25 | import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; 26 | import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; 27 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; 28 | 29 | import org.openjdk.jmh.annotations.Benchmark; 30 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 31 | import org.openjdk.jmh.runner.Runner; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | import org.openjdk.jmh.runner.options.Options; 34 | import org.openjdk.jmh.runner.options.OptionsBuilder; 35 | import org.openjdk.jmh.runner.options.VerboseMode; 36 | 37 | import java.time.Duration; 38 | 39 | @OperationsPerInvocation(value = WindowBenchmarks.RECORDS_PER_INVOCATION) 40 | public class WindowBenchmarks extends BenchmarkBase { 41 | 42 | public static final int RECORDS_PER_INVOCATION = 7_000_000; 43 | 44 | public static void main(String[] args) throws RunnerException { 45 | Options options = 46 | new OptionsBuilder() 47 | .verbosity(VerboseMode.NORMAL) 48 | .include(".*" + WindowBenchmarks.class.getCanonicalName() + ".*") 49 | .build(); 50 | 51 | new Runner(options).run(); 52 | } 53 | 54 | @Benchmark 55 | public void globalWindow(TimeWindowContext context) throws Exception { 56 | IntLongApplications.reduceWithWindow(context.source, GlobalWindows.create()); 57 | context.execute(); 58 | } 59 | 60 | @Benchmark 61 | public void tumblingWindow(TimeWindowContext context) throws Exception { 62 | IntLongApplications.reduceWithWindow( 63 | context.source, TumblingEventTimeWindows.of(Duration.ofSeconds(10_000))); 64 | context.execute(); 65 | } 66 | 67 | @Benchmark 68 | public void slidingWindow(TimeWindowContext context) throws Exception { 69 | IntLongApplications.reduceWithWindow( 70 | context.source, 71 | SlidingEventTimeWindows.of(Duration.ofSeconds(10_000), Duration.ofSeconds(1000))); 72 | context.execute(); 73 | } 74 | 75 | @Benchmark 76 | public void sessionWindow(TimeWindowContext context) throws Exception { 77 | IntLongApplications.reduceWithWindow( 78 | context.source, EventTimeSessionWindows.withGap(Duration.ofSeconds(500))); 79 | context.execute(); 80 | } 81 | 82 | public static class TimeWindowContext extends FlinkEnvironmentContext { 83 | public final int numberOfElements = 1000; 84 | 85 | public DataStreamSource source; 86 | 87 | @Override 88 | public void setUp() throws Exception { 89 | super.setUp(); 90 | 91 | // event time is default 92 | // env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 93 | source = env.addSource(new IntegerLongSource(numberOfElements, RECORDS_PER_INVOCATION)); 94 | } 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/full/StringSerializationBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.full; 20 | 21 | import org.apache.flink.api.common.ExecutionConfig; 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | import org.apache.flink.api.common.typeutils.TypeSerializer; 24 | import org.apache.flink.benchmark.BenchmarkBase; 25 | import org.apache.flink.core.memory.DataInputView; 26 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 27 | import org.apache.flink.core.memory.DataOutputView; 28 | import org.apache.flink.core.memory.DataOutputViewStreamWrapper; 29 | 30 | import org.openjdk.jmh.annotations.Benchmark; 31 | import org.openjdk.jmh.annotations.BenchmarkMode; 32 | import org.openjdk.jmh.annotations.Mode; 33 | import org.openjdk.jmh.annotations.OutputTimeUnit; 34 | import org.openjdk.jmh.annotations.Param; 35 | import org.openjdk.jmh.annotations.Scope; 36 | import org.openjdk.jmh.annotations.Setup; 37 | import org.openjdk.jmh.annotations.State; 38 | import org.openjdk.jmh.runner.Runner; 39 | import org.openjdk.jmh.runner.RunnerException; 40 | import org.openjdk.jmh.runner.options.Options; 41 | import org.openjdk.jmh.runner.options.OptionsBuilder; 42 | import org.openjdk.jmh.runner.options.VerboseMode; 43 | 44 | import java.io.ByteArrayInputStream; 45 | import java.io.ByteArrayOutputStream; 46 | import java.io.IOException; 47 | import java.util.Random; 48 | import java.util.concurrent.TimeUnit; 49 | 50 | @State(Scope.Benchmark) 51 | @BenchmarkMode({Mode.Throughput}) 52 | @OutputTimeUnit(TimeUnit.MILLISECONDS) 53 | public class StringSerializationBenchmark extends BenchmarkBase { 54 | 55 | public static final char[] asciiChars = 56 | "qwertyuiopasdfghjklzxcvbnmQWERTYUIOPASDFGHJKLZXCVBNM1234567890".toCharArray(); 57 | public static final char[] russianChars = 58 | "йцукенгшщзхъфывапролджэячсмитьбюЙЦУКЕНГШЩЗХЪФЫВАПРОЛДЖЭЯЧСМИТЬБЮ".toCharArray(); 59 | public static final char[] chineseChars = 60 | "的是不了人我在有他这为之大来以个中上们到国说和地也子要时道出而于就下得可你年生".toCharArray(); 61 | @Param({"ascii", "russian", "chinese"}) 62 | public String type; 63 | @Param({"4", "128", "16384"}) 64 | public String lengthStr; 65 | int length; 66 | String input; 67 | ExecutionConfig config = new ExecutionConfig(); 68 | TypeSerializer serializer = TypeInformation.of(String.class).createSerializer(config.getSerializerConfig()); 69 | ByteArrayInputStream serializedBuffer; 70 | DataInputView serializedStream; 71 | 72 | public static void main(String[] args) throws RunnerException { 73 | Options options = 74 | new OptionsBuilder() 75 | .verbosity(VerboseMode.NORMAL) 76 | .include( 77 | ".*" + StringSerializationBenchmark.class.getCanonicalName() + ".*") 78 | .build(); 79 | 80 | new Runner(options).run(); 81 | } 82 | 83 | @Setup 84 | public void setup() throws IOException { 85 | length = Integer.parseInt(lengthStr); 86 | switch (type) { 87 | case "ascii": 88 | input = generate(asciiChars, length); 89 | break; 90 | case "russian": 91 | input = generate(russianChars, length); 92 | break; 93 | case "chinese": 94 | input = generate(chineseChars, length); 95 | break; 96 | default: 97 | throw new IllegalArgumentException(type + "charset is not supported"); 98 | } 99 | byte[] stringBytes = stringWrite(); 100 | serializedBuffer = new ByteArrayInputStream(stringBytes); 101 | serializedStream = new DataInputViewStreamWrapper(serializedBuffer); 102 | } 103 | 104 | @Benchmark 105 | public byte[] stringWrite() throws IOException { 106 | ByteArrayOutputStream buffer = new ByteArrayOutputStream(); 107 | DataOutputView out = new DataOutputViewStreamWrapper(buffer); 108 | serializer.serialize(input, out); 109 | return buffer.toByteArray(); 110 | } 111 | 112 | @Benchmark 113 | public String stringRead() throws IOException { 114 | serializedBuffer.reset(); 115 | return serializer.deserialize(serializedStream); 116 | } 117 | 118 | private String generate(char[] charset, int length) { 119 | char[] buffer = new char[length]; 120 | Random random = new Random(); 121 | for (int i = 0; i < length; i++) { 122 | buffer[i] = charset[random.nextInt(charset.length)]; 123 | } 124 | return new String(buffer); 125 | } 126 | } 127 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/full/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This package contains an extended benchmark set which is not used for regression tests but rather 21 | * for performance overview of certain parts of the code. 22 | * 23 | *

Please consider moving benchmarks here to keep the amount of regression benchmarks small. 24 | */ 25 | package org.apache.flink.benchmark.full; 26 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/BaseSourceWithKeyRange.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.streaming.api.functions.source.legacy.ParallelSourceFunction; 22 | 23 | /** Abstract base class for sources with a defined number of events and a fixed key range. */ 24 | public abstract class BaseSourceWithKeyRange implements ParallelSourceFunction { 25 | private static final long serialVersionUID = 8318018060123048234L; 26 | 27 | protected final int numKeys; 28 | protected int remainingEvents; 29 | 30 | public BaseSourceWithKeyRange(int numEvents, int numKeys) { 31 | this.remainingEvents = numEvents; 32 | this.numKeys = numKeys; 33 | } 34 | 35 | protected void init() {} 36 | 37 | protected abstract T getElement(int keyId); 38 | 39 | @Override 40 | public void run(SourceContext out) { 41 | init(); 42 | 43 | int keyId = 0; 44 | while (--remainingEvents >= 0) { 45 | T element = getElement(keyId); 46 | synchronized (out.getCheckpointLock()) { 47 | out.collect(element); 48 | } 49 | ++keyId; 50 | if (keyId >= numKeys) { 51 | keyId = 0; 52 | } 53 | } 54 | } 55 | 56 | @Override 57 | public void cancel() { 58 | this.remainingEvents = 0; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/IntLongApplications.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.benchmark.CollectSink; 22 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 23 | import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; 24 | import org.apache.flink.streaming.api.windowing.windows.Window; 25 | 26 | public class IntLongApplications { 27 | public static void reduceWithWindow( 28 | DataStreamSource source, 29 | WindowAssigner windowAssigner) { 30 | source.map(new MultiplyIntLongByTwo()) 31 | .keyBy(record -> record.key) 32 | .window(windowAssigner) 33 | .reduce(new SumReduceIntLong()) 34 | .addSink(new CollectSink()); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/IntegerLongSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction; 22 | 23 | public class IntegerLongSource extends RichParallelSourceFunction { 24 | private volatile boolean running = true; 25 | private int numberOfKeys; 26 | private long numberOfElements; 27 | public IntegerLongSource(int numberOfKeys, long numberOfElements) { 28 | this.numberOfKeys = numberOfKeys; 29 | this.numberOfElements = numberOfElements; 30 | } 31 | 32 | @Override 33 | public void run(SourceContext ctx) throws Exception { 34 | long counter = 0; 35 | 36 | while (running && counter < numberOfElements) { 37 | synchronized (ctx.getCheckpointLock()) { 38 | ctx.collectWithTimestamp( 39 | Record.of((int) (counter % numberOfKeys), counter), counter); 40 | counter++; 41 | } 42 | } 43 | running = false; 44 | } 45 | 46 | @Override 47 | public void cancel() { 48 | running = false; 49 | } 50 | 51 | public static final class Record { 52 | public final int key; 53 | public final long value; 54 | 55 | public Record() { 56 | this(0, 0); 57 | } 58 | 59 | public Record(int key, long value) { 60 | this.key = key; 61 | this.value = value; 62 | } 63 | 64 | public static Record of(int key, long value) { 65 | return new Record(key, value); 66 | } 67 | 68 | public int getKey() { 69 | return key; 70 | } 71 | 72 | @Override 73 | public String toString() { 74 | return String.format("(%s, %s)", key, value); 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/LongNewSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.connector.source.Boundedness; 22 | import org.apache.flink.api.connector.source.SplitEnumerator; 23 | import org.apache.flink.api.connector.source.SplitEnumeratorContext; 24 | import org.apache.flink.api.connector.source.lib.NumberSequenceSource; 25 | import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; 26 | 27 | import java.util.Collection; 28 | import java.util.List; 29 | import java.util.stream.Collectors; 30 | import java.util.stream.IntStream; 31 | 32 | /** 33 | * The source should produce same records as {@link LongSource}. 34 | * 35 | *

{@link LongSource} generates records from 0 to {@code maxValue} for every parallel instance. 36 | * The original {@link NumberSequenceSource} would split the range 0 to {@code maxValue} between all 37 | * subtasks. 38 | */ 39 | public class LongNewSource extends NumberSequenceSource { 40 | private final Boundedness boundedness; 41 | private final long maxValue; 42 | 43 | public LongNewSource(Boundedness boundedness, long maxValue) { 44 | super(-1, -1); // we do not use the from/to of the underlying source 45 | this.boundedness = boundedness; 46 | this.maxValue = maxValue; 47 | } 48 | 49 | @Override 50 | public Boundedness getBoundedness() { 51 | return boundedness; 52 | } 53 | 54 | @Override 55 | public SplitEnumerator> createEnumerator( 56 | SplitEnumeratorContext splitEnumeratorContext) { 57 | 58 | final List splits = 59 | IntStream.range(0, splitEnumeratorContext.currentParallelism()) 60 | .mapToObj(id -> new NumberSequenceSplit(String.valueOf(id), 0, maxValue)) 61 | .collect(Collectors.toList()); 62 | return new IteratorSourceEnumerator<>(splitEnumeratorContext, splits); 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/LongSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction; 22 | 23 | public class LongSource extends RichParallelSourceFunction { 24 | 25 | private volatile boolean running = true; 26 | private long maxValue; 27 | 28 | public LongSource(long maxValue) { 29 | this.maxValue = maxValue; 30 | } 31 | 32 | @Override 33 | public void run(SourceContext ctx) throws Exception { 34 | long counter = 0; 35 | 36 | while (running) { 37 | synchronized (ctx.getCheckpointLock()) { 38 | ctx.collect(counter); 39 | counter++; 40 | if (counter >= maxValue) { 41 | cancel(); 42 | } 43 | } 44 | } 45 | } 46 | 47 | @Override 48 | public void cancel() { 49 | running = false; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/LongSourceType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.eventtime.WatermarkStrategy; 22 | import org.apache.flink.api.connector.source.Boundedness; 23 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | 26 | import java.util.function.BiFunction; 27 | 28 | /** Enum based factory for different Long sources. */ 29 | public enum LongSourceType { 30 | LEGACY( 31 | (env, maxValue) -> { 32 | return env.addSource(new LongSource(maxValue)); 33 | }), 34 | F27_BOUNDED( 35 | (env, maxValue) -> { 36 | return env.fromSource( 37 | new LongNewSource(Boundedness.BOUNDED, maxValue), 38 | WatermarkStrategy.noWatermarks(), 39 | "NewLongSource"); 40 | }), 41 | F27_UNBOUNDED( 42 | (env, maxValue) -> { 43 | return env.fromSource( 44 | new LongNewSource(Boundedness.CONTINUOUS_UNBOUNDED, maxValue), 45 | WatermarkStrategy.noWatermarks(), 46 | "NewLongSource"); 47 | }); 48 | private final BiFunction> factory; 49 | 50 | LongSourceType(BiFunction> factory) { 51 | this.factory = factory; 52 | } 53 | 54 | public DataStreamSource source(StreamExecutionEnvironment environment, long maxValue) { 55 | return factory.apply(environment, maxValue); 56 | } 57 | }; 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/MultiplyByTwo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.functions.MapFunction; 22 | 23 | public class MultiplyByTwo implements MapFunction { 24 | @Override 25 | public Long map(Long value) throws Exception { 26 | return value * 2; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/MultiplyIntLongByTwo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.functions.MapFunction; 22 | 23 | public class MultiplyIntLongByTwo 24 | implements MapFunction { 25 | @Override 26 | public IntegerLongSource.Record map(IntegerLongSource.Record record) throws Exception { 27 | return IntegerLongSource.Record.of(record.key, record.value * 2); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/QueuingLongSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | public class QueuingLongSource extends LongSource { 22 | 23 | private static Object lock = new Object(); 24 | 25 | private static int currentRank = 1; 26 | 27 | private final int rank; 28 | 29 | public QueuingLongSource(int rank, long maxValue) { 30 | super(maxValue); 31 | this.rank = rank; 32 | } 33 | 34 | public static void reset() { 35 | currentRank = 1; 36 | } 37 | 38 | @Override 39 | public void run(SourceContext ctx) throws Exception { 40 | synchronized (lock) { 41 | while (currentRank != rank) { 42 | lock.wait(); 43 | } 44 | } 45 | 46 | super.run(ctx); 47 | 48 | synchronized (lock) { 49 | currentRank++; 50 | lock.notifyAll(); 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/SuccessException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | public class SuccessException extends Exception {} 22 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/SumReduce.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.functions.ReduceFunction; 22 | 23 | public class SumReduce implements ReduceFunction { 24 | @Override 25 | public Long reduce(Long value1, Long value2) throws Exception { 26 | return value1 + value2; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/SumReduceIntLong.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.functions.ReduceFunction; 22 | 23 | public class SumReduceIntLong implements ReduceFunction { 24 | @Override 25 | public IntegerLongSource.Record reduce( 26 | IntegerLongSource.Record var1, IntegerLongSource.Record var2) throws Exception { 27 | return IntegerLongSource.Record.of(var1.key, var1.value + var2.value); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/TestUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.JobExecutionResult; 22 | import org.apache.flink.client.program.ProgramInvocationException; 23 | import org.apache.flink.runtime.client.JobExecutionException; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | 26 | import static org.junit.Assert.fail; 27 | 28 | /** Test utilities. */ 29 | public class TestUtils { 30 | 31 | public static JobExecutionResult tryExecute(StreamExecutionEnvironment see, String name) 32 | throws Exception { 33 | try { 34 | return see.execute(name); 35 | } catch (ProgramInvocationException | JobExecutionException root) { 36 | Throwable cause = root.getCause(); 37 | 38 | // search for nested SuccessExceptions 39 | int depth = 0; 40 | while (!(cause instanceof SuccessException)) { 41 | if (cause == null || depth++ == 20) { 42 | root.printStackTrace(); 43 | fail("Test failed: " + root.getMessage()); 44 | } else { 45 | cause = cause.getCause(); 46 | } 47 | } 48 | } 49 | 50 | return null; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/functions/ValidatingCounter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.functions; 20 | 21 | import org.apache.flink.api.common.functions.ReduceFunction; 22 | 23 | public class ValidatingCounter implements ReduceFunction { 24 | private long expectedCount; 25 | private long count = 0; 26 | 27 | public ValidatingCounter(long expectedCount) { 28 | this.expectedCount = expectedCount; 29 | } 30 | 31 | @Override 32 | public T reduce(T value1, T value2) throws Exception { 33 | count++; 34 | if (count >= expectedCount) { 35 | throw new SuccessException(); 36 | } 37 | return value1; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/operators/MultiplyByTwoCoStreamMap.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.operators; 20 | 21 | import org.apache.flink.streaming.api.operators.AbstractStreamOperator; 22 | import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; 23 | import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; 24 | 25 | public class MultiplyByTwoCoStreamMap extends AbstractStreamOperator 26 | implements TwoInputStreamOperator { 27 | 28 | @Override 29 | public void processElement1(StreamRecord element) { 30 | output.collect(element.replace(element.getValue() * 2)); 31 | } 32 | 33 | @Override 34 | public void processElement2(StreamRecord element) { 35 | output.collect(element.replace(element.getValue() * 2)); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/benchmark/operators/MultiplyByTwoOperatorFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark.operators; 20 | 21 | import org.apache.flink.streaming.api.operators.AbstractInput; 22 | import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; 23 | import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; 24 | import org.apache.flink.streaming.api.operators.Input; 25 | import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; 26 | import org.apache.flink.streaming.api.operators.StreamOperator; 27 | import org.apache.flink.streaming.api.operators.StreamOperatorParameters; 28 | import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; 29 | 30 | import java.util.Arrays; 31 | import java.util.List; 32 | 33 | @SuppressWarnings({"unchecked", "rawtypes"}) 34 | public class MultiplyByTwoOperatorFactory extends AbstractStreamOperatorFactory { 35 | @Override 36 | public > T createStreamOperator( 37 | StreamOperatorParameters parameters) { 38 | return (T) new MultiplyByTwoOperator(parameters); 39 | } 40 | 41 | @Override 42 | public Class getStreamOperatorClass(ClassLoader classLoader) { 43 | return MultiplyByTwoOperator.class; 44 | } 45 | 46 | public static class MultiplyByTwoOperator extends AbstractStreamOperatorV2 47 | implements MultipleInputStreamOperator { 48 | public MultiplyByTwoOperator(StreamOperatorParameters parameters) { 49 | super(parameters, 2); 50 | } 51 | 52 | @Override 53 | public List getInputs() { 54 | return Arrays.asList( 55 | new MultiplyByTwoOperator.MultiplyByTwoInput(this, 1), 56 | new MultiplyByTwoOperator.MultiplyByTwoInput(this, 2)); 57 | } 58 | 59 | private static class MultiplyByTwoInput extends AbstractInput { 60 | MultiplyByTwoInput(AbstractStreamOperatorV2 owner, int inputId) { 61 | super(owner, inputId); 62 | } 63 | 64 | @Override 65 | public void processElement(StreamRecord element) { 66 | output.collect(element.replace(element.getValue() * 2)); 67 | } 68 | } 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/config/ConfigUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.config; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | 23 | import org.slf4j.Logger; 24 | import org.slf4j.LoggerFactory; 25 | 26 | import java.io.BufferedReader; 27 | import java.io.IOException; 28 | import java.io.InputStream; 29 | import java.io.InputStreamReader; 30 | 31 | public class ConfigUtil { 32 | 33 | private static final Logger LOG = LoggerFactory.getLogger(ConfigUtil.class); 34 | 35 | private static final String BENCHMARK_CONF = "benchmark-conf.yaml"; 36 | 37 | /** Load benchmark conf from classpath. */ 38 | public static Configuration loadBenchMarkConf() { 39 | InputStream inputStream = 40 | ConfigUtil.class.getClassLoader().getResourceAsStream(BENCHMARK_CONF); 41 | return loadYAMLResource(inputStream); 42 | } 43 | 44 | /** 45 | * This is copied from {@code GlobalConfiguration#loadYAMLResource} to avoid depending 46 | * on @Internal api. 47 | */ 48 | private static Configuration loadYAMLResource(InputStream inputStream) { 49 | final Configuration config = new Configuration(); 50 | 51 | try (BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { 52 | 53 | String line; 54 | int lineNo = 0; 55 | while ((line = reader.readLine()) != null) { 56 | lineNo++; 57 | // 1. check for comments 58 | String[] comments = line.split("#", 2); 59 | String conf = comments[0].trim(); 60 | 61 | // 2. get key and value 62 | if (conf.length() > 0) { 63 | String[] kv = conf.split(": ", 2); 64 | 65 | // skip line with no valid key-value pair 66 | if (kv.length == 1) { 67 | LOG.warn( 68 | "Error while trying to split key and value in configuration file " 69 | + ":" 70 | + lineNo 71 | + ": \"" 72 | + line 73 | + "\""); 74 | continue; 75 | } 76 | 77 | String key = kv[0].trim(); 78 | String value = kv[1].trim(); 79 | 80 | // sanity check 81 | if (key.length() == 0 || value.length() == 0) { 82 | LOG.warn( 83 | "Error after splitting key and value in configuration file " 84 | + ":" 85 | + lineNo 86 | + ": \"" 87 | + line 88 | + "\""); 89 | continue; 90 | } 91 | 92 | LOG.info("Loading configuration property: {}, {}", key, value); 93 | config.setString(key, value); 94 | } 95 | } 96 | } catch (IOException e) { 97 | throw new RuntimeException("Error parsing YAML configuration.", e); 98 | } 99 | 100 | return config; 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/config/StateBenchmarkOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.config; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | 23 | import static org.apache.flink.configuration.ConfigOptions.key; 24 | 25 | public class StateBenchmarkOptions { 26 | 27 | public static final ConfigOption STATE_DATA_DIR = 28 | key("benchmark.state.data-dir") 29 | .stringType() 30 | .noDefaultValue() 31 | .withDescription("The dir to put state data."); 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/SchedulerBenchmarkExecutorBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark; 20 | 21 | import org.openjdk.jmh.annotations.BenchmarkMode; 22 | import org.openjdk.jmh.annotations.Fork; 23 | import org.openjdk.jmh.annotations.Mode; 24 | import org.openjdk.jmh.annotations.OutputTimeUnit; 25 | import org.openjdk.jmh.annotations.Scope; 26 | import org.openjdk.jmh.annotations.State; 27 | import org.openjdk.jmh.runner.Runner; 28 | import org.openjdk.jmh.runner.RunnerException; 29 | import org.openjdk.jmh.runner.options.Options; 30 | import org.openjdk.jmh.runner.options.OptionsBuilder; 31 | import org.openjdk.jmh.runner.options.VerboseMode; 32 | 33 | import java.util.concurrent.TimeUnit; 34 | 35 | /** The base class of all benchmarks related to the scheduler. */ 36 | @SuppressWarnings("MethodMayBeStatic") 37 | @State(Scope.Thread) 38 | @OutputTimeUnit(TimeUnit.MILLISECONDS) 39 | @BenchmarkMode(Mode.AverageTime) 40 | @Fork( 41 | value = 6, 42 | jvmArgsAppend = { 43 | "-Djava.rmi.server.hostname=127.0.0.1", 44 | "-Dcom.sun.management.jmxremote.authenticate=false", 45 | "-Dcom.sun.management.jmxremote.ssl=false", 46 | "-Dcom.sun.management.jmxremote.ssl" 47 | }) 48 | public class SchedulerBenchmarkExecutorBase { 49 | 50 | public static void runBenchmark(Class clazz) throws RunnerException { 51 | Options options = 52 | new OptionsBuilder() 53 | .verbosity(VerboseMode.NORMAL) 54 | .include(".*" + clazz.getCanonicalName() + ".*") 55 | .build(); 56 | 57 | new Runner(options).run(); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingDownstreamTasksInBatchJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.deploying; 20 | 21 | import org.apache.flink.runtime.executiongraph.Execution; 22 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 23 | import org.apache.flink.runtime.scheduler.benchmark.deploying.DeployingDownstreamTasksInBatchJobBenchmark; 24 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 25 | 26 | import org.openjdk.jmh.annotations.Benchmark; 27 | import org.openjdk.jmh.annotations.BenchmarkMode; 28 | import org.openjdk.jmh.annotations.Level; 29 | import org.openjdk.jmh.annotations.Mode; 30 | import org.openjdk.jmh.annotations.Param; 31 | import org.openjdk.jmh.annotations.Setup; 32 | import org.openjdk.jmh.annotations.TearDown; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** 36 | * The benchmark of deploying downstream tasks in a BATCH job. The related method is {@link 37 | * Execution#deploy}. 38 | */ 39 | public class DeployingDownstreamTasksInBatchJobBenchmarkExecutor 40 | extends SchedulerBenchmarkExecutorBase { 41 | 42 | @Param("BATCH") 43 | private JobConfiguration jobConfiguration; 44 | 45 | private DeployingDownstreamTasksInBatchJobBenchmark benchmark; 46 | 47 | public static void main(String[] args) throws RunnerException { 48 | runBenchmark(DeployingDownstreamTasksInBatchJobBenchmarkExecutor.class); 49 | } 50 | 51 | @Setup(Level.Trial) 52 | public void setup() throws Exception { 53 | benchmark = new DeployingDownstreamTasksInBatchJobBenchmark(); 54 | benchmark.setup(jobConfiguration); 55 | } 56 | 57 | @Benchmark 58 | @BenchmarkMode(Mode.SingleShotTime) 59 | public void deployDownstreamTasks() throws Exception { 60 | benchmark.deployDownstreamTasks(); 61 | } 62 | 63 | @TearDown(Level.Trial) 64 | public void teardown() { 65 | benchmark.teardown(); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/deploying/DeployingTasksInStreamingJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.deploying; 20 | 21 | import org.apache.flink.runtime.executiongraph.Execution; 22 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 23 | import org.apache.flink.runtime.scheduler.benchmark.deploying.DeployingTasksInStreamingJobBenchmark; 24 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 25 | 26 | import org.openjdk.jmh.annotations.Benchmark; 27 | import org.openjdk.jmh.annotations.BenchmarkMode; 28 | import org.openjdk.jmh.annotations.Level; 29 | import org.openjdk.jmh.annotations.Mode; 30 | import org.openjdk.jmh.annotations.Param; 31 | import org.openjdk.jmh.annotations.Setup; 32 | import org.openjdk.jmh.annotations.TearDown; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** 36 | * The benchmark of deploying tasks in a STREAMING job. The related method is {@link 37 | * Execution#deploy}. 38 | */ 39 | public class DeployingTasksInStreamingJobBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 40 | 41 | @Param("STREAMING") 42 | private JobConfiguration jobConfiguration; 43 | 44 | private DeployingTasksInStreamingJobBenchmark benchmark; 45 | 46 | public static void main(String[] args) throws RunnerException { 47 | runBenchmark(DeployingTasksInStreamingJobBenchmark.class); 48 | } 49 | 50 | @Setup(Level.Trial) 51 | public void setup() throws Exception { 52 | benchmark = new DeployingTasksInStreamingJobBenchmark(); 53 | benchmark.setup(jobConfiguration); 54 | } 55 | 56 | @Benchmark 57 | @BenchmarkMode(Mode.SingleShotTime) 58 | public void deployAllTasks() throws Exception { 59 | benchmark.deployAllTasks(); 60 | } 61 | 62 | @TearDown(Level.Trial) 63 | public void teardown() { 64 | benchmark.teardown(); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/e2e/CreateSchedulerBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.e2e; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.e2e.CreateSchedulerBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.infra.Blackhole; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** The benchmark of creating the scheduler in a STREAMING/BATCH job. */ 36 | public class CreateSchedulerBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 37 | 38 | @Param({"BATCH", "STREAMING"}) 39 | private JobConfiguration jobConfiguration; 40 | 41 | private CreateSchedulerBenchmark benchmark; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | runBenchmark(CreateSchedulerBenchmarkExecutor.class); 45 | } 46 | 47 | @Setup(Level.Trial) 48 | public void setup() throws Exception { 49 | benchmark = new CreateSchedulerBenchmark(); 50 | benchmark.setup(jobConfiguration); 51 | } 52 | 53 | @Benchmark 54 | @BenchmarkMode(Mode.SingleShotTime) 55 | public void createScheduler(Blackhole blackhole) throws Exception { 56 | blackhole.consume(benchmark.createScheduler()); 57 | } 58 | 59 | @TearDown(Level.Trial) 60 | public void teardown() { 61 | benchmark.teardown(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/e2e/HandleGlobalFailureAndRestartAllTasksBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.e2e; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.e2e.HandleGlobalFailureAndRestartAllTasksBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | 34 | /** The benchmark of handle global failure and restarting tasks in a STREAMING/BATCH job. */ 35 | public class HandleGlobalFailureAndRestartAllTasksBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 36 | 37 | @Param({"BATCH", "STREAMING", "BATCH_EVENLY", "STREAMING_EVENLY"}) 38 | private JobConfiguration jobConfiguration; 39 | 40 | private HandleGlobalFailureAndRestartAllTasksBenchmark benchmark; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | runBenchmark(HandleGlobalFailureAndRestartAllTasksBenchmarkExecutor.class); 44 | } 45 | 46 | @Setup(Level.Trial) 47 | public void setup() throws Exception { 48 | benchmark = new HandleGlobalFailureAndRestartAllTasksBenchmark(); 49 | benchmark.setup(jobConfiguration); 50 | } 51 | 52 | @Benchmark 53 | @BenchmarkMode(Mode.SingleShotTime) 54 | public void handleGlobalFailureAndRestartAllTasks() throws Exception { 55 | benchmark.handleGlobalFailureAndRestartAllTasks(); 56 | } 57 | 58 | @TearDown(Level.Trial) 59 | public void teardown() { 60 | benchmark.teardown(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/e2e/SchedulingAndDeployingBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.e2e; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.e2e.SchedulingAndDeployingBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | 34 | /** The benchmark of scheduling and deploying tasks in a STREAMING/BATCH job. */ 35 | public class SchedulingAndDeployingBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 36 | 37 | @Param({"BATCH", "STREAMING"}) 38 | private JobConfiguration jobConfiguration; 39 | 40 | private SchedulingAndDeployingBenchmark benchmark; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | runBenchmark(SchedulingAndDeployingBenchmarkExecutor.class); 44 | } 45 | 46 | @Setup(Level.Trial) 47 | public void setup() throws Exception { 48 | benchmark = new SchedulingAndDeployingBenchmark(); 49 | benchmark.setup(jobConfiguration); 50 | } 51 | 52 | @Benchmark 53 | @BenchmarkMode(Mode.SingleShotTime) 54 | public void startScheduling() throws Exception { 55 | benchmark.startScheduling(); 56 | } 57 | 58 | @TearDown(Level.Trial) 59 | public void teardown() { 60 | benchmark.teardown(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/failover/RegionToRestartInBatchJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.failover; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.failover.RegionToRestartInBatchJobBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.infra.Blackhole; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** The benchmark of calculating the regions to restart when failover occurs in a BATCH job. */ 36 | public class RegionToRestartInBatchJobBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 37 | 38 | @Param("BATCH") 39 | private JobConfiguration jobConfiguration; 40 | 41 | private RegionToRestartInBatchJobBenchmark benchmark; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | runBenchmark(RegionToRestartInBatchJobBenchmarkExecutor.class); 45 | } 46 | 47 | @Setup(Level.Trial) 48 | public void setup() throws Exception { 49 | benchmark = new RegionToRestartInBatchJobBenchmark(); 50 | benchmark.setup(jobConfiguration); 51 | } 52 | 53 | @Benchmark 54 | @BenchmarkMode(Mode.SingleShotTime) 55 | public void calculateRegionToRestart(Blackhole blackhole) { 56 | blackhole.consume(benchmark.calculateRegionToRestart()); 57 | } 58 | 59 | @TearDown(Level.Trial) 60 | public void teardown() { 61 | benchmark.teardown(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/failover/RegionToRestartInStreamingJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.failover; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.failover.RegionToRestartInStreamingJobBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.infra.Blackhole; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** The benchmark of calculating region to restart when failover occurs in a STREAMING job. */ 36 | public class RegionToRestartInStreamingJobBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 37 | 38 | @Param("STREAMING") 39 | private JobConfiguration jobConfiguration; 40 | 41 | private RegionToRestartInStreamingJobBenchmark benchmark; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | runBenchmark(RegionToRestartInStreamingJobBenchmarkExecutor.class); 45 | } 46 | 47 | @Setup(Level.Trial) 48 | public void setup() throws Exception { 49 | benchmark = new RegionToRestartInStreamingJobBenchmark(); 50 | benchmark.setup(jobConfiguration); 51 | } 52 | 53 | @Benchmark 54 | @BenchmarkMode(Mode.SingleShotTime) 55 | public void calculateRegionToRestart(Blackhole blackhole) { 56 | blackhole.consume(benchmark.calculateRegionToRestart()); 57 | } 58 | 59 | @TearDown(Level.Trial) 60 | public void teardown() { 61 | benchmark.teardown(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/partitionrelease/PartitionReleaseInBatchJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.partitionrelease; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.partitionrelease.PartitionReleaseInBatchJobBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | 34 | /** The benchmark of releasing partitions in a BATCH job. */ 35 | public class PartitionReleaseInBatchJobBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 36 | 37 | @Param("BATCH") 38 | private JobConfiguration jobConfiguration; 39 | 40 | private PartitionReleaseInBatchJobBenchmark benchmark; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | runBenchmark(PartitionReleaseInBatchJobBenchmarkExecutor.class); 44 | } 45 | 46 | @Setup(Level.Trial) 47 | public void setup() throws Exception { 48 | benchmark = new PartitionReleaseInBatchJobBenchmark(); 49 | benchmark.setup(jobConfiguration); 50 | } 51 | 52 | @Benchmark 53 | @BenchmarkMode(Mode.SingleShotTime) 54 | public void partitionRelease() { 55 | benchmark.partitionRelease(); 56 | } 57 | 58 | @TearDown(Level.Trial) 59 | public void teardown() { 60 | benchmark.teardown(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/scheduling/InitSchedulingStrategyBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.scheduling; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.scheduling.InitSchedulingStrategyBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.infra.Blackhole; 33 | import org.openjdk.jmh.runner.RunnerException; 34 | 35 | /** The benchmark of initializing the scheduling strategy in a STREAMING/BATCH job. */ 36 | public class InitSchedulingStrategyBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 37 | 38 | @Param({"BATCH", "STREAMING"}) 39 | private JobConfiguration jobConfiguration; 40 | 41 | private InitSchedulingStrategyBenchmark benchmark; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | runBenchmark(InitSchedulingStrategyBenchmarkExecutor.class); 45 | } 46 | 47 | @Setup(Level.Trial) 48 | public void setup() throws Exception { 49 | benchmark = new InitSchedulingStrategyBenchmark(); 50 | benchmark.setup(jobConfiguration); 51 | } 52 | 53 | @Benchmark 54 | @BenchmarkMode(Mode.SingleShotTime) 55 | public void initSchedulingStrategy(Blackhole blackhole) { 56 | blackhole.consume(benchmark.initSchedulingStrategy()); 57 | } 58 | 59 | @TearDown(Level.Trial) 60 | public void teardown() { 61 | benchmark.teardown(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/scheduling/SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.scheduling; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.scheduling.SchedulingDownstreamTasksInBatchJobBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | 34 | /** The benchmark of scheduling downstream task in a BATCH job. */ 35 | public class SchedulingDownstreamTasksInBatchJobBenchmarkExecutor 36 | extends SchedulerBenchmarkExecutorBase { 37 | 38 | @Param({"BATCH", "BATCH_HYBRID_DEFAULT", "BATCH_HYBRID_PARTIAL_FINISHED", "BATCH_HYBRID_ALL_FINISHED"}) 39 | private JobConfiguration jobConfiguration; 40 | 41 | private SchedulingDownstreamTasksInBatchJobBenchmark benchmark; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | runBenchmark(SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.class); 45 | } 46 | 47 | @Setup(Level.Trial) 48 | public void setup() throws Exception { 49 | benchmark = new SchedulingDownstreamTasksInBatchJobBenchmark(); 50 | benchmark.setup(jobConfiguration); 51 | } 52 | 53 | @Benchmark 54 | @BenchmarkMode(Mode.SingleShotTime) 55 | public void schedulingDownstreamTasks() { 56 | benchmark.schedulingDownstreamTasks(); 57 | } 58 | 59 | @TearDown(Level.Trial) 60 | public void teardown() { 61 | benchmark.teardown(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/scheduler/benchmark/topology/BuildExecutionGraphBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.scheduler.benchmark.topology; 20 | 21 | import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; 22 | import org.apache.flink.runtime.scheduler.benchmark.topology.BuildExecutionGraphBenchmark; 23 | import org.apache.flink.scheduler.benchmark.SchedulerBenchmarkExecutorBase; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.BenchmarkMode; 27 | import org.openjdk.jmh.annotations.Level; 28 | import org.openjdk.jmh.annotations.Mode; 29 | import org.openjdk.jmh.annotations.Param; 30 | import org.openjdk.jmh.annotations.Setup; 31 | import org.openjdk.jmh.annotations.TearDown; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | 34 | /** The benchmark of building the topology of ExecutionGraph in a STREAMING/BATCH job. */ 35 | public class BuildExecutionGraphBenchmarkExecutor extends SchedulerBenchmarkExecutorBase { 36 | 37 | @Param({"BATCH", "STREAMING"}) 38 | private JobConfiguration jobConfiguration; 39 | 40 | private BuildExecutionGraphBenchmark benchmark; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | runBenchmark(BuildExecutionGraphBenchmarkExecutor.class); 44 | } 45 | 46 | @Setup(Level.Trial) 47 | public void setup() throws Exception { 48 | benchmark = new BuildExecutionGraphBenchmark(); 49 | benchmark.setup(jobConfiguration); 50 | } 51 | 52 | @Benchmark 53 | @BenchmarkMode(Mode.SingleShotTime) 54 | public void buildTopology() throws Exception { 55 | benchmark.buildTopology(); 56 | } 57 | 58 | @TearDown(Level.Trial) 59 | public void teardown() { 60 | benchmark.teardown(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/HashMapStateBackendRescalingBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark; 20 | 21 | import org.apache.flink.api.common.JobID; 22 | import org.apache.flink.config.ConfigUtil; 23 | import org.apache.flink.config.StateBenchmarkOptions; 24 | import org.apache.flink.configuration.Configuration; 25 | import org.apache.flink.state.benchmark.RescalingBenchmarkBuilder; 26 | import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; 27 | import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage; 28 | import org.openjdk.jmh.annotations.*; 29 | import org.openjdk.jmh.runner.RunnerException; 30 | 31 | import java.io.IOException; 32 | import java.net.URI; 33 | 34 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 35 | import static org.openjdk.jmh.annotations.Mode.AverageTime; 36 | 37 | @OutputTimeUnit(MILLISECONDS) 38 | @BenchmarkMode(AverageTime) 39 | @Warmup(iterations = 3) 40 | public class HashMapStateBackendRescalingBenchmarkExecutor extends RescalingBenchmarkBase { 41 | // numberOfKeys = 1250000, keyLen = 96, valueLen = 128, state size ~= 270MB 42 | private final int numberOfKeys = 1250000; 43 | private final int keyLen = 96; 44 | 45 | public static void main(String[] args) throws RunnerException { 46 | runBenchmark(HashMapStateBackendRescalingBenchmarkExecutor.class); 47 | } 48 | 49 | @Setup(Level.Trial) 50 | public void setUp() throws Exception { 51 | // FsStateBackend is deprecated in favor of HashMapStateBackend with setting checkpointStorage. 52 | HashMapStateBackend stateBackend = new HashMapStateBackend(); 53 | benchmark = 54 | new RescalingBenchmarkBuilder() 55 | .setMaxParallelism(128) 56 | .setParallelismBefore(rescaleType.getParallelismBefore()) 57 | .setParallelismAfter(rescaleType.getParallelismAfter()) 58 | .setCheckpointStorageAccess( 59 | new FileSystemCheckpointStorage(new URI("file://" + prepareDirectory("rescaleDb").getAbsolutePath()), 0) 60 | .createCheckpointStorage(new JobID())) 61 | .setStateBackend(stateBackend) 62 | .setStreamRecordGenerator(new ByteArrayRecordGenerator(numberOfKeys, keyLen)) 63 | .setStateProcessFunctionSupplier(TestKeyedFunction::new) 64 | .build(); 65 | benchmark.setUp(); 66 | } 67 | 68 | @Setup(Level.Invocation) 69 | public void setUpPerInvocation() throws Exception { 70 | benchmark.prepareStateForOperator(rescaleType.getSubtaskIndex()); 71 | } 72 | 73 | @TearDown(Level.Trial) 74 | public void tearDown() throws IOException { 75 | benchmark.tearDown(); 76 | } 77 | 78 | @Benchmark 79 | public void rescaleHeap() throws Exception { 80 | benchmark.rescale(); 81 | } 82 | 83 | @TearDown(Level.Invocation) 84 | public void tearDownPerInvocation() throws Exception { 85 | benchmark.closeOperator(); 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/RocksdbStateBackendRescalingBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.flink.state.benchmark; 19 | 20 | import org.apache.flink.api.common.JobID; 21 | import org.apache.flink.config.ConfigUtil; 22 | import org.apache.flink.config.StateBenchmarkOptions; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; 25 | import org.apache.flink.state.benchmark.RescalingBenchmarkBuilder; 26 | import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage; 27 | 28 | import org.openjdk.jmh.annotations.*; 29 | import org.openjdk.jmh.runner.RunnerException; 30 | 31 | import java.io.IOException; 32 | 33 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 34 | import static org.openjdk.jmh.annotations.Mode.AverageTime; 35 | 36 | @OutputTimeUnit(MILLISECONDS) 37 | @BenchmarkMode(AverageTime) 38 | @Warmup(iterations = 3) 39 | public class RocksdbStateBackendRescalingBenchmarkExecutor extends RescalingBenchmarkBase { 40 | // numberOfKeys = 10_000_000, keyLen = 96, valueLen = 128, state size ~= 2.2GB 41 | private final int numberOfKeys = 10_000_000; 42 | private final int keyLen = 96; 43 | 44 | public static void main(String[] args) throws RunnerException { 45 | runBenchmark(RocksdbStateBackendRescalingBenchmarkExecutor.class); 46 | } 47 | 48 | @Setup(Level.Trial) 49 | public void setUp() throws Exception { 50 | EmbeddedRocksDBStateBackend stateBackend = new EmbeddedRocksDBStateBackend(true); 51 | benchmark = 52 | new RescalingBenchmarkBuilder() 53 | .setMaxParallelism(128) 54 | .setParallelismBefore(rescaleType.getParallelismBefore()) 55 | .setParallelismAfter(rescaleType.getParallelismAfter()) 56 | .setManagedMemorySize(512 * 1024 * 1024) 57 | .setCheckpointStorageAccess( 58 | new FileSystemCheckpointStorage("file://" + prepareDirectory("rescaleDb").getAbsolutePath()) 59 | .createCheckpointStorage(new JobID())) 60 | .setStateBackend(stateBackend) 61 | .setStreamRecordGenerator(new ByteArrayRecordGenerator(numberOfKeys, keyLen)) 62 | .setStateProcessFunctionSupplier(TestKeyedFunction::new) 63 | .build(); 64 | benchmark.setUp(); 65 | } 66 | 67 | @Setup(Level.Invocation) 68 | public void setUpPerInvocation() throws Exception { 69 | benchmark.prepareStateForOperator(rescaleType.getSubtaskIndex()); 70 | } 71 | 72 | @TearDown(Level.Trial) 73 | public void tearDown() throws IOException { 74 | benchmark.tearDown(); 75 | } 76 | 77 | @Benchmark 78 | public void rescaleRocksDB() throws Exception { 79 | benchmark.rescale(); 80 | } 81 | 82 | @TearDown(Level.Invocation) 83 | public void tearDownPerInvocation() throws Exception { 84 | benchmark.closeOperator(); 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/StateBenchmarkConstants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark; 20 | 21 | import java.util.ArrayList; 22 | import java.util.Collections; 23 | import java.util.Random; 24 | 25 | /** 26 | * Constants for state benchmark tests. Also generates random keys/values in advance to avoid 27 | * possible affect of using {@link Random#nextLong()} 28 | */ 29 | public class StateBenchmarkConstants { 30 | // TODO: why all of those static fields? Those should be inside a context class 31 | public static final int mapKeyCount = 10; 32 | public static final int listValueCount = 100; 33 | public static final int setupKeyCount = 500_000; 34 | public static final String rootDirName = "benchmark"; 35 | public static final String recoveryDirName = "localRecovery"; 36 | public static final String dbDirName = "dbPath"; 37 | 38 | public static final ArrayList mapKeys = new ArrayList<>(mapKeyCount); 39 | public static final ArrayList mapValues = new ArrayList<>(mapKeyCount); 40 | public static final ArrayList setupKeys = new ArrayList<>(setupKeyCount); 41 | public static final int newKeyCount = 500_000; 42 | public static final ArrayList newKeys = new ArrayList<>(newKeyCount); 43 | public static final int randomValueCount = 1_000_000; 44 | public static final ArrayList randomValues = new ArrayList<>(randomValueCount); 45 | 46 | static { 47 | for (int i = 0; i < mapKeyCount; i++) { 48 | mapKeys.add((long) i); 49 | } 50 | Collections.shuffle(mapKeys); 51 | } 52 | 53 | static { 54 | Random random = new Random(); 55 | for (int i = 0; i < mapKeyCount; i++) { 56 | mapValues.add(random.nextDouble()); 57 | } 58 | Collections.shuffle(mapValues); 59 | } 60 | 61 | static { 62 | for (long i = 0; i < setupKeyCount; i++) { 63 | setupKeys.add(i); 64 | } 65 | Collections.shuffle(setupKeys); 66 | } 67 | 68 | static { 69 | for (long i = 0; i < newKeyCount; i++) { 70 | newKeys.add(i + setupKeyCount); 71 | } 72 | Collections.shuffle(newKeys); 73 | } 74 | 75 | static { 76 | for (long i = 0; i < randomValueCount; i++) { 77 | randomValues.add(i); 78 | } 79 | Collections.shuffle(randomValues); 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/ValueStateBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark; 20 | 21 | import org.apache.flink.api.common.state.ValueState; 22 | import org.apache.flink.api.common.state.ValueStateDescriptor; 23 | 24 | import org.openjdk.jmh.annotations.Benchmark; 25 | import org.openjdk.jmh.annotations.Setup; 26 | import org.openjdk.jmh.runner.Runner; 27 | import org.openjdk.jmh.runner.RunnerException; 28 | import org.openjdk.jmh.runner.options.Options; 29 | import org.openjdk.jmh.runner.options.OptionsBuilder; 30 | import org.openjdk.jmh.runner.options.VerboseMode; 31 | 32 | import java.io.IOException; 33 | import java.util.concurrent.atomic.AtomicInteger; 34 | 35 | import static org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.getValueState; 36 | import static org.apache.flink.state.benchmark.StateBenchmarkConstants.setupKeyCount; 37 | 38 | /** Implementation for listValue state benchmark testing. */ 39 | public class ValueStateBenchmark extends StateBenchmarkBase { 40 | private ValueState valueState; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options opt = 44 | new OptionsBuilder() 45 | .verbosity(VerboseMode.NORMAL) 46 | .include(".*" + ValueStateBenchmark.class.getCanonicalName() + ".*") 47 | .build(); 48 | 49 | new Runner(opt).run(); 50 | } 51 | 52 | @Setup 53 | public void setUp() throws Exception { 54 | keyedStateBackend = createKeyedStateBackend(); 55 | valueState = 56 | getValueState(keyedStateBackend, new ValueStateDescriptor<>("kvState", Long.class)); 57 | for (int i = 0; i < setupKeyCount; ++i) { 58 | keyedStateBackend.setCurrentKey((long) i); 59 | valueState.update(random.nextLong()); 60 | } 61 | keyIndex = new AtomicInteger(); 62 | } 63 | 64 | @Benchmark 65 | public void valueUpdate(KeyValue keyValue) throws IOException { 66 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 67 | valueState.update(keyValue.value); 68 | } 69 | 70 | @Benchmark 71 | public void valueAdd(KeyValue keyValue) throws IOException { 72 | keyedStateBackend.setCurrentKey(keyValue.newKey); 73 | valueState.update(keyValue.value); 74 | } 75 | 76 | @Benchmark 77 | public Long valueGet(KeyValue keyValue) throws IOException { 78 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 79 | return valueState.value(); 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/ttl/TtlMapStateBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark.ttl; 20 | 21 | import org.apache.flink.api.common.state.MapState; 22 | import org.apache.flink.api.common.state.MapStateDescriptor; 23 | import org.apache.flink.state.benchmark.StateBenchmarkBase; 24 | import org.openjdk.jmh.annotations.Benchmark; 25 | import org.openjdk.jmh.annotations.Level; 26 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 27 | import org.openjdk.jmh.annotations.Setup; 28 | import org.openjdk.jmh.infra.Blackhole; 29 | import org.openjdk.jmh.runner.Runner; 30 | import org.openjdk.jmh.runner.RunnerException; 31 | import org.openjdk.jmh.runner.options.Options; 32 | import org.openjdk.jmh.runner.options.OptionsBuilder; 33 | import org.openjdk.jmh.runner.options.VerboseMode; 34 | 35 | import java.util.HashMap; 36 | import java.util.Iterator; 37 | import java.util.Map; 38 | import java.util.concurrent.atomic.AtomicInteger; 39 | 40 | import static org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.getMapState; 41 | import static org.apache.flink.state.benchmark.StateBenchmarkConstants.mapKeyCount; 42 | import static org.apache.flink.state.benchmark.StateBenchmarkConstants.mapKeys; 43 | import static org.apache.flink.state.benchmark.StateBenchmarkConstants.setupKeyCount; 44 | 45 | /** Implementation for map state benchmark testing. */ 46 | public class TtlMapStateBenchmark extends TtlStateBenchmarkBase { 47 | private MapState mapState; 48 | private Map dummyMaps; 49 | 50 | public static void main(String[] args) throws RunnerException { 51 | Options opt = 52 | new OptionsBuilder() 53 | .verbosity(VerboseMode.NORMAL) 54 | .include(".*" + TtlMapStateBenchmark.class.getCanonicalName() + ".*") 55 | .build(); 56 | 57 | new Runner(opt).run(); 58 | } 59 | 60 | @Setup 61 | public void setUp() throws Exception { 62 | keyedStateBackend = createKeyedStateBackend(); 63 | mapState = 64 | getMapState( 65 | keyedStateBackend, 66 | configTtl(new MapStateDescriptor<>("mapState", Long.class, Double.class))); 67 | dummyMaps = new HashMap<>(mapKeyCount); 68 | for (int i = 0; i < mapKeyCount; ++i) { 69 | dummyMaps.put(mapKeys.get(i), random.nextDouble()); 70 | } 71 | for (int i = 0; i < setupKeyCount; ++i) { 72 | keyedStateBackend.setCurrentKey((long) i); 73 | for (int j = 0; j < mapKeyCount; j++) { 74 | setTtlWhenInitialization(); 75 | mapState.put(mapKeys.get(j), random.nextDouble()); 76 | } 77 | } 78 | keyIndex = new AtomicInteger(); 79 | } 80 | 81 | @Setup(Level.Iteration) 82 | public void setUpPerIteration() throws Exception { 83 | advanceTimePerIteration(); 84 | } 85 | 86 | @Benchmark 87 | public void mapUpdate(StateBenchmarkBase.KeyValue keyValue) throws Exception { 88 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 89 | mapState.put(keyValue.mapKey, keyValue.mapValue); 90 | } 91 | 92 | @Benchmark 93 | public void mapAdd(StateBenchmarkBase.KeyValue keyValue) throws Exception { 94 | keyedStateBackend.setCurrentKey(keyValue.newKey); 95 | mapState.put(keyValue.mapKey, keyValue.mapValue); 96 | } 97 | 98 | @Benchmark 99 | public Double mapGet(StateBenchmarkBase.KeyValue keyValue) throws Exception { 100 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 101 | return mapState.get(keyValue.mapKey); 102 | } 103 | 104 | @Benchmark 105 | public boolean mapIsEmpty(StateBenchmarkBase.KeyValue keyValue) throws Exception { 106 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 107 | return mapState.isEmpty(); 108 | } 109 | 110 | @Benchmark 111 | @OperationsPerInvocation(mapKeyCount) 112 | public void mapIterator(StateBenchmarkBase.KeyValue keyValue, Blackhole bh) throws Exception { 113 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 114 | Iterator> iterator = mapState.iterator(); 115 | while (iterator.hasNext()) { 116 | Map.Entry entry = iterator.next(); 117 | bh.consume(entry.getKey()); 118 | bh.consume(entry.getValue()); 119 | } 120 | } 121 | 122 | @Benchmark 123 | public void mapPutAll(StateBenchmarkBase.KeyValue keyValue) throws Exception { 124 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 125 | mapState.putAll(dummyMaps); 126 | } 127 | } 128 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/ttl/TtlStateBenchmarkBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark.ttl; 20 | 21 | import org.apache.flink.api.common.state.StateDescriptor; 22 | import org.apache.flink.api.common.state.StateTtlConfig; 23 | import org.apache.flink.runtime.state.KeyedStateBackend; 24 | import org.apache.flink.runtime.state.ttl.TtlTimeProvider; 25 | import org.apache.flink.state.benchmark.StateBenchmarkBase; 26 | import org.openjdk.jmh.annotations.Param; 27 | 28 | import java.time.Duration; 29 | 30 | /** The base class for state tests with ttl. */ 31 | public class TtlStateBenchmarkBase extends StateBenchmarkBase { 32 | 33 | private static final long initialTime = 1000000; 34 | 35 | /** The expired time of ttl. */ 36 | public enum ExpiredTimeOptions { 37 | 38 | /** Expire 3 percent of the initial keys per iteration. */ 39 | Expire3PercentPerIteration(3), 40 | 41 | /** never expired but enable the ttl. */ 42 | NeverExpired(0); 43 | 44 | public long advanceTimePerIteration; 45 | ExpiredTimeOptions(int expirePercentPerIteration) { 46 | this.advanceTimePerIteration = initialTime * expirePercentPerIteration / 100; 47 | } 48 | } 49 | 50 | @Param({"Expire3PercentPerIteration", "NeverExpired"}) 51 | protected ExpiredTimeOptions expiredOption; 52 | 53 | @Param({"OnCreateAndWrite", "OnReadAndWrite"}) 54 | protected StateTtlConfig.UpdateType updateType; 55 | 56 | @Param({"NeverReturnExpired", "ReturnExpiredIfNotCleanedUp"}) 57 | protected StateTtlConfig.StateVisibility stateVisibility; 58 | 59 | protected ControllableTtlTimeProvider timeProvider; 60 | 61 | /** Configure the state descriptor with ttl. */ 62 | protected > T configTtl(T stateDescriptor) { 63 | StateTtlConfig ttlConfig = 64 | StateTtlConfig.newBuilder(Duration.ofMillis(initialTime)) 65 | .setUpdateType(updateType) 66 | .setStateVisibility(stateVisibility) 67 | .build(); 68 | stateDescriptor.enableTimeToLive(ttlConfig); 69 | return stateDescriptor; 70 | } 71 | 72 | @Override 73 | protected KeyedStateBackend createKeyedStateBackend() throws Exception { 74 | timeProvider = new ControllableTtlTimeProvider(); 75 | return createKeyedStateBackend(timeProvider); 76 | } 77 | 78 | protected void setTtlWhenInitialization() { 79 | timeProvider.setCurrentTimestamp(random.nextLong(initialTime + 1)); 80 | } 81 | 82 | protected void finishInitialization() { 83 | timeProvider.setCurrentTimestamp(initialTime); 84 | } 85 | 86 | protected void advanceTimePerIteration() { 87 | timeProvider.advanceTimestamp(expiredOption.advanceTimePerIteration); 88 | } 89 | 90 | static class ControllableTtlTimeProvider implements TtlTimeProvider { 91 | 92 | long current = 0L; 93 | 94 | @Override 95 | public long currentTimestamp() { 96 | return current; 97 | } 98 | 99 | public void setCurrentTimestamp(long value) { 100 | current = value; 101 | } 102 | 103 | public void advanceTimestamp(long value) { 104 | current += value; 105 | } 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/state/benchmark/ttl/TtlValueStateBenchmark.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.state.benchmark.ttl; 20 | 21 | import org.apache.flink.api.common.state.ValueState; 22 | import org.apache.flink.api.common.state.ValueStateDescriptor; 23 | import org.openjdk.jmh.annotations.Benchmark; 24 | import org.openjdk.jmh.annotations.Level; 25 | import org.openjdk.jmh.annotations.Setup; 26 | import org.openjdk.jmh.runner.Runner; 27 | import org.openjdk.jmh.runner.RunnerException; 28 | import org.openjdk.jmh.runner.options.Options; 29 | import org.openjdk.jmh.runner.options.OptionsBuilder; 30 | import org.openjdk.jmh.runner.options.VerboseMode; 31 | 32 | import java.io.IOException; 33 | import java.util.concurrent.atomic.AtomicInteger; 34 | 35 | import static org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.getValueState; 36 | import static org.apache.flink.state.benchmark.StateBenchmarkConstants.setupKeyCount; 37 | 38 | /** Implementation for listValue state benchmark testing. */ 39 | public class TtlValueStateBenchmark extends TtlStateBenchmarkBase { 40 | private ValueState valueState; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options opt = 44 | new OptionsBuilder() 45 | .verbosity(VerboseMode.NORMAL) 46 | .include(".*" + TtlValueStateBenchmark.class.getCanonicalName() + ".*") 47 | .build(); 48 | 49 | new Runner(opt).run(); 50 | } 51 | 52 | @Setup 53 | public void setUp() throws Exception { 54 | keyedStateBackend = createKeyedStateBackend(); 55 | valueState = getValueState(keyedStateBackend, configTtl(new ValueStateDescriptor<>("kvState", Long.class))); 56 | for (int i = 0; i < setupKeyCount; ++i) { 57 | setTtlWhenInitialization(); 58 | keyedStateBackend.setCurrentKey((long) i); 59 | valueState.update(random.nextLong()); 60 | } 61 | keyIndex = new AtomicInteger(); 62 | finishInitialization(); 63 | } 64 | 65 | @Setup(Level.Iteration) 66 | public void setUpPerIteration() throws Exception { 67 | advanceTimePerIteration(); 68 | } 69 | 70 | @Benchmark 71 | public void valueUpdate(KeyValue keyValue) throws IOException { 72 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 73 | valueState.update(keyValue.value); 74 | } 75 | 76 | @Benchmark 77 | public void valueAdd(KeyValue keyValue) throws IOException { 78 | keyedStateBackend.setCurrentKey(keyValue.newKey); 79 | valueState.update(keyValue.value); 80 | } 81 | 82 | @Benchmark 83 | public Long valueGet(KeyValue keyValue) throws IOException { 84 | keyedStateBackend.setCurrentKey(keyValue.setUpKey); 85 | return valueState.value(); 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/resources/avro/mypojo.avsc: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | {"namespace": "org.apache.flink.benchmark.avro", 20 | "type": "record", 21 | "name": "MyPojo", 22 | "fields": [ 23 | { 24 | "name": "id", 25 | "type": "int" 26 | }, 27 | { 28 | "name": "name", 29 | "type": "string" 30 | }, 31 | { 32 | "name": "operationName", 33 | "type": { 34 | "type":"array", 35 | "items": "string" 36 | } 37 | }, 38 | { 39 | "name": "operations", 40 | "type": { 41 | "type": "array", 42 | "items": { 43 | "type": "record", 44 | "name": "MyOperation", 45 | "fields": [ 46 | { 47 | "name": "id", 48 | "type": "int" 49 | }, 50 | { 51 | "name": "name", 52 | "type": "string" 53 | } 54 | ] 55 | 56 | } 57 | } 58 | }, 59 | { 60 | "name": "otherId1", 61 | "type": "int" 62 | }, 63 | { 64 | "name": "otherId2", 65 | "type": "int" 66 | }, 67 | { 68 | "name": "otherId3", 69 | "type": "int" 70 | }, 71 | { 72 | "name": "nullable", 73 | "type": ["null", "string"] 74 | } 75 | ] 76 | } -------------------------------------------------------------------------------- /src/main/resources/benchmark-conf.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # The dir to put state data during test. If not set, the system default temp dir will be used. 20 | 21 | #benchmark.state.data-dir: /tmp/flink-benchmark -------------------------------------------------------------------------------- /src/main/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | rootLogger.level = ERROR 20 | rootLogger.appenderRef.main.ref = main 21 | 22 | appender.main.name = main 23 | appender.main.type = CONSOLE 24 | appender.main.target = SYSTEM_ERR 25 | appender.main.layout.type = PatternLayout 26 | appender.main.layout.pattern = %d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n 27 | 28 | 29 | -------------------------------------------------------------------------------- /src/main/resources/protobuf/MyPojo.proto: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | syntax = "proto3"; 20 | package org.apache.flink.benchmark.protobuf; 21 | 22 | message MyPojo { 23 | int32 id = 1; 24 | string name = 2; 25 | repeated string operationName = 3; 26 | repeated MyOperation operations = 4; 27 | int32 otherId1 = 5; 28 | int32 otherId2 = 6; 29 | int32 otherId3 = 7; 30 | string someObject = 8; 31 | } 32 | 33 | message MyOperation { 34 | int32 id = 1; 35 | string name = 2; 36 | } 37 | -------------------------------------------------------------------------------- /src/main/resources/thrift/mypojo.thrift: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /* 20 | * If you want to create updated Java Thrift classes, you need to install the 21 | * thrift binary with a matching version to the dependency inside the pom.xml or 22 | * override it and generate classes via 23 | * 24 | * > mvn generate-sources -Pgenerate-thrift -Dthrift.version=0.13.0 25 | * 26 | * or more dynamically: 27 | * 28 | * > mvn generate-sources -Pgenerate-thrift -Dthrift.version=$(thrift --version | rev | cut -d' ' -f 1 | rev) 29 | * 30 | * Be sure to use the same thrift version when compiling and running 31 | * tests/benchmarks as well to avoid potential conflicts. 32 | */ 33 | 34 | namespace java org.apache.flink.benchmark.thrift 35 | 36 | typedef i32 int 37 | 38 | struct MyPojo { 39 | 1: int id; 40 | 2: string name; 41 | 3: list operationName; 42 | 4: list operations; 43 | 5: int otherId1; 44 | 6: int otherId2; 45 | 7: int otherId3; 46 | 8: optional string someObject; 47 | } 48 | 49 | struct MyOperation { 50 | 1: int id; 51 | 2: string name; 52 | } 53 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/benchmark/DataSkewStreamNetworkThroughputBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.streaming.runtime.io.benchmark.DataSkewStreamNetworkThroughputBenchmark; 23 | 24 | import org.openjdk.jmh.annotations.Benchmark; 25 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 26 | import org.openjdk.jmh.annotations.Setup; 27 | import org.openjdk.jmh.annotations.State; 28 | import org.openjdk.jmh.runner.Runner; 29 | import org.openjdk.jmh.runner.RunnerException; 30 | import org.openjdk.jmh.runner.options.Options; 31 | import org.openjdk.jmh.runner.options.OptionsBuilder; 32 | import org.openjdk.jmh.runner.options.VerboseMode; 33 | 34 | import static org.openjdk.jmh.annotations.Scope.Thread; 35 | 36 | /** JMH throughput benchmark runner for data skew scenario. */ 37 | @OperationsPerInvocation( 38 | value = DataSkewStreamNetworkThroughputBenchmarkExecutor.RECORDS_PER_INVOCATION) 39 | public class DataSkewStreamNetworkThroughputBenchmarkExecutor extends BenchmarkBase { 40 | 41 | static final int RECORDS_PER_INVOCATION = 5_000_000; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | Options options = 45 | new OptionsBuilder() 46 | .verbosity(VerboseMode.NORMAL) 47 | .include( 48 | ".*" 49 | + DataSkewStreamNetworkThroughputBenchmarkExecutor.class 50 | .getCanonicalName() 51 | + ".*") 52 | .build(); 53 | 54 | new Runner(options).run(); 55 | } 56 | 57 | @Benchmark 58 | public void networkSkewedThroughput(MultiEnvironment context) throws Exception { 59 | context.executeBenchmark(RECORDS_PER_INVOCATION); 60 | } 61 | 62 | /** Setup for the benchmark(s). */ 63 | @State(Thread) 64 | public static class MultiEnvironment extends DataSkewStreamNetworkThroughputBenchmark { 65 | // 1ms buffer timeout 66 | private final int flushTimeout = 1; 67 | 68 | // 1000 num of channels (subpartitions) 69 | private final int channels = 1000; 70 | 71 | // 10 writer threads, to increase the load on the machine 72 | private final int writers = 10; 73 | 74 | @Setup 75 | public void setUp() throws Exception { 76 | setUp(writers, channels, flushTimeout, false, false, -1, -1, new Configuration()); 77 | } 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/benchmark/StreamNetworkBroadcastThroughputBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.streaming.runtime.io.benchmark.StreamNetworkBroadcastThroughputBenchmark; 22 | 23 | import org.openjdk.jmh.annotations.Benchmark; 24 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 25 | import org.openjdk.jmh.annotations.Setup; 26 | import org.openjdk.jmh.annotations.State; 27 | import org.openjdk.jmh.annotations.TearDown; 28 | import org.openjdk.jmh.runner.Runner; 29 | import org.openjdk.jmh.runner.RunnerException; 30 | import org.openjdk.jmh.runner.options.Options; 31 | import org.openjdk.jmh.runner.options.OptionsBuilder; 32 | import org.openjdk.jmh.runner.options.VerboseMode; 33 | 34 | import static org.openjdk.jmh.annotations.Scope.Thread; 35 | 36 | /** JMH throughput benchmark runner. */ 37 | @OperationsPerInvocation( 38 | value = StreamNetworkBroadcastThroughputBenchmarkExecutor.RECORDS_PER_INVOCATION) 39 | public class StreamNetworkBroadcastThroughputBenchmarkExecutor extends BenchmarkBase { 40 | 41 | static final int RECORDS_PER_INVOCATION = 500_000; 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | Options options = 45 | new OptionsBuilder() 46 | .verbosity(VerboseMode.NORMAL) 47 | .include( 48 | ".*" 49 | + StreamNetworkBroadcastThroughputBenchmarkExecutor.class 50 | .getCanonicalName() 51 | + ".*") 52 | .build(); 53 | 54 | new Runner(options).run(); 55 | } 56 | 57 | @Benchmark 58 | public void networkBroadcastThroughput(MultiEnvironment context) throws Exception { 59 | context.executeBenchmark(RECORDS_PER_INVOCATION); 60 | } 61 | 62 | /** Setup for the benchmark(s). */ 63 | @State(Thread) 64 | public static class MultiEnvironment extends StreamNetworkBroadcastThroughputBenchmark { 65 | 66 | @Setup 67 | public void setUp() throws Exception { 68 | super.setUp(4, 100, 100); 69 | } 70 | 71 | @TearDown 72 | public void tearDown() throws Exception { 73 | super.tearDown(); 74 | } 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/benchmark/StreamNetworkLatencyBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.streaming.runtime.io.benchmark.StreamNetworkPointToPointBenchmark; 22 | 23 | import org.openjdk.jmh.annotations.Benchmark; 24 | import org.openjdk.jmh.annotations.BenchmarkMode; 25 | import org.openjdk.jmh.annotations.OutputTimeUnit; 26 | import org.openjdk.jmh.annotations.Setup; 27 | import org.openjdk.jmh.annotations.State; 28 | import org.openjdk.jmh.annotations.TearDown; 29 | import org.openjdk.jmh.runner.Runner; 30 | import org.openjdk.jmh.runner.RunnerException; 31 | import org.openjdk.jmh.runner.options.Options; 32 | import org.openjdk.jmh.runner.options.OptionsBuilder; 33 | import org.openjdk.jmh.runner.options.VerboseMode; 34 | 35 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 36 | import static org.openjdk.jmh.annotations.Mode.AverageTime; 37 | import static org.openjdk.jmh.annotations.Scope.Thread; 38 | 39 | /** JMH latency benchmark runner. */ 40 | @OutputTimeUnit(MILLISECONDS) 41 | @BenchmarkMode(AverageTime) 42 | public class StreamNetworkLatencyBenchmarkExecutor extends BenchmarkBase { 43 | 44 | private static final int RECORDS_PER_INVOCATION = 100; 45 | 46 | public static void main(String[] args) throws RunnerException { 47 | Options options = 48 | new OptionsBuilder() 49 | .verbosity(VerboseMode.NORMAL) 50 | .include( 51 | ".*" 52 | + StreamNetworkLatencyBenchmarkExecutor.class 53 | .getCanonicalName() 54 | + ".*") 55 | .build(); 56 | 57 | new Runner(options).run(); 58 | } 59 | 60 | @Benchmark 61 | public void networkLatency1to1(Environment context) throws Exception { 62 | context.executeBenchmark(RECORDS_PER_INVOCATION, false); 63 | } 64 | 65 | /** Setup for the benchmark(s). */ 66 | @State(Thread) 67 | public static class Environment extends StreamNetworkPointToPointBenchmark { 68 | @Setup 69 | public void setUp() throws Exception { 70 | super.setUp(10); 71 | } 72 | 73 | @TearDown 74 | public void tearDown() { 75 | super.tearDown(); 76 | } 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/benchmark/StreamNetworkThroughputBenchmarkExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.benchmark; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.runtime.net.SSLUtilsTest; 23 | import org.apache.flink.streaming.runtime.io.benchmark.StreamNetworkThroughputBenchmark; 24 | 25 | import org.openjdk.jmh.annotations.Benchmark; 26 | import org.openjdk.jmh.annotations.OperationsPerInvocation; 27 | import org.openjdk.jmh.annotations.Param; 28 | import org.openjdk.jmh.annotations.Setup; 29 | import org.openjdk.jmh.annotations.State; 30 | import org.openjdk.jmh.annotations.TearDown; 31 | import org.openjdk.jmh.runner.Runner; 32 | import org.openjdk.jmh.runner.RunnerException; 33 | import org.openjdk.jmh.runner.options.Options; 34 | import org.openjdk.jmh.runner.options.OptionsBuilder; 35 | import org.openjdk.jmh.runner.options.VerboseMode; 36 | 37 | import java.util.Arrays; 38 | 39 | import static org.apache.flink.util.Preconditions.checkArgument; 40 | import static org.openjdk.jmh.annotations.Scope.Thread; 41 | 42 | /** JMH throughput benchmark runner. */ 43 | @OperationsPerInvocation(value = StreamNetworkThroughputBenchmarkExecutor.RECORDS_PER_INVOCATION) 44 | public class StreamNetworkThroughputBenchmarkExecutor extends BenchmarkBase { 45 | 46 | static final int RECORDS_PER_INVOCATION = 5_000_000; 47 | 48 | public static void main(String[] args) throws RunnerException { 49 | Options options = 50 | new OptionsBuilder() 51 | .verbosity(VerboseMode.NORMAL) 52 | .include( 53 | ".*" 54 | + StreamNetworkThroughputBenchmarkExecutor.class 55 | .getCanonicalName() 56 | + ".*") 57 | .build(); 58 | 59 | new Runner(options).run(); 60 | } 61 | 62 | @Benchmark 63 | public void networkThroughput(MultiEnvironment context) throws Exception { 64 | context.executeBenchmark(RECORDS_PER_INVOCATION); 65 | } 66 | 67 | /** Setup for the benchmark(s). */ 68 | @State(Thread) 69 | public static class MultiEnvironment extends StreamNetworkThroughputBenchmark { 70 | 71 | @Param({ 72 | "100,100ms", 73 | "100,100ms,SSL", 74 | "1000,1ms", 75 | "1000,100ms", 76 | "1000,100ms,SSL", 77 | "1000,100ms,OpenSSL" 78 | }) 79 | public String channelsFlushTimeout = "100,100ms"; 80 | 81 | // Do not spam continuous benchmarking with number of writers parameter. 82 | // @Param({"1", "4"}) 83 | public int writers = 4; 84 | 85 | private static String parseEnableSSL(String channelsFlushTimeout) { 86 | String[] parameters = channelsFlushTimeout.split(","); 87 | if (Arrays.asList(parameters).contains("SSL")) { 88 | return "JDK"; 89 | } else if (Arrays.asList(parameters).contains("OpenSSL")) { 90 | return "OPENSSL"; 91 | } else { 92 | return null; 93 | } 94 | } 95 | 96 | private static int parseFlushTimeout(String channelsFlushTimeout) { 97 | String[] parameters = channelsFlushTimeout.split(","); 98 | checkArgument(parameters.length >= 2); 99 | String flushTimeout = parameters[1]; 100 | 101 | checkArgument(flushTimeout.endsWith("ms")); 102 | return Integer.parseInt(flushTimeout.substring(0, flushTimeout.length() - 2)); 103 | } 104 | 105 | private static int parseChannels(String channelsFlushTimeout) { 106 | String[] parameters = channelsFlushTimeout.split(","); 107 | checkArgument(parameters.length >= 1); 108 | return Integer.parseInt(parameters[0]); 109 | } 110 | 111 | @Setup 112 | public void setUp() throws Exception { 113 | int channels = parseChannels(channelsFlushTimeout); 114 | int flushTimeout = parseFlushTimeout(channelsFlushTimeout); 115 | String sslProvider = parseEnableSSL(channelsFlushTimeout); 116 | 117 | setUp( 118 | writers, 119 | channels, 120 | flushTimeout, 121 | false, 122 | false, 123 | -1, 124 | -1, 125 | sslProvider != null 126 | ? SSLUtilsTest.createInternalSslConfigWithKeyAndTrustStores(sslProvider) 127 | : new Configuration()); 128 | } 129 | 130 | @TearDown 131 | public void tearDown() throws Exception { 132 | super.tearDown(); 133 | } 134 | } 135 | } 136 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/config/ConfigUtilTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.config; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | public class ConfigUtilTest { 27 | 28 | @Test 29 | public void testLoadConf() { 30 | Configuration cfg = ConfigUtil.loadBenchMarkConf(); 31 | String dir = cfg.get(StateBenchmarkOptions.STATE_DATA_DIR); 32 | Assert.assertEquals("/tmp/data", dir); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/test/resources/benchmark-conf.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | benchmark.state.data-dir: /tmp/data -------------------------------------------------------------------------------- /src/test/resources/local127.keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-benchmarks/714b6f1d3d1d93768bae07e443686a9e524e8b0a/src/test/resources/local127.keystore -------------------------------------------------------------------------------- /src/test/resources/local127.truststore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-benchmarks/714b6f1d3d1d93768bae07e443686a9e524e8b0a/src/test/resources/local127.truststore --------------------------------------------------------------------------------