diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml
index aaef44bad89714bc269af83ba50f00e3d63217a9..7cf1347be8adcbc862e960063fa48735a3531ca7 100644
--- a/.gitlab-ci.yml
+++ b/.gitlab-ci.yml
@@ -38,6 +38,7 @@ build-benchmarks:
   artifacts:
     paths:
       - "theodolite-benchmarks/build/libs/*.jar"
+      - "theodolite-benchmarks/*/build/libs/*.jar"
       - "theodolite-benchmarks/*/build/distributions/*.tar"
     expire_in: 1 day
 
@@ -124,30 +125,54 @@ spotbugs-benchmarks:
       when: manual
       allow_failure: true
 
-deploy-uc1-kstreams-app:
+deploy-uc1-kstreams:
   extends: .deploy-benchmarks
   variables:
     IMAGE_NAME: "theodolite-uc1-kstreams-app"
     JAVA_PROJECT_NAME: "uc1-application"
 
-deploy-uc2-kstreams-app:
+deploy-uc2-kstreams:
   extends: .deploy-benchmarks
   variables:
     IMAGE_NAME: "theodolite-uc2-kstreams-app"
     JAVA_PROJECT_NAME: "uc2-application"
 
-deploy-uc3-kstreams-app:
+deploy-uc3-kstreams:
   extends: .deploy-benchmarks
   variables:
     IMAGE_NAME: "theodolite-uc3-kstreams-app"
     JAVA_PROJECT_NAME: "uc3-application"
 
-deploy-uc4-kstreams-app:
+deploy-uc4-kstreams:
   extends: .deploy-benchmarks
   variables:
     IMAGE_NAME: "theodolite-uc4-kstreams-app"
     JAVA_PROJECT_NAME: "uc4-application"
 
+deploy-uc1-flink:
+  extends: .deploy-benchmarks
+  variables:
+    IMAGE_NAME: "theodolite-uc1-flink"
+    JAVA_PROJECT_NAME: "uc1-application-flink"
+
+deploy-uc2-flink:
+  extends: .deploy-benchmarks
+  variables:
+    IMAGE_NAME: "theodolite-uc2-flink"
+    JAVA_PROJECT_NAME: "uc2-application-flink"
+
+deploy-uc3-flink:
+  extends: .deploy-benchmarks
+  variables:
+    IMAGE_NAME: "theodolite-uc3-flink"
+    JAVA_PROJECT_NAME: "uc3-application-flink"
+
+deploy-uc4-flink:
+  extends: .deploy-benchmarks
+  variables:
+    IMAGE_NAME: "theodolite-uc4-flink"
+    JAVA_PROJECT_NAME: "uc4-application-flink"
+
 deploy-uc1-load-generator:
   extends: .deploy-benchmarks
   variables:
diff --git a/theodolite-benchmarks/application-kafkastreams-commons/build.gradle b/theodolite-benchmarks/application-kafkastreams-commons/build.gradle
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..c1ce7502eddd48c7fb50f754012334e01823a3c6 100644
--- a/theodolite-benchmarks/application-kafkastreams-commons/build.gradle
+++ b/theodolite-benchmarks/application-kafkastreams-commons/build.gradle
@@ -0,0 +1,10 @@
+dependencies {
+  // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
+  // implementation 'org.slf4j:slf4j-simple:1.7.25'
+  implementation('org.industrial-devops:titan-ccp-common:0.1.0-SNAPSHOT') { changing = true }
+  implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
+  implementation 'org.apache.kafka:kafka-streams:2.6.0'
+
+  // Use JUnit test framework
+  testImplementation 'junit:junit:4.12'
+  }
\ No newline at end of file
diff --git a/theodolite-benchmarks/build.gradle b/theodolite-benchmarks/build.gradle
index ea8fb80bb2c2bac6121dbaaf72f742aa0e9c62bb..4e49dc99394304e6d9fa99e163b60a9877218c15 100644
--- a/theodolite-benchmarks/build.gradle
+++ b/theodolite-benchmarks/build.gradle
@@ -7,14 +7,15 @@ buildscript {
   }
   dependencies {
     classpath "gradle.plugin.com.github.spotbugs.snom:spotbugs-gradle-plugin:4.6.0"
+    classpath "com.github.jengelman.gradle.plugins:shadow:6.0.0"
   }
 }
 
 // Variables used to distinct different subprojects
 def useCaseProjects = subprojects.findAll {it -> it.name.matches('uc(.)*')}
 def useCaseApplications = subprojects.findAll {it -> it.name.matches('uc[0-9]+-application')}
+def useCaseApplicationsFlink = subprojects.findAll {it -> it.name.matches('uc[0-9]+-application-flink')}
 def useCaseGenerators = subprojects.findAll {it -> it.name.matches('uc[0-9]+-workload-generator*')}
-def commonProjects = subprojects.findAll {it -> it.name.matches('(.)*commons(.)*')}
 
 // Plugins
 allprojects {
@@ -32,6 +33,11 @@ configure(useCaseProjects){
     apply plugin: 'application'
 }
 
+configure(useCaseApplicationsFlink){
+    apply plugin: 'com.github.johnrengelman.shadow'
+    applicationDefaultJvmArgs = ["-Dlog4j.configuration=log4j.properties"]
+}
+
 // Java version for all subprojects
 subprojects {
   java {
@@ -52,13 +58,16 @@ allprojects {
 	    maven {
 	    	url "https://oss.sonatype.org/content/repositories/snapshots/"
 	    }
-      maven {
-        url 'https://packages.confluent.io/maven/'
+	    maven {
+	        url 'https://packages.confluent.io/maven/'
+	    }
+    	maven {
+        	url 'https://repository.apache.org/content/repositories/snapshots/'
+    	} // TODO required?
     }
-	}
 }
 
-// Dependencies for all use case applications
+// Dependencies for all Kafka Streams benchmarks (use case applications)
 configure(useCaseApplications) {
   dependencies {
       // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
@@ -75,30 +84,66 @@ configure(useCaseApplications) {
   }
 }
 
-// Dependencies for all use case generators
-configure(useCaseGenerators) {
+// Dependencies for all Flink benchmarks (use case applications)
+configure(useCaseApplicationsFlink) {
+  ext {
+    flinkVersion = '1.12.2'
+    scalaBinaryVersion = '2.12'
+  }
+
   dependencies {
-      // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
-      implementation('org.industrial-devops:titan-ccp-common:0.1.0-SNAPSHOT') { changing = true }
+      // Special version required because of https://issues.apache.org/jira/browse/FLINK-13703
+      implementation('org.industrial-devops:titan-ccp-common:0.1.0-flink-ready-SNAPSHOT') { changing = true }
       implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
-      implementation 'org.slf4j:slf4j-simple:1.7.25'
 
-      // These dependencies are used for the workload-generator-commmon
-      implementation project(':workload-generator-commons')
+      // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
+      implementation 'org.apache.kafka:kafka-clients:2.2.0'
+      implementation 'com.google.guava:guava:24.1-jre'
+      implementation 'com.google.code.gson:gson:2.8.2'
+      implementation 'org.slf4j:slf4j-simple:1.6.1'
+      compile project(':flink-commons')
+
+      //compile group: 'org.apache.kafka', name: 'kafka-clients', version: "2.2.0"
+      compile group: 'org.apache.flink', name: 'flink-java', version: "${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-streaming-java_${scalaBinaryVersion}", version:"${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-table-api-java-bridge_${scalaBinaryVersion}", version: "${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-table-planner-blink_${scalaBinaryVersion}", version: "${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-connector-kafka_${scalaBinaryVersion}", version: "${flinkVersion}"
+      implementation "org.apache.flink:flink-avro:${flinkVersion}"
+      implementation "org.apache.flink:flink-avro-confluent-registry:${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-runtime-web_${scalaBinaryVersion}", version: "${flinkVersion}" // TODO: remove after development
+      compile group: 'org.apache.flink', name: "flink-statebackend-rocksdb_${scalaBinaryVersion}", version: "${flinkVersion}"
+      compile group: 'org.apache.flink', name: "flink-metrics-prometheus_${scalaBinaryVersion}", version: "${flinkVersion}"
 
       // Use JUnit test framework
       testImplementation 'junit:junit:4.12'
   }
+  
+  run.classpath = sourceSets.main.runtimeClasspath
+
+  jar {
+      manifest {
+          attributes 'Built-By': System.getProperty('user.name'),
+                     'Build-Jdk': System.getProperty('java.version')
+      }
+  }
+
+  shadowJar {
+      configurations = [project.configurations.compile]
+      zip64 true
+  }
 }
 
-// Dependencies for all commons
-configure(commonProjects) {
+// Dependencies for all load generators
+configure(useCaseGenerators) {
   dependencies {
       // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
-      implementation 'org.slf4j:slf4j-simple:1.7.25'
       implementation('org.industrial-devops:titan-ccp-common:0.1.0-SNAPSHOT') { changing = true }
       implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
-      implementation 'org.apache.kafka:kafka-streams:2.6.0'
+      implementation 'org.slf4j:slf4j-simple:1.7.25'
+
+      // These dependencies are used for the workload-generator-commmon
+      implementation project(':workload-generator-commons')
 
       // Use JUnit test framework
       testImplementation 'junit:junit:4.12'
diff --git a/theodolite-benchmarks/docker-test/uc1-flink-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc1-flink-docker-compose/docker-compose.yml
new file mode 100755
index 0000000000000000000000000000000000000000..aa35ac2d1dee01cdf25d2eb2ac77bd056865479a
--- /dev/null
+++ b/theodolite-benchmarks/docker-test/uc1-flink-docker-compose/docker-compose.yml
@@ -0,0 +1,69 @@
+version: '2'
+services:
+  zookeeper:
+    image: confluentinc/cp-zookeeper
+    expose:
+      - "9092"
+    environment:
+      ZOOKEEPER_CLIENT_PORT: 2181
+  kafka:
+    image: wurstmeister/kafka
+    expose:
+      - "9092"
+    #ports:
+    #  - 19092:19092
+    environment:
+      KAFKA_LISTENERS: PLAINTEXT://:9092,CONNECTIONS_FROM_HOST://:19092
+      KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092,CONNECTIONS_FROM_HOST://localhost:19092
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,CONNECTIONS_FROM_HOST:PLAINTEXT
+      KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 30000
+      KAFKA_CREATE_TOPICS: "input:3:1,output:3:1,configuration:3:1,aggregation-feedback:3:1"
+  schema-registry:
+    image: confluentinc/cp-schema-registry:5.3.1
+    depends_on:
+      - zookeeper
+      - kafka
+    expose:
+      - "8081"
+    #ports:
+    #  - 8081:8081
+    environment:
+      SCHEMA_REGISTRY_HOST_NAME: schema-registry
+      SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
+  load-generator: 
+    image: ghcr.io/cau-se/theodolite-uc1-workload-generator:latest
+    depends_on:
+      - schema-registry
+      - kafka
+    environment:
+      BOOTSTRAP_SERVER: uc-wg:5701
+      PORT: 5701
+      KAFKA_BOOTSTRAP_SERVERS: kafka:9092
+      SCHEMA_REGISTRY_URL: http://schema-registry:8081
+      NUM_SENSORS: 10
+  benchmark-jobmanager:
+    image: ghcr.io/cau-se/theodolite-uc1-flink:latest
+    ports:
+      - "8080:8081"
+    command: standalone-job --job-classname theodolite.uc1.application.HistoryServiceFlinkJob
+    environment:
+      - KAFKA_BOOTSTRAP_SERVERS=kafka:9092
+      - SCHEMA_REGISTRY_URL=http://schema-registry:8081
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+        parallelism.default: 1
+    depends_on:
+      - schema-registry
+      - kafka
+  benchmark-taskmanager:
+    image: ghcr.io/cau-se/theodolite-uc1-flink:latest
+    command: taskmanager
+    environment:
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+    depends_on:
+      - schema-registry
+      - kafka    
diff --git a/theodolite-benchmarks/docker-test/uc1-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc1-kstreams-docker-compose/docker-compose.yml
similarity index 97%
rename from theodolite-benchmarks/docker-test/uc1-docker-compose/docker-compose.yml
rename to theodolite-benchmarks/docker-test/uc1-kstreams-docker-compose/docker-compose.yml
index 41d981bc4a2d770d42da39c96005d3ed6fcefef5..403becacff5a386eddfaa8e59fe7873d2adb006c 100755
--- a/theodolite-benchmarks/docker-test/uc1-docker-compose/docker-compose.yml
+++ b/theodolite-benchmarks/docker-test/uc1-kstreams-docker-compose/docker-compose.yml
@@ -31,7 +31,7 @@ services:
     environment:
       SCHEMA_REGISTRY_HOST_NAME: schema-registry
       SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
-  uc-app:
+  benchmark:
     image: ghcr.io/cau-se/theodolite-uc1-kstreams-app:latest
     depends_on:
       - schema-registry
@@ -39,7 +39,7 @@ services:
     environment:
       KAFKA_BOOTSTRAP_SERVERS: kafka:9092
       SCHEMA_REGISTRY_URL: http://schema-registry:8081
-  uc-wg: 
+  load-generator: 
     image: ghcr.io/cau-se/theodolite-uc1-workload-generator:latest
     depends_on:
       - schema-registry
diff --git a/theodolite-benchmarks/docker-test/uc2-flink-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc2-flink-docker-compose/docker-compose.yml
new file mode 100755
index 0000000000000000000000000000000000000000..a8bf56d52c1be7fea3f172d86f6deac27fcc24f7
--- /dev/null
+++ b/theodolite-benchmarks/docker-test/uc2-flink-docker-compose/docker-compose.yml
@@ -0,0 +1,70 @@
+version: '2'
+services:
+  zookeeper:
+    #image: wurstmeister/zookeeper
+    image: confluentinc/cp-zookeeper
+    ports:
+      - "2181:2181"
+    environment:
+      ZOOKEEPER_CLIENT_PORT: 2181
+  kafka:
+    image: wurstmeister/kafka
+    expose:
+      - "9092"
+    #ports:
+    #  - 19092:19092
+    environment:
+      KAFKA_LISTENERS: PLAINTEXT://:9092,CONNECTIONS_FROM_HOST://:19092
+      KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092,CONNECTIONS_FROM_HOST://localhost:19092
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,CONNECTIONS_FROM_HOST:PLAINTEXT
+      KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 30000
+      KAFKA_CREATE_TOPICS: "input:3:1,output:3:1,configuration:3:1,aggregation-feedback:3:1"
+  schema-registry:
+    image: confluentinc/cp-schema-registry:5.3.1
+    depends_on:
+      - zookeeper
+      - kafka
+    #ports:
+    #  - "8081:8081"
+    expose:
+      - "8081"
+    environment:
+      SCHEMA_REGISTRY_HOST_NAME: schema-registry
+      SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
+  load-generator: 
+    image: ghcr.io/cau-se/theodolite-uc2-workload-generator:latest
+    depends_on:
+      - schema-registry
+      - kafka
+    environment:
+      BOOTSTRAP_SERVER: uc-wg:5701
+      PORT: 5701
+      KAFKA_BOOTSTRAP_SERVERS: kafka:9092
+      SCHEMA_REGISTRY_URL: http://schema-registry:8081
+      NUM_SENSORS: 10  
+  benchmark-jobmanager:
+    image: ghcr.io/cau-se/theodolite-uc2-flink:latest
+    ports:
+      - "8080:8081"
+    command: standalone-job --job-classname theodolite.uc2.application.HistoryServiceFlinkJob
+    environment:
+      - KAFKA_BOOTSTRAP_SERVERS=kafka:9092
+      - SCHEMA_REGISTRY_URL=http://schema-registry:8081
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+        parallelism.default: 1
+    depends_on:
+      - schema-registry
+      - kafka
+  benchmark-taskmanager:
+    image: ghcr.io/cau-se/theodolite-uc2-flink:latest
+    command: taskmanager
+    environment:
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+    depends_on:
+      - schema-registry
+      - kafka
diff --git a/theodolite-benchmarks/docker-test/uc2-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc2-kstreams-docker-compose/docker-compose.yml
similarity index 98%
rename from theodolite-benchmarks/docker-test/uc2-docker-compose/docker-compose.yml
rename to theodolite-benchmarks/docker-test/uc2-kstreams-docker-compose/docker-compose.yml
index 6bb0c1e7d5afccab7724c40eabc30d4c01d59cb6..20d2c62dac13af29ec50439670308f2911f0d57a 100755
--- a/theodolite-benchmarks/docker-test/uc2-docker-compose/docker-compose.yml
+++ b/theodolite-benchmarks/docker-test/uc2-kstreams-docker-compose/docker-compose.yml
@@ -32,7 +32,7 @@ services:
     environment:
       SCHEMA_REGISTRY_HOST_NAME: schema-registry
       SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
-  uc-app:
+  benchmark:
     image: ghcr.io/cau-se/theodolite-uc2-kstreams-app:latest
     depends_on:
       - schema-registry
@@ -41,7 +41,7 @@ services:
       KAFKA_BOOTSTRAP_SERVERS: kafka:9092
       SCHEMA_REGISTRY_URL: http://schema-registry:8081
       KAFKA_WINDOW_DURATION_MINUTES: 60
-  uc-wg: 
+  load-generator: 
     image: ghcr.io/cau-se/theodolite-uc2-workload-generator:latest
     depends_on:
       - schema-registry
diff --git a/theodolite-benchmarks/docker-test/uc3-flink-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc3-flink-docker-compose/docker-compose.yml
new file mode 100755
index 0000000000000000000000000000000000000000..9999caf046e844d066200ecfbf15d3351c167d31
--- /dev/null
+++ b/theodolite-benchmarks/docker-test/uc3-flink-docker-compose/docker-compose.yml
@@ -0,0 +1,70 @@
+version: '2'
+services:
+  zookeeper:
+    #image: wurstmeister/zookeeper
+    image: confluentinc/cp-zookeeper
+    ports:
+      - "2181:2181"
+    environment:
+      ZOOKEEPER_CLIENT_PORT: 2181
+  kafka:
+    image: wurstmeister/kafka
+    expose:
+      - "9092"
+    #ports:
+    #  - 19092:19092
+    environment:
+      KAFKA_LISTENERS: PLAINTEXT://:9092,CONNECTIONS_FROM_HOST://:19092
+      KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092,CONNECTIONS_FROM_HOST://localhost:19092
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,CONNECTIONS_FROM_HOST:PLAINTEXT
+      KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 30000
+      KAFKA_CREATE_TOPICS: "input:3:1,output:3:1,configuration:3:1,aggregation-feedback:3:1"
+  schema-registry:
+    image: confluentinc/cp-schema-registry:5.3.1
+    depends_on:
+      - zookeeper
+      - kafka
+    #ports:
+    #  - "8081:8081"
+    expose:
+      - "8081"
+    environment:
+      SCHEMA_REGISTRY_HOST_NAME: schema-registry
+      SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
+  load-generator: 
+    image: ghcr.io/cau-se/theodolite-uc3-workload-generator:latest
+    depends_on:
+      - schema-registry
+      - kafka
+    environment:
+      BOOTSTRAP_SERVER: uc-wg:5701
+      PORT: 5701
+      KAFKA_BOOTSTRAP_SERVERS: kafka:9092
+      SCHEMA_REGISTRY_URL: http://schema-registry:8081
+      NUM_SENSORS: 10
+  benchmark-jobmanager:
+    image: ghcr.io/cau-se/theodolite-uc3-flink:latest
+    ports:
+      - "8080:8081"
+    command: standalone-job --job-classname theodolite.uc3.application.HistoryServiceFlinkJob
+    environment:
+      - KAFKA_BOOTSTRAP_SERVERS=kafka:9092
+      - SCHEMA_REGISTRY_URL=http://schema-registry:8081
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+        parallelism.default: 1
+    depends_on:
+      - schema-registry
+      - kafka
+  benchmark-taskmanager:
+    image: ghcr.io/cau-se/theodolite-uc3-flink:latest
+    command: taskmanager
+    environment:
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+    depends_on:
+      - schema-registry
+      - kafka
diff --git a/theodolite-benchmarks/docker-test/uc3-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc3-kstreams-docker-compose/docker-compose.yml
similarity index 97%
rename from theodolite-benchmarks/docker-test/uc3-docker-compose/docker-compose.yml
rename to theodolite-benchmarks/docker-test/uc3-kstreams-docker-compose/docker-compose.yml
index b4ae8c16f9fc21164134aad62e6ef776225fc7db..ef16b858536b0d133dc49d002d16cf6c04193297 100755
--- a/theodolite-benchmarks/docker-test/uc3-docker-compose/docker-compose.yml
+++ b/theodolite-benchmarks/docker-test/uc3-kstreams-docker-compose/docker-compose.yml
@@ -32,7 +32,7 @@ services:
     environment:
       SCHEMA_REGISTRY_HOST_NAME: schema-registry
       SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
-  uc-app:
+      benchmark:
     image: ghcr.io/cau-se/theodolite-uc3-kstreams-app:latest
     depends_on:
       - schema-registry
@@ -40,7 +40,7 @@ services:
     environment:
       KAFKA_BOOTSTRAP_SERVERS: kafka:9092
       SCHEMA_REGISTRY_URL: http://schema-registry:8081
-  uc-wg: 
+  load-generator: 
     image: ghcr.io/cau-se/theodolite-uc3-workload-generator:latest
     depends_on:
       - schema-registry
diff --git a/theodolite-benchmarks/docker-test/uc4-flink-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc4-flink-docker-compose/docker-compose.yml
new file mode 100755
index 0000000000000000000000000000000000000000..80720063991100bae2c8c148f14cd6f1a32bb0ff
--- /dev/null
+++ b/theodolite-benchmarks/docker-test/uc4-flink-docker-compose/docker-compose.yml
@@ -0,0 +1,70 @@
+version: '2'
+services:
+  zookeeper:
+    image: confluentinc/cp-zookeeper
+    expose:
+      - "2181"
+    environment:
+      ZOOKEEPER_CLIENT_PORT: 2181
+  kafka:
+    image: wurstmeister/kafka
+    expose:
+      - "9092"
+    #ports:
+    #  - 19092:19092
+    environment:
+      KAFKA_LISTENERS: PLAINTEXT://:9092,CONNECTIONS_FROM_HOST://:19092
+      KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092,CONNECTIONS_FROM_HOST://localhost:19092
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,CONNECTIONS_FROM_HOST:PLAINTEXT
+      KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 30000
+      KAFKA_CREATE_TOPICS: "input:3:1,output:3:1,configuration:3:1,aggregation-feedback:3:1"
+  schema-registry:
+    image: confluentinc/cp-schema-registry:5.3.1
+    depends_on:
+      - zookeeper
+      - kafka
+    expose:
+      - "8081"
+    #ports:
+    #  - 8081:8081
+    environment:
+      SCHEMA_REGISTRY_HOST_NAME: schema-registry
+      SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
+  load-generator: 
+    image: ghcr.io/cau-se/theodolite-uc4-workload-generator:latest
+    depends_on:
+      - schema-registry
+      - kafka
+    environment:
+      BOOTSTRAP_SERVER: uc-wg:5701
+      PORT: 5701
+      KAFKA_BOOTSTRAP_SERVERS: kafka:9092
+      SCHEMA_REGISTRY_URL: http://schema-registry:8081
+      NUM_SENSORS: 4
+      NUM_NESTED_GROUPS: 4
+  benchmark-jobmanager:
+    image: ghcr.io/cau-se/theodolite-uc4-flink:latest
+    ports:
+      - "8080:8081"
+    command: standalone-job --job-classname theodolite.uc4.application.AggregationServiceFlinkJob
+    environment:
+      - KAFKA_BOOTSTRAP_SERVERS=kafka:9092
+      - SCHEMA_REGISTRY_URL=http://schema-registry:8081
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+        parallelism.default: 1
+    depends_on:
+      - schema-registry
+      - kafka
+  benchmark-taskmanager:
+    image: ghcr.io/cau-se/theodolite-uc4-flink:latest
+    command: taskmanager
+    environment:
+      - |
+        FLINK_PROPERTIES=
+        jobmanager.rpc.address: benchmark-jobmanager
+    depends_on:
+      - schema-registry
+      - kafka
diff --git a/theodolite-benchmarks/docker-test/uc4-docker-compose/docker-compose.yml b/theodolite-benchmarks/docker-test/uc4-kstreams-docker-compose/docker-compose.yml
similarity index 98%
rename from theodolite-benchmarks/docker-test/uc4-docker-compose/docker-compose.yml
rename to theodolite-benchmarks/docker-test/uc4-kstreams-docker-compose/docker-compose.yml
index 14b1c1ad38379f45682c8fffa263ca9c9ada2d4d..5e4cb94469f2f6cc8c48694a7ea6c885f066622d 100755
--- a/theodolite-benchmarks/docker-test/uc4-docker-compose/docker-compose.yml
+++ b/theodolite-benchmarks/docker-test/uc4-kstreams-docker-compose/docker-compose.yml
@@ -31,7 +31,7 @@ services:
     environment:
       SCHEMA_REGISTRY_HOST_NAME: schema-registry
       SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
-  uc-app:
+  benchmark:
     image: ghcr.io/cau-se/theodolite-uc4-kstreams-app:latest
     depends_on:
       - schema-registry
@@ -39,7 +39,7 @@ services:
     environment:
       KAFKA_BOOTSTRAP_SERVERS: kafka:9092
       SCHEMA_REGISTRY_URL: http://schema-registry:8081
-  uc-wg: 
+  load-generator: 
     image: ghcr.io/cau-se/theodolite-uc4-workload-generator:latest
     depends_on:
       - schema-registry
diff --git a/theodolite-benchmarks/flink-commons/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/flink-commons/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..66b402b58f39b79066638ce679c27c0378d5be54
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,128 @@
+cleanup.add_default_serial_version_id=true
+cleanup.add_generated_serial_version_id=false
+cleanup.add_missing_annotations=true
+cleanup.add_missing_deprecated_annotations=true
+cleanup.add_missing_methods=false
+cleanup.add_missing_nls_tags=false
+cleanup.add_missing_override_annotations=true
+cleanup.add_missing_override_annotations_interface_methods=true
+cleanup.add_serial_version_id=false
+cleanup.always_use_blocks=true
+cleanup.always_use_parentheses_in_expressions=false
+cleanup.always_use_this_for_non_static_field_access=true
+cleanup.always_use_this_for_non_static_method_access=true
+cleanup.convert_functional_interfaces=false
+cleanup.convert_to_enhanced_for_loop=true
+cleanup.correct_indentation=true
+cleanup.format_source_code=true
+cleanup.format_source_code_changes_only=false
+cleanup.insert_inferred_type_arguments=false
+cleanup.make_local_variable_final=true
+cleanup.make_parameters_final=true
+cleanup.make_private_fields_final=true
+cleanup.make_type_abstract_if_missing_method=false
+cleanup.make_variable_declarations_final=true
+cleanup.never_use_blocks=false
+cleanup.never_use_parentheses_in_expressions=true
+cleanup.organize_imports=true
+cleanup.qualify_static_field_accesses_with_declaring_class=false
+cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+cleanup.qualify_static_member_accesses_with_declaring_class=true
+cleanup.qualify_static_method_accesses_with_declaring_class=false
+cleanup.remove_private_constructors=true
+cleanup.remove_redundant_modifiers=false
+cleanup.remove_redundant_semicolons=true
+cleanup.remove_redundant_type_arguments=true
+cleanup.remove_trailing_whitespaces=true
+cleanup.remove_trailing_whitespaces_all=true
+cleanup.remove_trailing_whitespaces_ignore_empty=false
+cleanup.remove_unnecessary_casts=true
+cleanup.remove_unnecessary_nls_tags=true
+cleanup.remove_unused_imports=true
+cleanup.remove_unused_local_variables=false
+cleanup.remove_unused_private_fields=true
+cleanup.remove_unused_private_members=false
+cleanup.remove_unused_private_methods=true
+cleanup.remove_unused_private_types=true
+cleanup.sort_members=false
+cleanup.sort_members_all=false
+cleanup.use_anonymous_class_creation=false
+cleanup.use_blocks=true
+cleanup.use_blocks_only_for_return_and_throw=false
+cleanup.use_lambda=true
+cleanup.use_parentheses_in_expressions=true
+cleanup.use_this_for_non_static_field_access=true
+cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+cleanup.use_this_for_non_static_method_access=true
+cleanup.use_this_for_non_static_method_access_only_if_necessary=false
+cleanup_profile=_CAU-SE-Style
+cleanup_settings_version=2
+eclipse.preferences.version=1
+editor_save_participant_org.eclipse.jdt.ui.postsavelistener.cleanup=true
+formatter_profile=_CAU-SE-Style
+formatter_settings_version=15
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=99
+org.eclipse.jdt.ui.text.custom_code_templates=
+sp_cleanup.add_default_serial_version_id=true
+sp_cleanup.add_generated_serial_version_id=false
+sp_cleanup.add_missing_annotations=true
+sp_cleanup.add_missing_deprecated_annotations=true
+sp_cleanup.add_missing_methods=false
+sp_cleanup.add_missing_nls_tags=false
+sp_cleanup.add_missing_override_annotations=true
+sp_cleanup.add_missing_override_annotations_interface_methods=true
+sp_cleanup.add_serial_version_id=false
+sp_cleanup.always_use_blocks=true
+sp_cleanup.always_use_parentheses_in_expressions=false
+sp_cleanup.always_use_this_for_non_static_field_access=true
+sp_cleanup.always_use_this_for_non_static_method_access=true
+sp_cleanup.convert_functional_interfaces=false
+sp_cleanup.convert_to_enhanced_for_loop=true
+sp_cleanup.correct_indentation=true
+sp_cleanup.format_source_code=true
+sp_cleanup.format_source_code_changes_only=false
+sp_cleanup.insert_inferred_type_arguments=false
+sp_cleanup.make_local_variable_final=true
+sp_cleanup.make_parameters_final=true
+sp_cleanup.make_private_fields_final=true
+sp_cleanup.make_type_abstract_if_missing_method=false
+sp_cleanup.make_variable_declarations_final=true
+sp_cleanup.never_use_blocks=false
+sp_cleanup.never_use_parentheses_in_expressions=true
+sp_cleanup.on_save_use_additional_actions=true
+sp_cleanup.organize_imports=true
+sp_cleanup.qualify_static_field_accesses_with_declaring_class=false
+sp_cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_with_declaring_class=true
+sp_cleanup.qualify_static_method_accesses_with_declaring_class=false
+sp_cleanup.remove_private_constructors=true
+sp_cleanup.remove_redundant_modifiers=false
+sp_cleanup.remove_redundant_semicolons=false
+sp_cleanup.remove_redundant_type_arguments=true
+sp_cleanup.remove_trailing_whitespaces=true
+sp_cleanup.remove_trailing_whitespaces_all=true
+sp_cleanup.remove_trailing_whitespaces_ignore_empty=false
+sp_cleanup.remove_unnecessary_casts=true
+sp_cleanup.remove_unnecessary_nls_tags=true
+sp_cleanup.remove_unused_imports=true
+sp_cleanup.remove_unused_local_variables=false
+sp_cleanup.remove_unused_private_fields=true
+sp_cleanup.remove_unused_private_members=false
+sp_cleanup.remove_unused_private_methods=true
+sp_cleanup.remove_unused_private_types=true
+sp_cleanup.sort_members=false
+sp_cleanup.sort_members_all=false
+sp_cleanup.use_anonymous_class_creation=false
+sp_cleanup.use_blocks=true
+sp_cleanup.use_blocks_only_for_return_and_throw=false
+sp_cleanup.use_lambda=true
+sp_cleanup.use_parentheses_in_expressions=true
+sp_cleanup.use_this_for_non_static_field_access=true
+sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+sp_cleanup.use_this_for_non_static_method_access=true
+sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=false
diff --git a/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.checkstyle.prefs b/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.checkstyle.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..87860c815222845c1d264d7d0ce498d3397f8280
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.checkstyle.prefs
@@ -0,0 +1,4 @@
+configFilePath=../config/checkstyle.xml
+customModulesJarPaths=
+eclipse.preferences.version=1
+enabled=true
diff --git a/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.pmd.prefs b/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.pmd.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..efbcb8c9e5d449194a48ca1ea42b7d807b573db9
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/.settings/qa.eclipse.plugin.pmd.prefs
@@ -0,0 +1,4 @@
+customRulesJars=
+eclipse.preferences.version=1
+enabled=true
+ruleSetFilePath=../config/pmd.xml
diff --git a/theodolite-benchmarks/flink-commons/build.gradle b/theodolite-benchmarks/flink-commons/build.gradle
new file mode 100644
index 0000000000000000000000000000000000000000..bbfedff3ded8381a9d42bbad5a0926876bfe4e99
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/build.gradle
@@ -0,0 +1,21 @@
+ext {
+    flinkVersion = '1.12.0'
+    scalaBinaryVersion = '2.12'
+}
+
+dependencies {
+    // Special version required because of https://issues.apache.org/jira/browse/FLINK-13703
+    implementation('org.industrial-devops:titan-ccp-common:0.1.0-flink-ready-SNAPSHOT') { changing = true }
+    implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
+    implementation 'com.google.guava:guava:30.1-jre'
+    compile group: 'org.apache.flink', name: "flink-connector-kafka_${scalaBinaryVersion}", version: "${flinkVersion}"
+    compile group: 'org.apache.flink', name: "flink-statebackend-rocksdb_${scalaBinaryVersion}", version: "${flinkVersion}"
+    compile group: 'org.apache.flink', name: "flink-runtime_${scalaBinaryVersion}", version: "${flinkVersion}"
+    compile group: 'org.apache.flink', name: 'flink-java', version: "${flinkVersion}"
+    compile group: 'org.apache.flink', name: "flink-streaming-java_${scalaBinaryVersion}", version:"${flinkVersion}"
+    implementation "org.apache.flink:flink-avro:${flinkVersion}"
+    implementation "org.apache.flink:flink-avro-confluent-registry:${flinkVersion}"
+    
+    // Use JUnit test framework
+    testImplementation 'junit:junit:4.12'
+  }
\ No newline at end of file
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/ConfigurationKeys.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/ConfigurationKeys.java
new file mode 100644
index 0000000000000000000000000000000000000000..2847ede440ecd65bdf35fc8e825d0f7b723a3f8f
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/ConfigurationKeys.java
@@ -0,0 +1,19 @@
+package theodolite.commons.flink;
+
+/**
+ * Keys to access configuration parameters.
+ */
+public final class ConfigurationKeys {
+
+  public static final String FLINK_STATE_BACKEND = "flink.state.backend";
+
+  public static final String FLINK_STATE_BACKEND_PATH = "flink.state.backend.path";
+
+  public static final String FLINK_STATE_BACKEND_MEMORY_SIZE = // NOPMD
+      "flink.state.backend.memory.size";
+
+  public static final String FLINK_CHECKPOINTING = "checkpointing";
+
+  private ConfigurationKeys() {}
+
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/KafkaConnectorFactory.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/KafkaConnectorFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..55d73b0fb9274b0ae67468d50b7978799d7e6257
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/KafkaConnectorFactory.java
@@ -0,0 +1,154 @@
+package theodolite.commons.flink;
+
+import java.time.Duration;
+import java.util.Properties;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
+import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Serde;
+import theodolite.commons.flink.serialization.FlinkKafkaKeyValueSerde;
+import theodolite.commons.flink.util.SerializableSupplier;
+
+/**
+ * A class for creating {@link FlinkKafkaConsumer} and {@link FlinkKafkaProducer}.
+ */
+public class KafkaConnectorFactory {
+
+  private static final Duration PRODUCER_TRANSACTION_TIMEOUT = Duration.ofMinutes(5);
+
+  private final Properties kafkaProps = new Properties();
+  private final boolean checkpointingEnabled;
+  private final String schemaRegistryUrl;
+
+  /**
+   * Create a new {@link KafkaConnectorFactory} from the provided parameters.
+   */
+  public KafkaConnectorFactory(
+      final String appName,
+      final String bootstrapServers,
+      final boolean checkpointingEnabled,
+      final String schemaRegistryUrl) {
+    this.checkpointingEnabled = checkpointingEnabled;
+    this.schemaRegistryUrl = schemaRegistryUrl;
+    this.kafkaProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+    this.kafkaProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, appName);
+  }
+
+  /**
+   * Create a new {@link FlinkKafkaConsumer} that consumes data using a
+   * {@link DeserializationSchema}.
+   */
+  public <T> FlinkKafkaConsumer<T> createConsumer(final String topic,
+      final DeserializationSchema<T> deserializationSchema) {
+    return this.createBaseConsumer(
+        new FlinkKafkaConsumer<>(topic, deserializationSchema, this.cloneProperties()));
+  }
+
+  /**
+   * Create a new {@link FlinkKafkaConsumer} that consumes data using a
+   * {@link KafkaDeserializationSchema}.
+   */
+  public <T> FlinkKafkaConsumer<T> createConsumer(final String topic,
+      final KafkaDeserializationSchema<T> deserializationSchema) {
+    return this.createBaseConsumer(
+        new FlinkKafkaConsumer<>(topic, deserializationSchema, this.cloneProperties()));
+  }
+
+  /**
+   * Create a new {@link FlinkKafkaConsumer} that consumes {@link Tuple2}s using two Kafka
+   * {@link Serde}s.
+   */
+  public <K, V> FlinkKafkaConsumer<Tuple2<K, V>> createConsumer(
+      final String topic,
+      final SerializableSupplier<Serde<K>> kafkaKeySerde,
+      final SerializableSupplier<Serde<V>> kafkaValueSerde,
+      final TypeInformation<Tuple2<K, V>> typeInformation) {
+    return this.<Tuple2<K, V>>createConsumer(
+        topic,
+        new FlinkKafkaKeyValueSerde<>(
+            topic,
+            kafkaKeySerde,
+            kafkaValueSerde,
+            typeInformation));
+  }
+
+  /**
+   * Create a new {@link FlinkKafkaConsumer} that consumes from a topic associated with Confluent
+   * Schema Registry.
+   */
+  public <T extends SpecificRecord> FlinkKafkaConsumer<T> createConsumer(final String topic,
+      final Class<T> typeClass) {
+    // Maybe move to subclass for Confluent-Schema-Registry-specific things
+    final DeserializationSchema<T> deserializationSchema =
+        ConfluentRegistryAvroDeserializationSchema.forSpecific(typeClass, this.schemaRegistryUrl);
+    return this.createConsumer(topic, deserializationSchema);
+  }
+
+  private <T> FlinkKafkaConsumer<T> createBaseConsumer(final FlinkKafkaConsumer<T> baseConsumer) {
+    baseConsumer.setStartFromGroupOffsets();
+    if (this.checkpointingEnabled) {
+      baseConsumer.setCommitOffsetsOnCheckpoints(true); // TODO Validate if this is sensible
+    }
+    baseConsumer.assignTimestampsAndWatermarks(WatermarkStrategy.forMonotonousTimestamps());
+    return baseConsumer;
+  }
+
+
+  /**
+   * Create a new {@link FlinkKafkaProducer} that produces data using a
+   * {@link KafkaSerializationSchema}.
+   */
+  public <T> FlinkKafkaProducer<T> createProducer(final String topic,
+      final KafkaSerializationSchema<T> serializationSchema) {
+    final Properties producerProps = this.buildProducerProperties();
+    return this.createBaseProducer(new FlinkKafkaProducer<>(
+        topic, serializationSchema, producerProps, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE));
+  }
+
+  /**
+   * Create a new {@link FlinkKafkaProducer} that produces {@link Tuple2}s using two Kafka
+   * {@link Serde}s.
+   */
+  public <K, V> FlinkKafkaProducer<Tuple2<K, V>> createProducer(
+      final String topic,
+      final SerializableSupplier<Serde<K>> kafkaKeySerde,
+      final SerializableSupplier<Serde<V>> kafkaValueSerde,
+      final TypeInformation<Tuple2<K, V>> typeInformation) {
+    return this.createProducer(
+        topic,
+        new FlinkKafkaKeyValueSerde<>(
+            topic,
+            kafkaKeySerde,
+            kafkaValueSerde,
+            typeInformation));
+  }
+
+  private <T> FlinkKafkaProducer<T> createBaseProducer(final FlinkKafkaProducer<T> baseProducer) {
+    baseProducer.setWriteTimestampToKafka(true);
+    return baseProducer;
+  }
+
+  private Properties buildProducerProperties() {
+    final Properties producerProps = this.cloneProperties();
+    producerProps.setProperty(
+        ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
+        String.valueOf(PRODUCER_TRANSACTION_TIMEOUT.toMillis())); // TODO necessary?
+    return producerProps;
+  }
+
+  private Properties cloneProperties() {
+    final Properties props = new Properties();
+    props.putAll(this.kafkaProps);
+    return props;
+  }
+
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/StateBackends.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/StateBackends.java
new file mode 100644
index 0000000000000000000000000000000000000000..a94927e4bf49e1dbe6d109eb8f19f7d292f3d879
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/StateBackends.java
@@ -0,0 +1,68 @@
+package theodolite.commons.flink;
+
+import java.io.IOException;
+import org.apache.commons.configuration2.Configuration;
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides factory methods for creating Flink {@link StateBackend}s.
+ */
+public final class StateBackends {
+
+  public static final String STATE_BACKEND_TYPE_MEMORY = "memory";
+  public static final String STATE_BACKEND_TYPE_FILESYSTEM = "filesystem";
+  public static final String STATE_BACKEND_TYPE_ROCKSDB = "rocksdb";
+  // public static final String STATE_BACKEND_TYPE_DEFAULT = STATE_BACKEND_TYPE_ROCKSDB;
+  public static final String STATE_BACKEND_TYPE_DEFAULT = STATE_BACKEND_TYPE_MEMORY;
+  public static final String DEFAULT_STATE_BACKEND_PATH = "file:///opt/flink/statebackend";
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(StateBackends.class);
+
+  private StateBackends() {}
+
+  /**
+   * Create a Flink {@link StateBackend} from a {@link Configuration} and the
+   * {@code ConfigurationKeys#FLINK_STATE_BACKEND},
+   * {@code ConfigurationKeys#FLINK_STATE_BACKEND_MEMORY_SIZE} and
+   * {@code ConfigurationKeys#FLINK_STATE_BACKEND_PATH} configuration keys. Possible options for the
+   * {@code ConfigurationKeys#FLINK_STATE_BACKEND} configuration are
+   * {@code #STATE_BACKEND_TYPE_ROCKSDB}, {@code #STATE_BACKEND_TYPE_FILESYSTEM} and
+   * {@code StateBackendFactory#STATE_BACKEND_TYPE_MEMORY}, where
+   * {@code StateBackendFactory#STATE_BACKEND_TYPE_ROCKSDB} is the default.
+   */
+  public static StateBackend fromConfiguration(final Configuration configuration) {
+    final String stateBackendType =
+        configuration.getString(ConfigurationKeys.FLINK_STATE_BACKEND, STATE_BACKEND_TYPE_DEFAULT);
+    switch (stateBackendType) {
+      case STATE_BACKEND_TYPE_MEMORY:
+        final int memoryStateBackendSize = configuration.getInt(
+            ConfigurationKeys.FLINK_STATE_BACKEND_MEMORY_SIZE,
+            MemoryStateBackend.DEFAULT_MAX_STATE_SIZE);
+        return new MemoryStateBackend(memoryStateBackendSize);
+      case STATE_BACKEND_TYPE_FILESYSTEM:
+        final String stateBackendPath = configuration.getString(
+            ConfigurationKeys.FLINK_STATE_BACKEND_PATH,
+            DEFAULT_STATE_BACKEND_PATH);
+        return new FsStateBackend(stateBackendPath);
+      case STATE_BACKEND_TYPE_ROCKSDB:
+        final String stateBackendPath2 = configuration.getString(
+            ConfigurationKeys.FLINK_STATE_BACKEND_PATH,
+            DEFAULT_STATE_BACKEND_PATH);
+        try {
+          return new RocksDBStateBackend(stateBackendPath2, true);
+        } catch (final IOException e) {
+          LOGGER.error("Cannot create RocksDB state backend.", e);
+          throw new IllegalStateException(e);
+        }
+      default:
+        throw new IllegalArgumentException(
+            "Unsupported state backend '" + stateBackendType + "' configured.");
+    }
+  }
+
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/TupleType.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/TupleType.java
new file mode 100644
index 0000000000000000000000000000000000000000..360331e4d1e4fdc47a24ac8ae995b7590301f7fd
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/TupleType.java
@@ -0,0 +1,22 @@
+package theodolite.commons.flink;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/**
+ * Helper methods for creating {@link TypeInformation} for {@link Tuple}s. In contrast to
+ * {@code Types#TUPLE(TypeInformation...)}, these methods bring real type safety.
+ */
+public final class TupleType {
+
+  private TupleType() {}
+
+  public static <T1, T2> TypeInformation<Tuple2<T1, T2>> of(// NOPMD
+      final TypeInformation<T1> t0,
+      final TypeInformation<T2> t1) {
+    return Types.TUPLE(t0, t1);
+  }
+
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/FlinkKafkaKeyValueSerde.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/FlinkKafkaKeyValueSerde.java
new file mode 100644
index 0000000000000000000000000000000000000000..22f615a6af4caf575af57dbe9b7f989889c4095f
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/FlinkKafkaKeyValueSerde.java
@@ -0,0 +1,80 @@
+package theodolite.commons.flink.serialization;
+
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
+import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.Serde;
+import theodolite.commons.flink.util.SerializableSupplier;
+
+/**
+ * A {@link KafkaSerializationSchema} and {@link KafkaDeserializationSchema} for an arbitrary
+ * key-value-pair in Kafka, mapped to/from a Flink {@link Tuple2}.
+ *
+ * @param <K> Type of the key.
+ * @param <V> Type of the value.
+ */
+public class FlinkKafkaKeyValueSerde<K, V>
+    implements KafkaDeserializationSchema<Tuple2<K, V>>, KafkaSerializationSchema<Tuple2<K, V>> {
+
+  private static final long serialVersionUID = 2469569396501933443L; // NOPMD
+
+  private final SerializableSupplier<Serde<K>> keySerdeSupplier;
+  private final SerializableSupplier<Serde<V>> valueSerdeSupplier;
+  private final String topic;
+  private final TypeInformation<Tuple2<K, V>> typeInfo;
+
+  private transient Serde<K> keySerde;
+  private transient Serde<V> valueSerde;
+
+  /**
+   * Create a new {@link FlinkKafkaKeyValueSerde}.
+   */
+  public FlinkKafkaKeyValueSerde(final String topic,
+      final SerializableSupplier<Serde<K>> keySerdeSupplier,
+      final SerializableSupplier<Serde<V>> valueSerdeSupplier,
+      final TypeInformation<Tuple2<K, V>> typeInfo) {
+    this.topic = topic;
+    this.typeInfo = typeInfo;
+    this.keySerdeSupplier = keySerdeSupplier;
+    this.valueSerdeSupplier = valueSerdeSupplier;
+  }
+
+  @Override
+  public boolean isEndOfStream(final Tuple2<K, V> nextElement) {
+    return false;
+  }
+
+  @Override
+  public Tuple2<K, V> deserialize(final ConsumerRecord<byte[], byte[]> record) {
+    this.ensureInitialized();
+    final K key = this.keySerde.deserializer().deserialize(this.topic, record.key());
+    final V value = this.valueSerde.deserializer().deserialize(this.topic, record.value());
+    return new Tuple2<>(key, value);
+  }
+
+  @Override
+  public TypeInformation<Tuple2<K, V>> getProducedType() {
+    return this.typeInfo;
+  }
+
+  @Override
+  public ProducerRecord<byte[], byte[]> serialize(final Tuple2<K, V> element,
+      @Nullable final Long timestamp) {
+    this.ensureInitialized();
+    final byte[] key = this.keySerde.serializer().serialize(this.topic, element.f0);
+    final byte[] value = this.valueSerde.serializer().serialize(this.topic, element.f1);
+    return new ProducerRecord<>(this.topic, key, value);
+  }
+
+  private void ensureInitialized() {
+    if (this.keySerde == null || this.valueSerde == null) {
+      this.keySerde = this.keySerdeSupplier.get();
+      this.valueSerde = this.valueSerdeSupplier.get();
+    }
+  }
+
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/StatsSerializer.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/StatsSerializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..f1f9870fda73ccec0fc25c5c70665759ab07d893
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/serialization/StatsSerializer.java
@@ -0,0 +1,30 @@
+package theodolite.commons.flink.serialization;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import com.google.common.math.Stats;
+
+import java.io.Serializable;
+
+/**
+ * Custom Kryo {@link Serializer} for efficient transmission between Flink instances.
+ */
+public class StatsSerializer extends Serializer<Stats> implements Serializable {
+
+  private static final long serialVersionUID = -1276866176534267373L; //NOPMD
+
+  @Override
+  public void write(final Kryo kryo, final Output output, final Stats object) {
+    final byte[] data = object.toByteArray();
+    output.writeInt(data.length);
+    output.writeBytes(data);
+  }
+
+  @Override
+  public Stats read(final Kryo kryo, final Input input, final Class<Stats> type) {
+    final int numBytes = input.readInt();
+    return Stats.fromByteArray(input.readBytes(numBytes));
+  }
+}
diff --git a/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/util/SerializableSupplier.java b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/util/SerializableSupplier.java
new file mode 100644
index 0000000000000000000000000000000000000000..bcc51a9ef7b8bb0f36398ea401f1d2c898472081
--- /dev/null
+++ b/theodolite-benchmarks/flink-commons/src/main/java/theodolite/commons/flink/util/SerializableSupplier.java
@@ -0,0 +1,13 @@
+package theodolite.commons.flink.util;
+
+import java.io.Serializable;
+import java.util.function.Supplier;
+
+/**
+ * Interface for {@link Supplier}s which are serializable.
+ *
+ * @param <T> the type of results supplied by this supplier
+ */
+public interface SerializableSupplier<T> extends Supplier<T>, Serializable { // NOPMD
+  // Nothing to do here
+}
diff --git a/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.jar b/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.jar
index 457aad0d98108420a977756b7145c93c8910b076..e708b1c023ec8b20f512888fe07c5bd3ff77bb8f 100644
Binary files a/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.jar and b/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.jar differ
diff --git a/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.properties b/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.properties
index 4d9ca1649142b0c20144adce78e2472e2da01c30..442d9132ea32808ad980df4bd233b359f76341a7 100644
--- a/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.properties
+++ b/theodolite-benchmarks/gradle/wrapper/gradle-wrapper.properties
@@ -1,5 +1,5 @@
 distributionBase=GRADLE_USER_HOME
 distributionPath=wrapper/dists
-distributionUrl=https\://services.gradle.org/distributions/gradle-6.7.1-bin.zip
+distributionUrl=https\://services.gradle.org/distributions/gradle-6.8.3-bin.zip
 zipStoreBase=GRADLE_USER_HOME
 zipStorePath=wrapper/dists
diff --git a/theodolite-benchmarks/gradlew b/theodolite-benchmarks/gradlew
index af6708ff229fda75da4f7cc4da4747217bac4d53..4f906e0c811fc9e230eb44819f509cd0627f2600 100755
--- a/theodolite-benchmarks/gradlew
+++ b/theodolite-benchmarks/gradlew
@@ -1,5 +1,21 @@
 #!/usr/bin/env sh
 
+#
+# Copyright 2015 the original author or authors.
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#      https://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 ##############################################################################
 ##
 ##  Gradle start up script for UN*X
@@ -28,7 +44,7 @@ APP_NAME="Gradle"
 APP_BASE_NAME=`basename "$0"`
 
 # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
-DEFAULT_JVM_OPTS='"-Xmx64m"'
+DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
 
 # Use the maximum available, or set MAX_FD != -1 to use that value.
 MAX_FD="maximum"
@@ -66,6 +82,7 @@ esac
 
 CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
 
+
 # Determine the Java command to use to start the JVM.
 if [ -n "$JAVA_HOME" ] ; then
     if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
@@ -109,10 +126,11 @@ if $darwin; then
     GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\""
 fi
 
-# For Cygwin, switch paths to Windows format before running java
-if $cygwin ; then
+# For Cygwin or MSYS, switch paths to Windows format before running java
+if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then
     APP_HOME=`cygpath --path --mixed "$APP_HOME"`
     CLASSPATH=`cygpath --path --mixed "$CLASSPATH"`
+
     JAVACMD=`cygpath --unix "$JAVACMD"`
 
     # We build the pattern for arguments to be converted via cygpath
@@ -138,19 +156,19 @@ if $cygwin ; then
         else
             eval `echo args$i`="\"$arg\""
         fi
-        i=$((i+1))
+        i=`expr $i + 1`
     done
     case $i in
-        (0) set -- ;;
-        (1) set -- "$args0" ;;
-        (2) set -- "$args0" "$args1" ;;
-        (3) set -- "$args0" "$args1" "$args2" ;;
-        (4) set -- "$args0" "$args1" "$args2" "$args3" ;;
-        (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
-        (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
-        (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
-        (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
-        (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
+        0) set -- ;;
+        1) set -- "$args0" ;;
+        2) set -- "$args0" "$args1" ;;
+        3) set -- "$args0" "$args1" "$args2" ;;
+        4) set -- "$args0" "$args1" "$args2" "$args3" ;;
+        5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
+        6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
+        7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
+        8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
+        9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
     esac
 fi
 
@@ -159,14 +177,9 @@ save () {
     for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done
     echo " "
 }
-APP_ARGS=$(save "$@")
+APP_ARGS=`save "$@"`
 
 # Collect all arguments for the java command, following the shell quoting and substitution rules
 eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS"
 
-# by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong
-if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then
-  cd "$(dirname "$0")"
-fi
-
 exec "$JAVACMD" "$@"
diff --git a/theodolite-benchmarks/gradlew.bat b/theodolite-benchmarks/gradlew.bat
index 0f8d5937c4ad18feb44a19e55ad1e37cc159260f..ac1b06f93825db68fb0c0b5150917f340eaa5d02 100644
--- a/theodolite-benchmarks/gradlew.bat
+++ b/theodolite-benchmarks/gradlew.bat
@@ -1,3 +1,19 @@
+@rem
+@rem Copyright 2015 the original author or authors.
+@rem
+@rem Licensed under the Apache License, Version 2.0 (the "License");
+@rem you may not use this file except in compliance with the License.
+@rem You may obtain a copy of the License at
+@rem
+@rem      https://www.apache.org/licenses/LICENSE-2.0
+@rem
+@rem Unless required by applicable law or agreed to in writing, software
+@rem distributed under the License is distributed on an "AS IS" BASIS,
+@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@rem See the License for the specific language governing permissions and
+@rem limitations under the License.
+@rem
+
 @if "%DEBUG%" == "" @echo off
 @rem ##########################################################################
 @rem
@@ -13,15 +29,18 @@ if "%DIRNAME%" == "" set DIRNAME=.
 set APP_BASE_NAME=%~n0
 set APP_HOME=%DIRNAME%
 
+@rem Resolve any "." and ".." in APP_HOME to make it shorter.
+for %%i in ("%APP_HOME%") do set APP_HOME=%%~fi
+
 @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
-set DEFAULT_JVM_OPTS="-Xmx64m"
+set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m"
 
 @rem Find java.exe
 if defined JAVA_HOME goto findJavaFromJavaHome
 
 set JAVA_EXE=java.exe
 %JAVA_EXE% -version >NUL 2>&1
-if "%ERRORLEVEL%" == "0" goto init
+if "%ERRORLEVEL%" == "0" goto execute
 
 echo.
 echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
@@ -35,7 +54,7 @@ goto fail
 set JAVA_HOME=%JAVA_HOME:"=%
 set JAVA_EXE=%JAVA_HOME%/bin/java.exe
 
-if exist "%JAVA_EXE%" goto init
+if exist "%JAVA_EXE%" goto execute
 
 echo.
 echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME%
@@ -45,28 +64,14 @@ echo location of your Java installation.
 
 goto fail
 
-:init
-@rem Get command-line arguments, handling Windows variants
-
-if not "%OS%" == "Windows_NT" goto win9xME_args
-
-:win9xME_args
-@rem Slurp the command line arguments.
-set CMD_LINE_ARGS=
-set _SKIP=2
-
-:win9xME_args_slurp
-if "x%~1" == "x" goto execute
-
-set CMD_LINE_ARGS=%*
-
 :execute
 @rem Setup the command line
 
 set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar
 
+
 @rem Execute Gradle
-"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS%
+"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %*
 
 :end
 @rem End local scope for the variables with windows NT shell
diff --git a/theodolite-benchmarks/settings.gradle b/theodolite-benchmarks/settings.gradle
index 5c524a57cedbfdaff4aa8e3e39ed3a07711948bc..6d499616ffe1528f925a56a982d4d26c651a28cd 100644
--- a/theodolite-benchmarks/settings.gradle
+++ b/theodolite-benchmarks/settings.gradle
@@ -1,16 +1,21 @@
 rootProject.name = 'theodolite-benchmarks'
 
-include 'workload-generator-commons'
-include 'application-kafkastreams-commons'
+include 'workload-generator-commons' // TODO Rename to load-generator-commons
+include 'application-kafkastreams-commons' // TODO Rename to kstreams-commons
+include 'flink-commons'
 
-include 'uc1-workload-generator'
-include 'uc1-application'
+include 'uc1-workload-generator' // TODO Rename to uc1-load-generator
+include 'uc1-application' // TODO Rename to uc1-kstreams
+include 'uc1-application-flink' // TODO Rename to uc1-flink
 
-include 'uc2-workload-generator'
-include 'uc2-application'
+include 'uc2-workload-generator' // TODO Rename to uc2-load-generator
+include 'uc2-application' // TODO Rename to uc1-kstreams
+include 'uc2-application-flink' // TODO Rename to uc2-flink
 
-include 'uc3-workload-generator'
-include 'uc3-application'
+include 'uc3-workload-generator' // TODO Rename to uc3-load-generator
+include 'uc3-application' // TODO Rename to uc1-kstreams
+include 'uc3-application-flink' // TODO Rename to uc3-flink
 
-include 'uc4-workload-generator'
-include 'uc4-application'
+include 'uc4-workload-generator' // TODO Rename to uc4-load-generator
+include 'uc4-application' // TODO Rename to uc4-kstreams
+include 'uc4-application-flink' // TODO Rename to uc4-flink
diff --git a/theodolite-benchmarks/uc1-application-flink/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc1-application-flink/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..fa98ca63d77bdee891150bd6713f70197a75cefc
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,127 @@
+cleanup.add_default_serial_version_id=true
+cleanup.add_generated_serial_version_id=false
+cleanup.add_missing_annotations=true
+cleanup.add_missing_deprecated_annotations=true
+cleanup.add_missing_methods=false
+cleanup.add_missing_nls_tags=false
+cleanup.add_missing_override_annotations=true
+cleanup.add_missing_override_annotations_interface_methods=true
+cleanup.add_serial_version_id=false
+cleanup.always_use_blocks=true
+cleanup.always_use_parentheses_in_expressions=false
+cleanup.always_use_this_for_non_static_field_access=true
+cleanup.always_use_this_for_non_static_method_access=true
+cleanup.convert_functional_interfaces=false
+cleanup.convert_to_enhanced_for_loop=true
+cleanup.correct_indentation=true
+cleanup.format_source_code=true
+cleanup.format_source_code_changes_only=false
+cleanup.insert_inferred_type_arguments=false
+cleanup.make_local_variable_final=true
+cleanup.make_parameters_final=true
+cleanup.make_private_fields_final=true
+cleanup.make_type_abstract_if_missing_method=false
+cleanup.make_variable_declarations_final=true
+cleanup.never_use_blocks=false
+cleanup.never_use_parentheses_in_expressions=true
+cleanup.organize_imports=true
+cleanup.qualify_static_field_accesses_with_declaring_class=false
+cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+cleanup.qualify_static_member_accesses_with_declaring_class=true
+cleanup.qualify_static_method_accesses_with_declaring_class=false
+cleanup.remove_private_constructors=true
+cleanup.remove_redundant_modifiers=false
+cleanup.remove_redundant_semicolons=true
+cleanup.remove_redundant_type_arguments=true
+cleanup.remove_trailing_whitespaces=true
+cleanup.remove_trailing_whitespaces_all=true
+cleanup.remove_trailing_whitespaces_ignore_empty=false
+cleanup.remove_unnecessary_casts=true
+cleanup.remove_unnecessary_nls_tags=true
+cleanup.remove_unused_imports=true
+cleanup.remove_unused_local_variables=false
+cleanup.remove_unused_private_fields=true
+cleanup.remove_unused_private_members=false
+cleanup.remove_unused_private_methods=true
+cleanup.remove_unused_private_types=true
+cleanup.sort_members=false
+cleanup.sort_members_all=false
+cleanup.use_anonymous_class_creation=false
+cleanup.use_blocks=true
+cleanup.use_blocks_only_for_return_and_throw=false
+cleanup.use_lambda=true
+cleanup.use_parentheses_in_expressions=true
+cleanup.use_this_for_non_static_field_access=true
+cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+cleanup.use_this_for_non_static_method_access=true
+cleanup.use_this_for_non_static_method_access_only_if_necessary=false
+cleanup_profile=_CAU-SE-Style
+cleanup_settings_version=2
+eclipse.preferences.version=1
+editor_save_participant_org.eclipse.jdt.ui.postsavelistener.cleanup=true
+formatter_profile=_CAU-SE-Style
+formatter_settings_version=15
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=99
+sp_cleanup.add_default_serial_version_id=true
+sp_cleanup.add_generated_serial_version_id=false
+sp_cleanup.add_missing_annotations=true
+sp_cleanup.add_missing_deprecated_annotations=true
+sp_cleanup.add_missing_methods=false
+sp_cleanup.add_missing_nls_tags=false
+sp_cleanup.add_missing_override_annotations=true
+sp_cleanup.add_missing_override_annotations_interface_methods=true
+sp_cleanup.add_serial_version_id=false
+sp_cleanup.always_use_blocks=true
+sp_cleanup.always_use_parentheses_in_expressions=false
+sp_cleanup.always_use_this_for_non_static_field_access=true
+sp_cleanup.always_use_this_for_non_static_method_access=true
+sp_cleanup.convert_functional_interfaces=false
+sp_cleanup.convert_to_enhanced_for_loop=true
+sp_cleanup.correct_indentation=true
+sp_cleanup.format_source_code=true
+sp_cleanup.format_source_code_changes_only=false
+sp_cleanup.insert_inferred_type_arguments=false
+sp_cleanup.make_local_variable_final=true
+sp_cleanup.make_parameters_final=true
+sp_cleanup.make_private_fields_final=true
+sp_cleanup.make_type_abstract_if_missing_method=false
+sp_cleanup.make_variable_declarations_final=true
+sp_cleanup.never_use_blocks=false
+sp_cleanup.never_use_parentheses_in_expressions=true
+sp_cleanup.on_save_use_additional_actions=true
+sp_cleanup.organize_imports=true
+sp_cleanup.qualify_static_field_accesses_with_declaring_class=false
+sp_cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_with_declaring_class=true
+sp_cleanup.qualify_static_method_accesses_with_declaring_class=false
+sp_cleanup.remove_private_constructors=true
+sp_cleanup.remove_redundant_modifiers=false
+sp_cleanup.remove_redundant_semicolons=true
+sp_cleanup.remove_redundant_type_arguments=true
+sp_cleanup.remove_trailing_whitespaces=true
+sp_cleanup.remove_trailing_whitespaces_all=true
+sp_cleanup.remove_trailing_whitespaces_ignore_empty=false
+sp_cleanup.remove_unnecessary_casts=true
+sp_cleanup.remove_unnecessary_nls_tags=true
+sp_cleanup.remove_unused_imports=true
+sp_cleanup.remove_unused_local_variables=false
+sp_cleanup.remove_unused_private_fields=true
+sp_cleanup.remove_unused_private_members=false
+sp_cleanup.remove_unused_private_methods=true
+sp_cleanup.remove_unused_private_types=true
+sp_cleanup.sort_members=false
+sp_cleanup.sort_members_all=false
+sp_cleanup.use_anonymous_class_creation=false
+sp_cleanup.use_blocks=true
+sp_cleanup.use_blocks_only_for_return_and_throw=false
+sp_cleanup.use_lambda=true
+sp_cleanup.use_parentheses_in_expressions=true
+sp_cleanup.use_this_for_non_static_field_access=true
+sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+sp_cleanup.use_this_for_non_static_method_access=true
+sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=false
diff --git a/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs b/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..87860c815222845c1d264d7d0ce498d3397f8280
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
@@ -0,0 +1,4 @@
+configFilePath=../config/checkstyle.xml
+customModulesJarPaths=
+eclipse.preferences.version=1
+enabled=true
diff --git a/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.pmd.prefs b/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..efbcb8c9e5d449194a48ca1ea42b7d807b573db9
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
@@ -0,0 +1,4 @@
+customRulesJars=
+eclipse.preferences.version=1
+enabled=true
+ruleSetFilePath=../config/pmd.xml
diff --git a/theodolite-benchmarks/uc1-application-flink/Dockerfile b/theodolite-benchmarks/uc1-application-flink/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..b66d5bc052693fff17d79bc715322a076589d6a3
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/Dockerfile
@@ -0,0 +1,3 @@
+FROM flink:1.12-scala_2.12-java11
+
+ADD build/libs/uc1-application-flink-all.jar /opt/flink/usrlib/artifacts/uc1-application-flink-all.jar
diff --git a/theodolite-benchmarks/uc1-application-flink/build.gradle b/theodolite-benchmarks/uc1-application-flink/build.gradle
new file mode 100644
index 0000000000000000000000000000000000000000..8b8552dbc0c116a0987dbdfe874ca3111c8f11b9
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/build.gradle
@@ -0,0 +1 @@
+mainClassName = "theodolite.uc1.application.HistoryServiceFlinkJob"
diff --git a/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/ConfigurationKeys.java b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/ConfigurationKeys.java
new file mode 100644
index 0000000000000000000000000000000000000000..ed961bab733a409dc07b1be7fa35562103c3e2f4
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/ConfigurationKeys.java
@@ -0,0 +1,24 @@
+package theodolite.uc1.application;
+
+/**
+ * Keys to access configuration parameters.
+ */
+public final class ConfigurationKeys {
+
+  public static final String APPLICATION_NAME = "application.name";
+
+  public static final String APPLICATION_VERSION = "application.version";
+
+  public static final String COMMIT_INTERVAL_MS = "commit.interval.ms";
+
+  public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers";
+
+  public static final String KAFKA_INPUT_TOPIC = "kafka.input.topic";
+
+  public static final String SCHEMA_REGISTRY_URL = "schema.registry.url";
+
+  public static final String CHECKPOINTING = "checkpointing";
+
+  private ConfigurationKeys() {}
+
+}
diff --git a/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/GsonMapper.java b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/GsonMapper.java
new file mode 100644
index 0000000000000000000000000000000000000000..831db7fe63be6529e6b7ba299dca92b138ff7d13
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/GsonMapper.java
@@ -0,0 +1,22 @@
+package theodolite.uc1.application;
+
+import com.google.gson.Gson;
+import org.apache.flink.api.common.functions.MapFunction;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * {@link MapFunction} which maps {@link ActivePowerRecord}s to their representation as JSON
+ * strings.
+ */
+public class GsonMapper implements MapFunction<ActivePowerRecord, String> {
+
+  private static final long serialVersionUID = -5263671231838353747L; // NOPMD
+
+  private static final Gson GSON = new Gson();
+
+  @Override
+  public String map(final ActivePowerRecord value) throws Exception {
+    return GSON.toJson(value);
+  }
+
+}
diff --git a/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/HistoryServiceFlinkJob.java
new file mode 100644
index 0000000000000000000000000000000000000000..6655b52ec3020f46bb8a37c7124ee870fa663573
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/src/main/java/theodolite/uc1/application/HistoryServiceFlinkJob.java
@@ -0,0 +1,81 @@
+package theodolite.uc1.application;
+
+import org.apache.commons.configuration2.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import theodolite.commons.flink.KafkaConnectorFactory;
+import titan.ccp.common.configuration.ServiceConfigurations;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * The History microservice implemented as a Flink job.
+ */
+public final class HistoryServiceFlinkJob {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(HistoryServiceFlinkJob.class);
+
+  private final Configuration config = ServiceConfigurations.createWithDefaults();
+  private final StreamExecutionEnvironment env;
+  private final String applicationId;
+
+  /**
+   * Create a new instance of the {@link HistoryServiceFlinkJob}.
+   */
+  public HistoryServiceFlinkJob() {
+    final String applicationName = this.config.getString(ConfigurationKeys.APPLICATION_NAME);
+    final String applicationVersion = this.config.getString(ConfigurationKeys.APPLICATION_VERSION);
+    this.applicationId = applicationName + "-" + applicationVersion;
+
+    this.env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    this.configureEnv();
+
+    this.buildPipeline();
+  }
+
+  private void configureEnv() {
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+    final int commitIntervalMs = this.config.getInt(ConfigurationKeys.COMMIT_INTERVAL_MS);
+    if (checkpointing) {
+      this.env.enableCheckpointing(commitIntervalMs);
+    }
+  }
+
+  private void buildPipeline() {
+    final String kafkaBroker = this.config.getString(ConfigurationKeys.KAFKA_BOOTSTRAP_SERVERS);
+    final String schemaRegistryUrl = this.config.getString(ConfigurationKeys.SCHEMA_REGISTRY_URL);
+    final String inputTopic = this.config.getString(ConfigurationKeys.KAFKA_INPUT_TOPIC);
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+
+    final KafkaConnectorFactory kafkaConnector = new KafkaConnectorFactory(
+        this.applicationId, kafkaBroker, checkpointing, schemaRegistryUrl);
+
+    final FlinkKafkaConsumer<ActivePowerRecord> kafkaConsumer =
+        kafkaConnector.createConsumer(inputTopic, ActivePowerRecord.class);
+
+    final DataStream<ActivePowerRecord> stream = this.env.addSource(kafkaConsumer);
+
+    stream
+        .rebalance()
+        .map(new GsonMapper())
+        .flatMap((record, c) -> LOGGER.info("Record: {}", record));
+  }
+
+  /**
+   * Start running this microservice.
+   */
+  public void run() {
+    try {
+      this.env.execute(this.applicationId);
+    } catch (final Exception e) { // NOPMD Execution thrown by Flink
+      LOGGER.error("An error occured while running this job.", e);
+    }
+  }
+
+  public static void main(final String[] args) {
+    new HistoryServiceFlinkJob().run();
+  }
+}
diff --git a/theodolite-benchmarks/uc1-application-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc1-application-flink/src/main/resources/META-INF/application.properties
new file mode 100644
index 0000000000000000000000000000000000000000..905e501b8cb66712f2b245470d96803987a9b93b
--- /dev/null
+++ b/theodolite-benchmarks/uc1-application-flink/src/main/resources/META-INF/application.properties
@@ -0,0 +1,12 @@
+application.name=theodolite-uc1-application
+application.version=0.0.1
+
+kafka.bootstrap.servers=localhost:9092
+kafka.input.topic=input
+kafka.output.topic=output
+
+schema.registry.url=http://localhost:8081
+
+num.threads=1
+commit.interval.ms=1000
+cache.max.bytes.buffering=-1
diff --git a/theodolite-benchmarks/uc1-application/src/main/java/theodolite/uc1/streamprocessing/TopologyBuilder.java b/theodolite-benchmarks/uc1-application/src/main/java/theodolite/uc1/streamprocessing/TopologyBuilder.java
index 75c833aa722654395b1adc6f739395eea5256820..427a838f45f6807ede00dcb68ebf8c5580f28ce6 100644
--- a/theodolite-benchmarks/uc1-application/src/main/java/theodolite/uc1/streamprocessing/TopologyBuilder.java
+++ b/theodolite-benchmarks/uc1-application/src/main/java/theodolite/uc1/streamprocessing/TopologyBuilder.java
@@ -17,11 +17,11 @@ import titan.ccp.model.records.ActivePowerRecord;
 public class TopologyBuilder {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(TopologyBuilder.class);
+  private static final Gson GSON = new Gson();
 
   private final String inputTopic;
   private final SchemaRegistryAvroSerdeFactory srAvroSerdeFactory;
 
-  private final Gson gson = new Gson();
   private final StreamsBuilder builder = new StreamsBuilder();
 
 
@@ -42,8 +42,8 @@ public class TopologyBuilder {
         .stream(this.inputTopic, Consumed.with(
             Serdes.String(),
             this.srAvroSerdeFactory.<ActivePowerRecord>forValues()))
-        .mapValues(v -> this.gson.toJson(v))
-        .foreach((k, v) -> LOGGER.info("Key: " + k + " Value: " + v));
+        .mapValues(v -> GSON.toJson(v))
+        .foreach((k, record) -> LOGGER.info("Record: {}", record));
 
     return this.builder.build(properties);
   }
diff --git a/theodolite-benchmarks/uc2-application-flink/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc2-application-flink/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..4d01df75552c562406705858b6368ecf59d6e82f
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,128 @@
+cleanup.add_default_serial_version_id=true
+cleanup.add_generated_serial_version_id=false
+cleanup.add_missing_annotations=true
+cleanup.add_missing_deprecated_annotations=true
+cleanup.add_missing_methods=false
+cleanup.add_missing_nls_tags=false
+cleanup.add_missing_override_annotations=true
+cleanup.add_missing_override_annotations_interface_methods=true
+cleanup.add_serial_version_id=false
+cleanup.always_use_blocks=true
+cleanup.always_use_parentheses_in_expressions=false
+cleanup.always_use_this_for_non_static_field_access=true
+cleanup.always_use_this_for_non_static_method_access=true
+cleanup.convert_functional_interfaces=false
+cleanup.convert_to_enhanced_for_loop=true
+cleanup.correct_indentation=true
+cleanup.format_source_code=true
+cleanup.format_source_code_changes_only=false
+cleanup.insert_inferred_type_arguments=false
+cleanup.make_local_variable_final=true
+cleanup.make_parameters_final=true
+cleanup.make_private_fields_final=true
+cleanup.make_type_abstract_if_missing_method=false
+cleanup.make_variable_declarations_final=true
+cleanup.never_use_blocks=false
+cleanup.never_use_parentheses_in_expressions=true
+cleanup.organize_imports=true
+cleanup.qualify_static_field_accesses_with_declaring_class=false
+cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+cleanup.qualify_static_member_accesses_with_declaring_class=true
+cleanup.qualify_static_method_accesses_with_declaring_class=false
+cleanup.remove_private_constructors=true
+cleanup.remove_redundant_modifiers=false
+cleanup.remove_redundant_semicolons=true
+cleanup.remove_redundant_type_arguments=true
+cleanup.remove_trailing_whitespaces=true
+cleanup.remove_trailing_whitespaces_all=true
+cleanup.remove_trailing_whitespaces_ignore_empty=false
+cleanup.remove_unnecessary_casts=true
+cleanup.remove_unnecessary_nls_tags=true
+cleanup.remove_unused_imports=true
+cleanup.remove_unused_local_variables=false
+cleanup.remove_unused_private_fields=true
+cleanup.remove_unused_private_members=false
+cleanup.remove_unused_private_methods=true
+cleanup.remove_unused_private_types=true
+cleanup.sort_members=false
+cleanup.sort_members_all=false
+cleanup.use_anonymous_class_creation=false
+cleanup.use_blocks=true
+cleanup.use_blocks_only_for_return_and_throw=false
+cleanup.use_lambda=true
+cleanup.use_parentheses_in_expressions=true
+cleanup.use_this_for_non_static_field_access=true
+cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+cleanup.use_this_for_non_static_method_access=true
+cleanup.use_this_for_non_static_method_access_only_if_necessary=false
+cleanup_profile=_CAU-SE-Style
+cleanup_settings_version=2
+eclipse.preferences.version=1
+editor_save_participant_org.eclipse.jdt.ui.postsavelistener.cleanup=true
+formatter_profile=_CAU-SE-Style
+formatter_settings_version=15
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=99
+org.eclipse.jdt.ui.text.custom_code_templates=
+sp_cleanup.add_default_serial_version_id=true
+sp_cleanup.add_generated_serial_version_id=false
+sp_cleanup.add_missing_annotations=true
+sp_cleanup.add_missing_deprecated_annotations=true
+sp_cleanup.add_missing_methods=false
+sp_cleanup.add_missing_nls_tags=false
+sp_cleanup.add_missing_override_annotations=true
+sp_cleanup.add_missing_override_annotations_interface_methods=true
+sp_cleanup.add_serial_version_id=false
+sp_cleanup.always_use_blocks=true
+sp_cleanup.always_use_parentheses_in_expressions=false
+sp_cleanup.always_use_this_for_non_static_field_access=true
+sp_cleanup.always_use_this_for_non_static_method_access=true
+sp_cleanup.convert_functional_interfaces=false
+sp_cleanup.convert_to_enhanced_for_loop=true
+sp_cleanup.correct_indentation=true
+sp_cleanup.format_source_code=true
+sp_cleanup.format_source_code_changes_only=false
+sp_cleanup.insert_inferred_type_arguments=false
+sp_cleanup.make_local_variable_final=true
+sp_cleanup.make_parameters_final=true
+sp_cleanup.make_private_fields_final=true
+sp_cleanup.make_type_abstract_if_missing_method=false
+sp_cleanup.make_variable_declarations_final=true
+sp_cleanup.never_use_blocks=false
+sp_cleanup.never_use_parentheses_in_expressions=true
+sp_cleanup.on_save_use_additional_actions=true
+sp_cleanup.organize_imports=true
+sp_cleanup.qualify_static_field_accesses_with_declaring_class=false
+sp_cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_with_declaring_class=true
+sp_cleanup.qualify_static_method_accesses_with_declaring_class=false
+sp_cleanup.remove_private_constructors=true
+sp_cleanup.remove_redundant_modifiers=false
+sp_cleanup.remove_redundant_semicolons=true
+sp_cleanup.remove_redundant_type_arguments=true
+sp_cleanup.remove_trailing_whitespaces=true
+sp_cleanup.remove_trailing_whitespaces_all=true
+sp_cleanup.remove_trailing_whitespaces_ignore_empty=false
+sp_cleanup.remove_unnecessary_casts=true
+sp_cleanup.remove_unnecessary_nls_tags=true
+sp_cleanup.remove_unused_imports=true
+sp_cleanup.remove_unused_local_variables=false
+sp_cleanup.remove_unused_private_fields=true
+sp_cleanup.remove_unused_private_members=false
+sp_cleanup.remove_unused_private_methods=true
+sp_cleanup.remove_unused_private_types=true
+sp_cleanup.sort_members=false
+sp_cleanup.sort_members_all=false
+sp_cleanup.use_anonymous_class_creation=false
+sp_cleanup.use_blocks=true
+sp_cleanup.use_blocks_only_for_return_and_throw=false
+sp_cleanup.use_lambda=true
+sp_cleanup.use_parentheses_in_expressions=true
+sp_cleanup.use_this_for_non_static_field_access=true
+sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+sp_cleanup.use_this_for_non_static_method_access=true
+sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=false
diff --git a/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs b/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..87860c815222845c1d264d7d0ce498d3397f8280
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
@@ -0,0 +1,4 @@
+configFilePath=../config/checkstyle.xml
+customModulesJarPaths=
+eclipse.preferences.version=1
+enabled=true
diff --git a/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.pmd.prefs b/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..efbcb8c9e5d449194a48ca1ea42b7d807b573db9
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
@@ -0,0 +1,4 @@
+customRulesJars=
+eclipse.preferences.version=1
+enabled=true
+ruleSetFilePath=../config/pmd.xml
diff --git a/theodolite-benchmarks/uc2-application-flink/Dockerfile b/theodolite-benchmarks/uc2-application-flink/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..fe7e7b75d77488a55a01e8d2a890ddd69cab76e3
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/Dockerfile
@@ -0,0 +1,3 @@
+FROM flink:1.12-scala_2.12-java11
+
+ADD build/libs/uc2-application-flink-all.jar /opt/flink/usrlib/artifacts/uc2-application-flink-all.jar
\ No newline at end of file
diff --git a/theodolite-benchmarks/uc2-application-flink/build.gradle b/theodolite-benchmarks/uc2-application-flink/build.gradle
new file mode 100644
index 0000000000000000000000000000000000000000..b5e847553db8f3847d5fe858c76b31520f728aff
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/build.gradle
@@ -0,0 +1,13 @@
+allprojects {
+	repositories {
+    	maven {
+    		url 'https://packages.confluent.io/maven/'
+    	}
+	}
+}
+
+dependencies {
+    compile('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT')
+}
+
+mainClassName = "theodolite.uc2.application.HistoryServiceFlinkJob"
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/ConfigurationKeys.java b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/ConfigurationKeys.java
new file mode 100644
index 0000000000000000000000000000000000000000..9ba56c828a0ae5c6147aadd90d449c7cf2324992
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/ConfigurationKeys.java
@@ -0,0 +1,35 @@
+package theodolite.uc2.application;
+
+/**
+ * Keys to access configuration parameters.
+ */
+public final class ConfigurationKeys {
+
+  public static final String APPLICATION_NAME = "application.name";
+
+  public static final String APPLICATION_VERSION = "application.version";
+
+  public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers";
+
+  public static final String KAFKA_OUTPUT_TOPIC = "kafka.output.topic";
+
+  public static final String KAFKA_INPUT_TOPIC = "kafka.input.topic";
+
+  public static final String SCHEMA_REGISTRY_URL = "schema.registry.url";
+
+  public static final String COMMIT_INTERVAL_MS = "commit.interval.ms";
+
+  public static final String KAFKA_WINDOW_DURATION_MINUTES = "kafka.window.duration.minutes";
+
+  public static final String FLINK_STATE_BACKEND = "flink.state.backend";
+
+  public static final String FLINK_STATE_BACKEND_PATH = "flink.state.backend.path";
+
+  public static final String FLINK_STATE_BACKEND_MEMORY_SIZE = // NOPMD
+      "flink.state.backend.memory.size";
+
+  public static final String CHECKPOINTING = "checkpointing";
+
+  private ConfigurationKeys() {}
+
+}
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/HistoryServiceFlinkJob.java
new file mode 100644
index 0000000000000000000000000000000000000000..b8452847df800226ad481f9309323a2a9a532939
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/HistoryServiceFlinkJob.java
@@ -0,0 +1,126 @@
+package theodolite.uc2.application;
+
+import com.google.common.math.Stats;
+import org.apache.commons.configuration2.Configuration;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import theodolite.commons.flink.KafkaConnectorFactory;
+import theodolite.commons.flink.StateBackends;
+import theodolite.commons.flink.serialization.StatsSerializer;
+import titan.ccp.common.configuration.ServiceConfigurations;
+import titan.ccp.model.records.ActivePowerRecord;
+
+
+/**
+ * The History microservice implemented as a Flink job.
+ */
+public final class HistoryServiceFlinkJob {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(HistoryServiceFlinkJob.class);
+
+  private final Configuration config = ServiceConfigurations.createWithDefaults();
+  private final StreamExecutionEnvironment env;
+  private final String applicationId;
+
+  /**
+   * Create a new instance of the {@link HistoryServiceFlinkJob}.
+   */
+  public HistoryServiceFlinkJob() {
+    final String applicationName = this.config.getString(ConfigurationKeys.APPLICATION_NAME);
+    final String applicationVersion = this.config.getString(ConfigurationKeys.APPLICATION_VERSION);
+    this.applicationId = applicationName + "-" + applicationVersion;
+
+    this.env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    this.configureEnv();
+
+    this.buildPipeline();
+  }
+
+  private void configureEnv() {
+    this.env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+    final int commitIntervalMs = this.config.getInt(ConfigurationKeys.COMMIT_INTERVAL_MS);
+    if (checkpointing) {
+      this.env.enableCheckpointing(commitIntervalMs);
+    }
+
+    // State Backend
+    final StateBackend stateBackend = StateBackends.fromConfiguration(this.config);
+    this.env.setStateBackend(stateBackend);
+
+    this.configureSerializers();
+  }
+
+  private void configureSerializers() {
+    this.env.getConfig().registerTypeWithKryoSerializer(Stats.class, new StatsSerializer());
+    this.env.getConfig().getRegisteredTypesWithKryoSerializers()
+        .forEach((c, s) -> LOGGER.info("Class " + c.getName() + " registered with serializer "
+            + s.getSerializer().getClass().getName()));
+
+  }
+
+  private void buildPipeline() {
+    final String kafkaBroker = this.config.getString(ConfigurationKeys.KAFKA_BOOTSTRAP_SERVERS);
+    final String schemaRegistryUrl = this.config.getString(ConfigurationKeys.SCHEMA_REGISTRY_URL);
+    final String inputTopic = this.config.getString(ConfigurationKeys.KAFKA_INPUT_TOPIC);
+    final String outputTopic = this.config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC);
+    final int windowDuration = this.config.getInt(ConfigurationKeys.KAFKA_WINDOW_DURATION_MINUTES);
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+
+    final KafkaConnectorFactory kafkaConnector = new KafkaConnectorFactory(
+        this.applicationId, kafkaBroker, checkpointing, schemaRegistryUrl);
+
+    final FlinkKafkaConsumer<ActivePowerRecord> kafkaSource =
+        kafkaConnector.createConsumer(inputTopic, ActivePowerRecord.class);
+
+    final FlinkKafkaProducer<Tuple2<String, String>> kafkaSink =
+        kafkaConnector.createProducer(outputTopic,
+            Serdes::String,
+            Serdes::String,
+            Types.TUPLE(Types.STRING, Types.STRING));
+
+    this.env
+        .addSource(kafkaSource).name("[Kafka Consumer] Topic: " + inputTopic)
+        .rebalance()
+        .keyBy(ActivePowerRecord::getIdentifier)
+        .window(TumblingEventTimeWindows.of(Time.minutes(windowDuration)))
+        .aggregate(new StatsAggregateFunction(), new StatsProcessWindowFunction())
+        .map(t -> {
+          final String key = t.f0;
+          final String value = t.f1.toString();
+          LOGGER.info("{}: {}", key, value);
+          return new Tuple2<>(key, value);
+        }).name("map").returns(Types.TUPLE(Types.STRING, Types.STRING))
+        .addSink(kafkaSink).name("[Kafka Producer] Topic: " + outputTopic);
+  }
+
+
+  /**
+   * Start running this microservice.
+   */
+  public void run() {
+    LOGGER.info("Execution plan: {}", this.env.getExecutionPlan());
+
+    try {
+      this.env.execute(this.applicationId);
+    } catch (final Exception e) { // NOPMD Execution thrown by Flink
+      LOGGER.error("An error occured while running this job.", e);
+    }
+  }
+
+  public static void main(final String[] args) {
+    new HistoryServiceFlinkJob().run();
+  }
+}
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsAggregateFunction.java b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsAggregateFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..7bd090de819ce0c0c73687bd53a191b66ae31ed9
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsAggregateFunction.java
@@ -0,0 +1,38 @@
+package theodolite.uc2.application;
+
+import com.google.common.math.Stats;
+import com.google.common.math.StatsAccumulator;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import theodolite.uc2.application.util.StatsFactory;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * Statistical aggregation of {@link ActivePowerRecord}s using {@link Stats}.
+ */
+public class StatsAggregateFunction implements AggregateFunction<ActivePowerRecord, Stats, Stats> {
+
+  private static final long serialVersionUID = -8873572990921515499L; // NOPMD
+
+  @Override
+  public Stats createAccumulator() {
+    return Stats.of();
+  }
+
+  @Override
+  public Stats add(final ActivePowerRecord value, final Stats accumulator) {
+    return StatsFactory.accumulate(accumulator, value.getValueInW());
+  }
+
+  @Override
+  public Stats getResult(final Stats accumulator) {
+    return accumulator;
+  }
+
+  @Override
+  public Stats merge(final Stats a, final Stats b) {
+    final StatsAccumulator statsAccumulator = new StatsAccumulator();
+    statsAccumulator.addAll(a);
+    statsAccumulator.addAll(b);
+    return statsAccumulator.snapshot();
+  }
+}
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsProcessWindowFunction.java b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsProcessWindowFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..d422c37b667d9d3309f0dd858758db29051807b9
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/StatsProcessWindowFunction.java
@@ -0,0 +1,24 @@
+package theodolite.uc2.application;
+
+import com.google.common.math.Stats;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.Collector;
+
+/**
+ * A {@link ProcessWindowFunction} that forwards a computed {@link Stats} object along with its
+ * associated key.
+ */
+public class StatsProcessWindowFunction
+    extends ProcessWindowFunction<Stats, Tuple2<String, Stats>, String, TimeWindow> {
+
+  private static final long serialVersionUID = 4363099880614593379L; // NOPMD
+
+  @Override
+  public void process(final String key, final Context context, final Iterable<Stats> elements,
+      final Collector<Tuple2<String, Stats>> out) {
+    final Stats stats = elements.iterator().next();
+    out.collect(new Tuple2<>(key, stats));
+  }
+}
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/util/StatsFactory.java b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/util/StatsFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..9697108eb8dacabf925f06067199a41eb0658dbe
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/java/theodolite/uc2/application/util/StatsFactory.java
@@ -0,0 +1,23 @@
+package theodolite.uc2.application.util;
+
+import com.google.common.math.Stats;
+import com.google.common.math.StatsAccumulator;
+
+/**
+ * Factory methods for working with {@link Stats}.
+ */
+public final class StatsFactory {
+
+  private StatsFactory() {}
+
+  /**
+   * Add a value to a {@link Stats} object.
+   */
+  public static Stats accumulate(final Stats stats, final double value) {
+    final StatsAccumulator statsAccumulator = new StatsAccumulator();
+    statsAccumulator.addAll(stats);
+    statsAccumulator.add(value);
+    return statsAccumulator.snapshot();
+  }
+
+}
diff --git a/theodolite-benchmarks/uc2-application-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc2-application-flink/src/main/resources/META-INF/application.properties
new file mode 100644
index 0000000000000000000000000000000000000000..f971390984ee41be1fce54e62f4f43ee2b9c02da
--- /dev/null
+++ b/theodolite-benchmarks/uc2-application-flink/src/main/resources/META-INF/application.properties
@@ -0,0 +1,11 @@
+application.name=theodolite-uc2-application
+application.version=0.0.1
+
+kafka.bootstrap.servers=localhost:9092
+kafka.input.topic=input
+kafka.output.topic=output
+schema.registry.url=http://localhost:8081
+num.threads=1
+commit.interval.ms=100
+cache.max.bytes.buffering=-1
+kafka.window.duration.minutes=1
\ No newline at end of file
diff --git a/theodolite-benchmarks/uc2-application/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc2-application/.settings/org.eclipse.jdt.ui.prefs
index fa98ca63d77bdee891150bd6713f70197a75cefc..4d01df75552c562406705858b6368ecf59d6e82f 100644
--- a/theodolite-benchmarks/uc2-application/.settings/org.eclipse.jdt.ui.prefs
+++ b/theodolite-benchmarks/uc2-application/.settings/org.eclipse.jdt.ui.prefs
@@ -66,6 +66,7 @@ org.eclipse.jdt.ui.ignorelowercasenames=true
 org.eclipse.jdt.ui.importorder=;
 org.eclipse.jdt.ui.ondemandthreshold=99
 org.eclipse.jdt.ui.staticondemandthreshold=99
+org.eclipse.jdt.ui.text.custom_code_templates=
 sp_cleanup.add_default_serial_version_id=true
 sp_cleanup.add_generated_serial_version_id=false
 sp_cleanup.add_missing_annotations=true
diff --git a/theodolite-benchmarks/uc3-application-flink/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc3-application-flink/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..4d01df75552c562406705858b6368ecf59d6e82f
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,128 @@
+cleanup.add_default_serial_version_id=true
+cleanup.add_generated_serial_version_id=false
+cleanup.add_missing_annotations=true
+cleanup.add_missing_deprecated_annotations=true
+cleanup.add_missing_methods=false
+cleanup.add_missing_nls_tags=false
+cleanup.add_missing_override_annotations=true
+cleanup.add_missing_override_annotations_interface_methods=true
+cleanup.add_serial_version_id=false
+cleanup.always_use_blocks=true
+cleanup.always_use_parentheses_in_expressions=false
+cleanup.always_use_this_for_non_static_field_access=true
+cleanup.always_use_this_for_non_static_method_access=true
+cleanup.convert_functional_interfaces=false
+cleanup.convert_to_enhanced_for_loop=true
+cleanup.correct_indentation=true
+cleanup.format_source_code=true
+cleanup.format_source_code_changes_only=false
+cleanup.insert_inferred_type_arguments=false
+cleanup.make_local_variable_final=true
+cleanup.make_parameters_final=true
+cleanup.make_private_fields_final=true
+cleanup.make_type_abstract_if_missing_method=false
+cleanup.make_variable_declarations_final=true
+cleanup.never_use_blocks=false
+cleanup.never_use_parentheses_in_expressions=true
+cleanup.organize_imports=true
+cleanup.qualify_static_field_accesses_with_declaring_class=false
+cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+cleanup.qualify_static_member_accesses_with_declaring_class=true
+cleanup.qualify_static_method_accesses_with_declaring_class=false
+cleanup.remove_private_constructors=true
+cleanup.remove_redundant_modifiers=false
+cleanup.remove_redundant_semicolons=true
+cleanup.remove_redundant_type_arguments=true
+cleanup.remove_trailing_whitespaces=true
+cleanup.remove_trailing_whitespaces_all=true
+cleanup.remove_trailing_whitespaces_ignore_empty=false
+cleanup.remove_unnecessary_casts=true
+cleanup.remove_unnecessary_nls_tags=true
+cleanup.remove_unused_imports=true
+cleanup.remove_unused_local_variables=false
+cleanup.remove_unused_private_fields=true
+cleanup.remove_unused_private_members=false
+cleanup.remove_unused_private_methods=true
+cleanup.remove_unused_private_types=true
+cleanup.sort_members=false
+cleanup.sort_members_all=false
+cleanup.use_anonymous_class_creation=false
+cleanup.use_blocks=true
+cleanup.use_blocks_only_for_return_and_throw=false
+cleanup.use_lambda=true
+cleanup.use_parentheses_in_expressions=true
+cleanup.use_this_for_non_static_field_access=true
+cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+cleanup.use_this_for_non_static_method_access=true
+cleanup.use_this_for_non_static_method_access_only_if_necessary=false
+cleanup_profile=_CAU-SE-Style
+cleanup_settings_version=2
+eclipse.preferences.version=1
+editor_save_participant_org.eclipse.jdt.ui.postsavelistener.cleanup=true
+formatter_profile=_CAU-SE-Style
+formatter_settings_version=15
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=99
+org.eclipse.jdt.ui.text.custom_code_templates=
+sp_cleanup.add_default_serial_version_id=true
+sp_cleanup.add_generated_serial_version_id=false
+sp_cleanup.add_missing_annotations=true
+sp_cleanup.add_missing_deprecated_annotations=true
+sp_cleanup.add_missing_methods=false
+sp_cleanup.add_missing_nls_tags=false
+sp_cleanup.add_missing_override_annotations=true
+sp_cleanup.add_missing_override_annotations_interface_methods=true
+sp_cleanup.add_serial_version_id=false
+sp_cleanup.always_use_blocks=true
+sp_cleanup.always_use_parentheses_in_expressions=false
+sp_cleanup.always_use_this_for_non_static_field_access=true
+sp_cleanup.always_use_this_for_non_static_method_access=true
+sp_cleanup.convert_functional_interfaces=false
+sp_cleanup.convert_to_enhanced_for_loop=true
+sp_cleanup.correct_indentation=true
+sp_cleanup.format_source_code=true
+sp_cleanup.format_source_code_changes_only=false
+sp_cleanup.insert_inferred_type_arguments=false
+sp_cleanup.make_local_variable_final=true
+sp_cleanup.make_parameters_final=true
+sp_cleanup.make_private_fields_final=true
+sp_cleanup.make_type_abstract_if_missing_method=false
+sp_cleanup.make_variable_declarations_final=true
+sp_cleanup.never_use_blocks=false
+sp_cleanup.never_use_parentheses_in_expressions=true
+sp_cleanup.on_save_use_additional_actions=true
+sp_cleanup.organize_imports=true
+sp_cleanup.qualify_static_field_accesses_with_declaring_class=false
+sp_cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_with_declaring_class=true
+sp_cleanup.qualify_static_method_accesses_with_declaring_class=false
+sp_cleanup.remove_private_constructors=true
+sp_cleanup.remove_redundant_modifiers=false
+sp_cleanup.remove_redundant_semicolons=true
+sp_cleanup.remove_redundant_type_arguments=true
+sp_cleanup.remove_trailing_whitespaces=true
+sp_cleanup.remove_trailing_whitespaces_all=true
+sp_cleanup.remove_trailing_whitespaces_ignore_empty=false
+sp_cleanup.remove_unnecessary_casts=true
+sp_cleanup.remove_unnecessary_nls_tags=true
+sp_cleanup.remove_unused_imports=true
+sp_cleanup.remove_unused_local_variables=false
+sp_cleanup.remove_unused_private_fields=true
+sp_cleanup.remove_unused_private_members=false
+sp_cleanup.remove_unused_private_methods=true
+sp_cleanup.remove_unused_private_types=true
+sp_cleanup.sort_members=false
+sp_cleanup.sort_members_all=false
+sp_cleanup.use_anonymous_class_creation=false
+sp_cleanup.use_blocks=true
+sp_cleanup.use_blocks_only_for_return_and_throw=false
+sp_cleanup.use_lambda=true
+sp_cleanup.use_parentheses_in_expressions=true
+sp_cleanup.use_this_for_non_static_field_access=true
+sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+sp_cleanup.use_this_for_non_static_method_access=true
+sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=false
diff --git a/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs b/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..87860c815222845c1d264d7d0ce498d3397f8280
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
@@ -0,0 +1,4 @@
+configFilePath=../config/checkstyle.xml
+customModulesJarPaths=
+eclipse.preferences.version=1
+enabled=true
diff --git a/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.pmd.prefs b/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..efbcb8c9e5d449194a48ca1ea42b7d807b573db9
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
@@ -0,0 +1,4 @@
+customRulesJars=
+eclipse.preferences.version=1
+enabled=true
+ruleSetFilePath=../config/pmd.xml
diff --git a/theodolite-benchmarks/uc3-application-flink/Dockerfile b/theodolite-benchmarks/uc3-application-flink/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..d582cd63fbc9cc5c5e540170bc7bc0aa2adc0ab1
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/Dockerfile
@@ -0,0 +1,3 @@
+FROM flink:1.12-scala_2.12-java11
+
+ADD build/libs/uc3-application-flink-all.jar /opt/flink/usrlib/artifacts/uc3-application-flink-all.jar
\ No newline at end of file
diff --git a/theodolite-benchmarks/uc3-application-flink/build.gradle b/theodolite-benchmarks/uc3-application-flink/build.gradle
new file mode 100644
index 0000000000000000000000000000000000000000..d50fa8efecd9b17e387d00c71934b8cc144240a1
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/build.gradle
@@ -0,0 +1,13 @@
+allprojects {
+	repositories {
+    	maven {
+    		url 'https://packages.confluent.io/maven/'
+    	}
+	}
+}
+
+dependencies {
+    compile('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT')
+}
+
+mainClassName = "theodolite.uc3.application.HistoryServiceFlinkJob"
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/ConfigurationKeys.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/ConfigurationKeys.java
new file mode 100644
index 0000000000000000000000000000000000000000..a895c74d89c5d788c47b3b78dc70500b4b5a6f5b
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/ConfigurationKeys.java
@@ -0,0 +1,39 @@
+package theodolite.uc3.application;
+
+/**
+ * Keys to access configuration parameters.
+ */
+public final class ConfigurationKeys {
+
+  public static final String APPLICATION_NAME = "application.name";
+
+  public static final String APPLICATION_VERSION = "application.version";
+
+  public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers";
+
+  public static final String KAFKA_INPUT_TOPIC = "kafka.input.topic";
+
+  public static final String KAFKA_OUTPUT_TOPIC = "kafka.output.topic";
+
+  public static final String SCHEMA_REGISTRY_URL = "schema.registry.url";
+
+  public static final String AGGREGATION_DURATION_DAYS = "aggregation.duration.days";
+
+  public static final String AGGREGATION_ADVANCE_DAYS = "aggregation.advance.days";
+
+  public static final String COMMIT_INTERVAL_MS = "commit.interval.ms";
+
+  public static final String TIME_ZONE = "time.zone";
+
+  public static final String FLINK_STATE_BACKEND = "flink.state.backend";
+
+  public static final String FLINK_STATE_BACKEND_PATH = "flink.state.backend.path";
+
+  public static final String FLINK_STATE_BACKEND_MEMORY_SIZE = // NOPMD
+      "flink.state.backend.memory.size";
+
+  public static final String CHECKPOINTING = "checkpointing";
+
+  private ConfigurationKeys() {}
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HistoryServiceFlinkJob.java
new file mode 100644
index 0000000000000000000000000000000000000000..0f26d37652924a16be1840fd759b3cd5b023f338
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HistoryServiceFlinkJob.java
@@ -0,0 +1,153 @@
+package theodolite.uc3.application;
+
+import com.google.common.math.Stats;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import org.apache.commons.configuration2.Configuration;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import theodolite.commons.flink.KafkaConnectorFactory;
+import theodolite.commons.flink.StateBackends;
+import theodolite.commons.flink.serialization.StatsSerializer;
+import theodolite.uc3.application.util.HourOfDayKey;
+import theodolite.uc3.application.util.HourOfDayKeyFactory;
+import theodolite.uc3.application.util.HourOfDayKeySerde;
+import theodolite.uc3.application.util.StatsKeyFactory;
+import titan.ccp.common.configuration.ServiceConfigurations;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * The History microservice implemented as a Flink job.
+ */
+public final class HistoryServiceFlinkJob {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(HistoryServiceFlinkJob.class);
+
+  private final Configuration config = ServiceConfigurations.createWithDefaults();
+  private final StreamExecutionEnvironment env;
+  private final String applicationId;
+
+  /**
+   * Create a new instance of the {@link HistoryServiceFlinkJob}.
+   */
+  public HistoryServiceFlinkJob() {
+    final String applicationName = this.config.getString(ConfigurationKeys.APPLICATION_NAME);
+    final String applicationVersion = this.config.getString(ConfigurationKeys.APPLICATION_VERSION);
+    this.applicationId = applicationName + "-" + applicationVersion;
+
+    this.env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    this.configureEnv();
+
+    this.buildPipeline();
+  }
+
+  private void configureEnv() {
+    this.env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+    final int commitIntervalMs = this.config.getInt(ConfigurationKeys.COMMIT_INTERVAL_MS);
+    if (checkpointing) {
+      this.env.enableCheckpointing(commitIntervalMs);
+    }
+
+    // State Backend
+    final StateBackend stateBackend = StateBackends.fromConfiguration(this.config);
+    this.env.setStateBackend(stateBackend);
+
+    this.configureSerializers();
+  }
+
+  private void configureSerializers() {
+    this.env.getConfig().registerTypeWithKryoSerializer(HourOfDayKey.class,
+        new HourOfDayKeySerde());
+    this.env.getConfig().registerTypeWithKryoSerializer(Stats.class, new StatsSerializer());
+    for (final var entry : this.env.getConfig().getRegisteredTypesWithKryoSerializers()
+        .entrySet()) {
+      LOGGER.info("Class {} registered with serializer {}.",
+          entry.getKey().getName(),
+          entry.getValue().getSerializer().getClass().getName());
+    }
+  }
+
+  private void buildPipeline() {
+    // Configurations
+    final String kafkaBroker = this.config.getString(ConfigurationKeys.KAFKA_BOOTSTRAP_SERVERS);
+    final String schemaRegistryUrl = this.config.getString(ConfigurationKeys.SCHEMA_REGISTRY_URL);
+    final String inputTopic = this.config.getString(ConfigurationKeys.KAFKA_INPUT_TOPIC);
+    final String outputTopic = this.config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC);
+    final ZoneId timeZone = ZoneId.of(this.config.getString(ConfigurationKeys.TIME_ZONE));
+    final Time aggregationDuration =
+        Time.days(this.config.getInt(ConfigurationKeys.AGGREGATION_DURATION_DAYS));
+    final Time aggregationAdvance =
+        Time.days(this.config.getInt(ConfigurationKeys.AGGREGATION_ADVANCE_DAYS));
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+
+    final KafkaConnectorFactory kafkaConnector = new KafkaConnectorFactory(
+        this.applicationId, kafkaBroker, checkpointing, schemaRegistryUrl);
+
+    // Sources and Sinks
+    final FlinkKafkaConsumer<ActivePowerRecord> kafkaSource =
+        kafkaConnector.createConsumer(inputTopic, ActivePowerRecord.class);
+    final FlinkKafkaProducer<Tuple2<String, String>> kafkaSink =
+        kafkaConnector.createProducer(outputTopic,
+            Serdes::String,
+            Serdes::String,
+            Types.TUPLE(Types.STRING, Types.STRING));
+
+    // Streaming topology
+    final StatsKeyFactory<HourOfDayKey> keyFactory = new HourOfDayKeyFactory();
+    this.env
+        .addSource(kafkaSource)
+        .name("[Kafka Consumer] Topic: " + inputTopic)
+        .rebalance()
+        .keyBy((KeySelector<ActivePowerRecord, HourOfDayKey>) record -> {
+          final Instant instant = Instant.ofEpochMilli(record.getTimestamp());
+          final LocalDateTime dateTime = LocalDateTime.ofInstant(instant, timeZone);
+          return keyFactory.createKey(record.getIdentifier(), dateTime);
+        })
+        .window(SlidingEventTimeWindows.of(aggregationDuration, aggregationAdvance))
+        .aggregate(new StatsAggregateFunction(), new HourOfDayProcessWindowFunction())
+        .map(tuple -> {
+          final String newKey = keyFactory.getSensorId(tuple.f0);
+          final String newValue = tuple.f1.toString();
+          final int hourOfDay = tuple.f0.getHourOfDay();
+          LOGGER.info("{}|{}: {}", newKey, hourOfDay, newValue);
+          return new Tuple2<>(newKey, newValue);
+        })
+        .name("map")
+        .returns(Types.TUPLE(Types.STRING, Types.STRING))
+        .addSink(kafkaSink).name("[Kafka Producer] Topic: " + outputTopic);
+  }
+
+  /**
+   * Start running this microservice.
+   */
+  public void run() {
+    // Execution plan
+    LOGGER.info("Execution Plan: {}", this.env.getExecutionPlan());
+
+    // Execute Job
+    try {
+      this.env.execute(this.applicationId);
+    } catch (final Exception e) { // NOPMD Execution thrown by Flink
+      LOGGER.error("An error occured while running this job.", e);
+    }
+  }
+
+  public static void main(final String[] args) {
+    new HistoryServiceFlinkJob().run();
+  }
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HourOfDayProcessWindowFunction.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HourOfDayProcessWindowFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..349c63413d0da792ad34e8ec8d94e7ff5dc06a42
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/HourOfDayProcessWindowFunction.java
@@ -0,0 +1,28 @@
+package theodolite.uc3.application;
+
+import com.google.common.math.Stats;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.Collector;
+import theodolite.uc3.application.util.HourOfDayKey;
+
+/**
+ * A {@link ProcessWindowFunction} that forwards a computed {@link Stats} object along with its
+ * associated key.
+ */
+public class HourOfDayProcessWindowFunction
+    extends ProcessWindowFunction<Stats, Tuple2<HourOfDayKey, Stats>, HourOfDayKey, TimeWindow> {
+
+  private static final long serialVersionUID = 7702216563302727315L; // NOPMD
+
+  @Override
+  public void process(final HourOfDayKey hourOfDayKey,
+      final Context context,
+      final Iterable<Stats> elements,
+      final Collector<Tuple2<HourOfDayKey, Stats>> out) {
+    final Stats stats = elements.iterator().next();
+    out.collect(new Tuple2<>(hourOfDayKey, stats));
+  }
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/StatsAggregateFunction.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/StatsAggregateFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..4706da0a9491e0391f25cd61639c3bb565509cb1
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/StatsAggregateFunction.java
@@ -0,0 +1,38 @@
+package theodolite.uc3.application;
+
+import com.google.common.math.Stats;
+import com.google.common.math.StatsAccumulator;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import theodolite.uc3.application.util.StatsFactory;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * Statistical aggregation of {@link ActivePowerRecord}s using {@link Stats}.
+ */
+public class StatsAggregateFunction implements AggregateFunction<ActivePowerRecord, Stats, Stats> {
+
+  private static final long serialVersionUID = -8873572990921515499L; // NOPMD
+
+  @Override
+  public Stats createAccumulator() {
+    return Stats.of();
+  }
+
+  @Override
+  public Stats add(final ActivePowerRecord value, final Stats accumulator) {
+    return StatsFactory.accumulate(accumulator, value.getValueInW());
+  }
+
+  @Override
+  public Stats getResult(final Stats accumulator) {
+    return accumulator;
+  }
+
+  @Override
+  public Stats merge(final Stats a, final Stats b) {
+    final StatsAccumulator statsAccumulator = new StatsAccumulator();
+    statsAccumulator.addAll(a);
+    statsAccumulator.addAll(b);
+    return statsAccumulator.snapshot();
+  }
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKey.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKey.java
new file mode 100644
index 0000000000000000000000000000000000000000..5def88b404f23a59955ca2de42b91c22b7b1b53d
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKey.java
@@ -0,0 +1,79 @@
+package theodolite.uc3.application.util;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Objects;
+
+/**
+ * Composed key of an hour of the day and a sensor id.
+ */
+public class HourOfDayKey {
+
+  private final int hourOfDay;
+  private final String sensorId;
+
+  public HourOfDayKey(final int hourOfDay, final String sensorId) {
+    this.hourOfDay = hourOfDay;
+    this.sensorId = sensorId;
+  }
+
+  public int getHourOfDay() {
+    return this.hourOfDay;
+  }
+
+  public String getSensorId() {
+    return this.sensorId;
+  }
+
+  @Override
+  public String toString() {
+    return this.sensorId + ";" + this.hourOfDay;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.hourOfDay, this.sensorId);
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (!(obj instanceof HourOfDayKey)) {
+      return false;
+    }
+    final HourOfDayKey k = (HourOfDayKey) obj;
+    return this.hourOfDay == k.hourOfDay && this.sensorId.equals(k.sensorId);
+  }
+
+  /**
+   * Convert this {@link HourOfDayKey} into a byte array. This method is the inverse to
+   * {@code HourOfDayKey#fromByteArray()}.
+   */
+  public byte[] toByteArray() {
+    final int numBytes = (2 * Integer.SIZE + this.sensorId.length() * Character.SIZE) / Byte.SIZE;
+    final ByteBuffer buffer = ByteBuffer.allocate(numBytes).order(ByteOrder.LITTLE_ENDIAN);
+    buffer.putInt(this.hourOfDay);
+    buffer.putInt(this.sensorId.length());
+    for (final char c : this.sensorId.toCharArray()) {
+      buffer.putChar(c);
+    }
+    return buffer.array();
+  }
+
+  /**
+   * Construct a new {@link HourOfDayKey} from a byte array. This method is the inverse to
+   * {@code HourOfDayKey#toByteArray()}.
+   */
+  public static HourOfDayKey fromByteArray(final byte[] bytes) {
+    final ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+    final int hourOfDay = buffer.getInt();
+    final int strLen = buffer.getInt();
+    final char[] sensorId = new char[strLen];
+    for (int i = 0; i < strLen; i++) {
+      sensorId[i] = buffer.getChar();
+    }
+    return new HourOfDayKey(hourOfDay, new String(sensorId));
+  }
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeyFactory.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeyFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..bd67b2508bc91a87635c52e95b963ed908ed92bf
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeyFactory.java
@@ -0,0 +1,24 @@
+package theodolite.uc3.application.util;
+
+import java.io.Serializable;
+import java.time.LocalDateTime;
+
+/**
+ * {@link StatsKeyFactory} for {@link HourOfDayKey}.
+ */
+public class HourOfDayKeyFactory implements StatsKeyFactory<HourOfDayKey>, Serializable {
+
+  private static final long serialVersionUID = 4357668496473645043L; // NOPMD
+
+  @Override
+  public HourOfDayKey createKey(final String sensorId, final LocalDateTime dateTime) {
+    final int hourOfDay = dateTime.getHour();
+    return new HourOfDayKey(hourOfDay, sensorId);
+  }
+
+  @Override
+  public String getSensorId(final HourOfDayKey key) {
+    return key.getSensorId();
+  }
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeySerde.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeySerde.java
new file mode 100644
index 0000000000000000000000000000000000000000..6e3ae9f754d2b1d4ab10349040f0c9e51134c4f7
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayKeySerde.java
@@ -0,0 +1,52 @@
+package theodolite.uc3.application.util;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.io.Serializable;
+import org.apache.kafka.common.serialization.Serde;
+import titan.ccp.common.kafka.simpleserdes.BufferSerde;
+import titan.ccp.common.kafka.simpleserdes.ReadBuffer;
+import titan.ccp.common.kafka.simpleserdes.SimpleSerdes;
+import titan.ccp.common.kafka.simpleserdes.WriteBuffer;
+
+/**
+ * {@link BufferSerde} for a {@link HourOfDayKey}. Use the {@link #create()} method to create a new
+ * Kafka {@link Serde}.
+ */
+public class HourOfDayKeySerde extends Serializer<HourOfDayKey>
+    implements BufferSerde<HourOfDayKey>, Serializable {
+
+  private static final long serialVersionUID = 1262778284661945041L; // NOPMD
+
+  @Override
+  public void serialize(final WriteBuffer buffer, final HourOfDayKey data) {
+    buffer.putInt(data.getHourOfDay());
+    buffer.putString(data.getSensorId());
+  }
+
+  @Override
+  public HourOfDayKey deserialize(final ReadBuffer buffer) {
+    final int hourOfDay = buffer.getInt();
+    final String sensorId = buffer.getString();
+    return new HourOfDayKey(hourOfDay, sensorId);
+  }
+
+  public static Serde<HourOfDayKey> create() {
+    return SimpleSerdes.create(new HourOfDayKeySerde());
+  }
+
+  @Override
+  public void write(final Kryo kryo, final Output output, final HourOfDayKey object) {
+    final byte[] data = object.toByteArray();
+    output.writeInt(data.length);
+    output.writeBytes(data);
+  }
+
+  @Override
+  public HourOfDayKey read(final Kryo kryo, final Input input, final Class<HourOfDayKey> type) {
+    final int numBytes = input.readInt();
+    return HourOfDayKey.fromByteArray(input.readBytes(numBytes));
+  }
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayRecordFactory.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayRecordFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..d8a42b74e5ca1cc55f9f21de62a5d8f877223e62
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/HourOfDayRecordFactory.java
@@ -0,0 +1,28 @@
+package theodolite.uc3.application.util;
+
+import com.google.common.math.Stats;
+import org.apache.kafka.streams.kstream.Windowed;
+import titan.ccp.model.records.HourOfDayActivePowerRecord;
+
+/**
+ * {@link StatsRecordFactory} to create an {@link HourOfDayActivePowerRecord}.
+ */
+public class HourOfDayRecordFactory
+    implements StatsRecordFactory<HourOfDayKey, HourOfDayActivePowerRecord> {
+
+  @Override
+  public HourOfDayActivePowerRecord create(final Windowed<HourOfDayKey> windowed,
+      final Stats stats) {
+    return new HourOfDayActivePowerRecord(
+        windowed.key().getSensorId(),
+        windowed.key().getHourOfDay(),
+        windowed.window().start(),
+        windowed.window().end(),
+        stats.count(),
+        stats.mean(),
+        stats.populationVariance(),
+        stats.min(),
+        stats.max());
+  }
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsFactory.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..b7880be4eb48035959251cc56273d16407bcb888
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsFactory.java
@@ -0,0 +1,23 @@
+package theodolite.uc3.application.util;
+
+import com.google.common.math.Stats;
+import com.google.common.math.StatsAccumulator;
+
+/**
+ * Factory methods for working with {@link Stats}.
+ */
+public final class StatsFactory {
+
+  private StatsFactory() {}
+
+  /**
+   * Add a value to a {@link Stats} object.
+   */
+  public static Stats accumulate(final Stats stats, final double value) {
+    final StatsAccumulator statsAccumulator = new StatsAccumulator();
+    statsAccumulator.addAll(stats);
+    statsAccumulator.add(value);
+    return statsAccumulator.snapshot();
+  }
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsKeyFactory.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsKeyFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..fdebccaa2d116253c41492cab3443057adef7b36
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsKeyFactory.java
@@ -0,0 +1,17 @@
+package theodolite.uc3.application.util;
+
+import java.time.LocalDateTime;
+
+/**
+ * Factory interface for creating a stats key from a sensor id and a {@link LocalDateTime} object
+ * and vice versa.
+ *
+ * @param <T> Type of the key
+ */
+public interface StatsKeyFactory<T> {
+
+  T createKey(String sensorId, LocalDateTime dateTime);
+
+  String getSensorId(T key);
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsRecordFactory.java b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsRecordFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..61333c99966b1ffea608d225f17d8460eac9ada1
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/java/theodolite/uc3/application/util/StatsRecordFactory.java
@@ -0,0 +1,22 @@
+package theodolite.uc3.application.util;
+
+import com.google.common.math.Stats;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+/**
+ * Factory interface for creating a stats Avro record from a {@link Windowed} and a {@link Stats}.
+ * The {@link Windowed} contains about information about the start end end of the {@link Window} as
+ * well as the sensor id and the aggregated time unit. The {@link Stats} objects contains the actual
+ * aggregation results.
+ *
+ * @param <K> Key type of the {@link Windowed}
+ * @param <R> Avro record type
+ */
+@FunctionalInterface
+public interface StatsRecordFactory<K, R extends SpecificRecord> {
+
+  R create(Windowed<K> windowed, Stats stats);
+
+}
diff --git a/theodolite-benchmarks/uc3-application-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc3-application-flink/src/main/resources/META-INF/application.properties
new file mode 100644
index 0000000000000000000000000000000000000000..6b6874674ce6a0abea73ea6d983c00c15deb8bb1
--- /dev/null
+++ b/theodolite-benchmarks/uc3-application-flink/src/main/resources/META-INF/application.properties
@@ -0,0 +1,13 @@
+application.name=theodolite-uc3-application
+application.version=0.0.1
+
+kafka.bootstrap.servers=localhost:9092
+kafka.input.topic=input
+kafka.output.topic=output
+schema.registry.url=http://localhost:8081
+aggregation.duration.days=30
+aggregation.advance.days=1
+num.threads=1
+commit.interval.ms=100
+cache.max.bytes.buffering=-1
+time.zone=Europe/Paris
\ No newline at end of file
diff --git a/theodolite-benchmarks/uc4-application-flink/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc4-application-flink/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..272e01533f6a345d53d2635c47e38c6d3c33dc8a
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,128 @@
+cleanup.add_default_serial_version_id=true
+cleanup.add_generated_serial_version_id=false
+cleanup.add_missing_annotations=true
+cleanup.add_missing_deprecated_annotations=true
+cleanup.add_missing_methods=false
+cleanup.add_missing_nls_tags=false
+cleanup.add_missing_override_annotations=true
+cleanup.add_missing_override_annotations_interface_methods=true
+cleanup.add_serial_version_id=false
+cleanup.always_use_blocks=true
+cleanup.always_use_parentheses_in_expressions=false
+cleanup.always_use_this_for_non_static_field_access=true
+cleanup.always_use_this_for_non_static_method_access=true
+cleanup.convert_functional_interfaces=false
+cleanup.convert_to_enhanced_for_loop=true
+cleanup.correct_indentation=true
+cleanup.format_source_code=true
+cleanup.format_source_code_changes_only=false
+cleanup.insert_inferred_type_arguments=false
+cleanup.make_local_variable_final=true
+cleanup.make_parameters_final=true
+cleanup.make_private_fields_final=true
+cleanup.make_type_abstract_if_missing_method=false
+cleanup.make_variable_declarations_final=true
+cleanup.never_use_blocks=false
+cleanup.never_use_parentheses_in_expressions=true
+cleanup.organize_imports=true
+cleanup.qualify_static_field_accesses_with_declaring_class=false
+cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+cleanup.qualify_static_member_accesses_with_declaring_class=true
+cleanup.qualify_static_method_accesses_with_declaring_class=false
+cleanup.remove_private_constructors=true
+cleanup.remove_redundant_modifiers=false
+cleanup.remove_redundant_semicolons=true
+cleanup.remove_redundant_type_arguments=true
+cleanup.remove_trailing_whitespaces=true
+cleanup.remove_trailing_whitespaces_all=true
+cleanup.remove_trailing_whitespaces_ignore_empty=false
+cleanup.remove_unnecessary_casts=true
+cleanup.remove_unnecessary_nls_tags=true
+cleanup.remove_unused_imports=true
+cleanup.remove_unused_local_variables=false
+cleanup.remove_unused_private_fields=true
+cleanup.remove_unused_private_members=false
+cleanup.remove_unused_private_methods=true
+cleanup.remove_unused_private_types=true
+cleanup.sort_members=false
+cleanup.sort_members_all=false
+cleanup.use_anonymous_class_creation=false
+cleanup.use_blocks=true
+cleanup.use_blocks_only_for_return_and_throw=false
+cleanup.use_lambda=true
+cleanup.use_parentheses_in_expressions=true
+cleanup.use_this_for_non_static_field_access=true
+cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+cleanup.use_this_for_non_static_method_access=true
+cleanup.use_this_for_non_static_method_access_only_if_necessary=false
+cleanup_profile=_CAU-SE-Style
+cleanup_settings_version=2
+eclipse.preferences.version=1
+editor_save_participant_org.eclipse.jdt.ui.postsavelistener.cleanup=true
+formatter_profile=_CAU-SE-Style
+formatter_settings_version=15
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=99
+org.eclipse.jdt.ui.text.custom_code_templates=
+sp_cleanup.add_default_serial_version_id=true
+sp_cleanup.add_generated_serial_version_id=false
+sp_cleanup.add_missing_annotations=true
+sp_cleanup.add_missing_deprecated_annotations=true
+sp_cleanup.add_missing_methods=false
+sp_cleanup.add_missing_nls_tags=false
+sp_cleanup.add_missing_override_annotations=true
+sp_cleanup.add_missing_override_annotations_interface_methods=true
+sp_cleanup.add_serial_version_id=false
+sp_cleanup.always_use_blocks=true
+sp_cleanup.always_use_parentheses_in_expressions=false
+sp_cleanup.always_use_this_for_non_static_field_access=true
+sp_cleanup.always_use_this_for_non_static_method_access=true
+sp_cleanup.convert_functional_interfaces=false
+sp_cleanup.convert_to_enhanced_for_loop=true
+sp_cleanup.correct_indentation=true
+sp_cleanup.format_source_code=true
+sp_cleanup.format_source_code_changes_only=false
+sp_cleanup.insert_inferred_type_arguments=false
+sp_cleanup.make_local_variable_final=true
+sp_cleanup.make_parameters_final=true
+sp_cleanup.make_private_fields_final=true
+sp_cleanup.make_type_abstract_if_missing_method=false
+sp_cleanup.make_variable_declarations_final=true
+sp_cleanup.never_use_blocks=false
+sp_cleanup.never_use_parentheses_in_expressions=true
+sp_cleanup.on_save_use_additional_actions=true
+sp_cleanup.organize_imports=true
+sp_cleanup.qualify_static_field_accesses_with_declaring_class=false
+sp_cleanup.qualify_static_member_accesses_through_instances_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class=true
+sp_cleanup.qualify_static_member_accesses_with_declaring_class=true
+sp_cleanup.qualify_static_method_accesses_with_declaring_class=false
+sp_cleanup.remove_private_constructors=true
+sp_cleanup.remove_redundant_modifiers=true
+sp_cleanup.remove_redundant_semicolons=true
+sp_cleanup.remove_redundant_type_arguments=true
+sp_cleanup.remove_trailing_whitespaces=true
+sp_cleanup.remove_trailing_whitespaces_all=true
+sp_cleanup.remove_trailing_whitespaces_ignore_empty=false
+sp_cleanup.remove_unnecessary_casts=true
+sp_cleanup.remove_unnecessary_nls_tags=true
+sp_cleanup.remove_unused_imports=true
+sp_cleanup.remove_unused_local_variables=false
+sp_cleanup.remove_unused_private_fields=true
+sp_cleanup.remove_unused_private_members=false
+sp_cleanup.remove_unused_private_methods=true
+sp_cleanup.remove_unused_private_types=true
+sp_cleanup.sort_members=false
+sp_cleanup.sort_members_all=false
+sp_cleanup.use_anonymous_class_creation=false
+sp_cleanup.use_blocks=true
+sp_cleanup.use_blocks_only_for_return_and_throw=false
+sp_cleanup.use_lambda=true
+sp_cleanup.use_parentheses_in_expressions=true
+sp_cleanup.use_this_for_non_static_field_access=true
+sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=false
+sp_cleanup.use_this_for_non_static_method_access=true
+sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=false
diff --git a/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs b/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..87860c815222845c1d264d7d0ce498d3397f8280
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.checkstyle.prefs
@@ -0,0 +1,4 @@
+configFilePath=../config/checkstyle.xml
+customModulesJarPaths=
+eclipse.preferences.version=1
+enabled=true
diff --git a/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.pmd.prefs b/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
new file mode 100644
index 0000000000000000000000000000000000000000..efbcb8c9e5d449194a48ca1ea42b7d807b573db9
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/.settings/qa.eclipse.plugin.pmd.prefs
@@ -0,0 +1,4 @@
+customRulesJars=
+eclipse.preferences.version=1
+enabled=true
+ruleSetFilePath=../config/pmd.xml
diff --git a/theodolite-benchmarks/uc4-application-flink/Dockerfile b/theodolite-benchmarks/uc4-application-flink/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..49521e7d1e96a7e7a295a1557e205c0e7d4316db
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/Dockerfile
@@ -0,0 +1,3 @@
+FROM flink:1.12-scala_2.12-java11
+
+ADD build/libs/uc4-application-flink-all.jar /opt/flink/usrlib/artifacts/uc4-application-flink-all.jar
\ No newline at end of file
diff --git a/theodolite-benchmarks/uc4-application-flink/build.gradle b/theodolite-benchmarks/uc4-application-flink/build.gradle
new file mode 100644
index 0000000000000000000000000000000000000000..0ad804c62566aff81d05f71a874f52c09be4ebcb
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/build.gradle
@@ -0,0 +1 @@
+mainClassName = "theodolite.uc4.application.AggregationServiceFlinkJob"
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/AggregationServiceFlinkJob.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/AggregationServiceFlinkJob.java
new file mode 100644
index 0000000000000000000000000000000000000000..0db5a3d524f74fbf22304e8f9b44fa55eead321a
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/AggregationServiceFlinkJob.java
@@ -0,0 +1,217 @@
+package theodolite.uc4.application; // NOPMD Imports required
+
+import java.time.Duration;
+import java.util.Set;
+import org.apache.commons.configuration2.Configuration;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import theodolite.commons.flink.KafkaConnectorFactory;
+import theodolite.commons.flink.StateBackends;
+import theodolite.commons.flink.TupleType;
+import theodolite.uc4.application.util.ImmutableSensorRegistrySerializer;
+import theodolite.uc4.application.util.ImmutableSetSerializer;
+import theodolite.uc4.application.util.SensorParentKey;
+import theodolite.uc4.application.util.SensorParentKeySerializer;
+import titan.ccp.common.configuration.ServiceConfigurations;
+import titan.ccp.common.kafka.avro.SchemaRegistryAvroSerdeFactory;
+import titan.ccp.configuration.events.Event;
+import titan.ccp.configuration.events.EventSerde;
+import titan.ccp.model.records.ActivePowerRecord;
+import titan.ccp.model.records.AggregatedActivePowerRecord;
+import titan.ccp.model.sensorregistry.ImmutableSensorRegistry;
+import titan.ccp.model.sensorregistry.SensorRegistry;
+
+/**
+ * The Aggregation microservice implemented as a Flink job.
+ */
+public final class AggregationServiceFlinkJob {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(AggregationServiceFlinkJob.class);
+
+  private final Configuration config = ServiceConfigurations.createWithDefaults();
+  private final StreamExecutionEnvironment env;
+  private final String applicationId;
+
+  /**
+   * Create a new {@link AggregationServiceFlinkJob}.
+   */
+  public AggregationServiceFlinkJob() {
+    final String applicationName = this.config.getString(ConfigurationKeys.APPLICATION_NAME);
+    final String applicationVersion = this.config.getString(ConfigurationKeys.APPLICATION_VERSION);
+    this.applicationId = applicationName + "-" + applicationVersion;
+
+    // Execution environment configuration
+    // org.apache.flink.configuration.Configuration conf = new
+    // org.apache.flink.configuration.Configuration();
+    // conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true);
+    // final StreamExecutionEnvironment env =
+    // StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(conf);
+    this.env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    this.configureEnv();
+
+    this.buildPipeline();
+  }
+
+  private void configureEnv() {
+    this.env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+    final int commitIntervalMs = this.config.getInt(ConfigurationKeys.COMMIT_INTERVAL_MS);
+    if (checkpointing) {
+      this.env.enableCheckpointing(commitIntervalMs);
+    }
+
+    // State Backend
+    final StateBackend stateBackend = StateBackends.fromConfiguration(this.config);
+    this.env.setStateBackend(stateBackend);
+
+    this.configureSerializers();
+  }
+
+  private void configureSerializers() {
+    this.env.getConfig().registerTypeWithKryoSerializer(ImmutableSensorRegistry.class,
+        new ImmutableSensorRegistrySerializer());
+    this.env.getConfig().registerTypeWithKryoSerializer(SensorParentKey.class,
+        new SensorParentKeySerializer());
+
+    this.env.getConfig().registerTypeWithKryoSerializer(Set.of().getClass(),
+        new ImmutableSetSerializer());
+    this.env.getConfig().registerTypeWithKryoSerializer(Set.of(1).getClass(),
+        new ImmutableSetSerializer());
+    this.env.getConfig().registerTypeWithKryoSerializer(Set.of(1, 2, 3, 4).getClass(), // NOCS
+        new ImmutableSetSerializer());
+
+    this.env.getConfig().getRegisteredTypesWithKryoSerializers()
+        .forEach((c, s) -> LOGGER.info("Class " + c.getName() + " registered with serializer "
+            + s.getSerializer().getClass().getName()));
+  }
+
+  private void buildPipeline() {
+    // Get configurations
+    final String kafkaBroker = this.config.getString(ConfigurationKeys.KAFKA_BOOTSTRAP_SERVERS);
+    final String schemaRegistryUrl = this.config.getString(ConfigurationKeys.SCHEMA_REGISTRY_URL);
+    final String inputTopic = this.config.getString(ConfigurationKeys.KAFKA_INPUT_TOPIC);
+    final String outputTopic = this.config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC);
+    final Time windowSize =
+        Time.milliseconds(this.config.getLong(ConfigurationKeys.WINDOW_SIZE_MS));
+    final Duration windowGrace =
+        Duration.ofMillis(this.config.getLong(ConfigurationKeys.WINDOW_GRACE_MS));
+    final String configurationTopic =
+        this.config.getString(ConfigurationKeys.CONFIGURATION_KAFKA_TOPIC);
+    final boolean checkpointing = this.config.getBoolean(ConfigurationKeys.CHECKPOINTING, true);
+
+    final KafkaConnectorFactory kafkaConnector = new KafkaConnectorFactory(
+        this.applicationId, kafkaBroker, checkpointing, schemaRegistryUrl);
+
+    // Source from input topic with ActivePowerRecords
+    final FlinkKafkaConsumer<ActivePowerRecord> kafkaInputSource =
+        kafkaConnector.createConsumer(inputTopic, ActivePowerRecord.class);
+    // TODO Watermarks?
+
+    // Source from output topic with AggregatedPowerRecords
+    final FlinkKafkaConsumer<AggregatedActivePowerRecord> kafkaOutputSource =
+        kafkaConnector.createConsumer(outputTopic, AggregatedActivePowerRecord.class);
+
+    final FlinkKafkaConsumer<Tuple2<Event, String>> kafkaConfigSource =
+        kafkaConnector.createConsumer(
+            configurationTopic,
+            EventSerde::serde,
+            Serdes::String,
+            TupleType.of(TypeInformation.of(Event.class), Types.STRING));
+
+    // Sink to output topic with SensorId, AggregatedActivePowerRecord
+    final FlinkKafkaProducer<Tuple2<String, AggregatedActivePowerRecord>> kafkaAggregationSink =
+        kafkaConnector.createProducer(
+            outputTopic,
+            Serdes::String,
+            () -> new SchemaRegistryAvroSerdeFactory(schemaRegistryUrl).forValues(),
+            Types.TUPLE(Types.STRING, TypeInformation.of(AggregatedActivePowerRecord.class)));
+
+    // Build input stream
+    final DataStream<ActivePowerRecord> inputStream = this.env.addSource(kafkaInputSource)
+        .name("[Kafka Consumer] Topic: " + inputTopic)// NOCS
+        .rebalance()
+        .map(r -> r)
+        .name("[Map] Rebalance Forward");
+
+    // Build aggregation stream
+    final DataStream<ActivePowerRecord> aggregationsInputStream =
+        this.env.addSource(kafkaOutputSource)
+            .name("[Kafka Consumer] Topic: " + outputTopic) // NOCS
+            .rebalance()
+            .map(r -> new ActivePowerRecord(r.getIdentifier(), r.getTimestamp(), r.getSumInW()))
+            .name("[Map] AggregatedActivePowerRecord -> ActivePowerRecord");
+
+    // Merge input and aggregation streams
+    final DataStream<ActivePowerRecord> mergedInputStream = inputStream
+        .union(aggregationsInputStream);
+
+    // Build parent sensor stream from configuration stream
+    final DataStream<Tuple2<String, Set<String>>> configurationsStream =
+        this.env.addSource(kafkaConfigSource)
+            .name("[Kafka Consumer] Topic: " + configurationTopic) // NOCS
+            .filter(tuple -> tuple.f0 == Event.SENSOR_REGISTRY_CHANGED
+                || tuple.f0 == Event.SENSOR_REGISTRY_STATUS)
+            .name("[Filter] SensorRegistry changed")
+            .map(tuple -> SensorRegistry.fromJson(tuple.f1)).name("[Map] JSON -> SensorRegistry")
+            .keyBy(sr -> 1)
+            .flatMap(new ChildParentsFlatMapFunction())
+            .name("[FlatMap] SensorRegistry -> (ChildSensor, ParentSensor[])");
+
+    final DataStream<Tuple2<SensorParentKey, ActivePowerRecord>> lastValueStream =
+        mergedInputStream.connect(configurationsStream)
+            .keyBy(ActivePowerRecord::getIdentifier,
+                (KeySelector<Tuple2<String, Set<String>>, String>) t -> t.f0)
+            .flatMap(new JoinAndDuplicateCoFlatMapFunction())
+            .name("[CoFlatMap] Join input-config, Flatten to ((Sensor, Group), ActivePowerRecord)");
+
+    final DataStream<AggregatedActivePowerRecord> aggregationStream = lastValueStream
+        .rebalance()
+        .assignTimestampsAndWatermarks(WatermarkStrategy.forBoundedOutOfOrderness(windowGrace))
+        .keyBy(t -> t.f0.getParent())
+        .window(TumblingEventTimeWindows.of(windowSize))
+        .process(new RecordAggregationProcessWindowFunction())
+        .name("[Aggregate] ((Sensor, Group), ActivePowerRecord) -> AggregatedActivePowerRecord");
+
+    // add Kafka Sink
+    aggregationStream
+        .map(value -> new Tuple2<>(value.getIdentifier(), value))
+        .name("[Map] AggregatedActivePowerRecord -> (Sensor, AggregatedActivePowerRecord)")
+        .returns(Types.TUPLE(Types.STRING, TypeInformation.of(AggregatedActivePowerRecord.class)))
+        .addSink(kafkaAggregationSink).name("[Kafka Producer] Topic: " + outputTopic);
+  }
+
+  /**
+   * Start running this microservice.
+   */
+  public void run() {
+    // Execution plan
+    LOGGER.info("Execution plan: {}", this.env.getExecutionPlan());
+
+    // Execute Job
+    try {
+      this.env.execute(this.applicationId);
+    } catch (final Exception e) { // NOPMD Execution thrown by Flink
+      LOGGER.error("An error occured while running this job.", e);
+    }
+  }
+
+  public static void main(final String[] args) {
+    new AggregationServiceFlinkJob().run();
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ChildParentsFlatMapFunction.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ChildParentsFlatMapFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..910dc359fa9b5b0810f7f9b6e67bfceaa68cc798
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ChildParentsFlatMapFunction.java
@@ -0,0 +1,102 @@
+package theodolite.uc4.application;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+import titan.ccp.model.sensorregistry.AggregatedSensor;
+import titan.ccp.model.sensorregistry.Sensor;
+import titan.ccp.model.sensorregistry.SensorRegistry;
+
+/**
+ * Transforms a {@link SensorRegistry} into key value pairs of Sensor identifiers and their parents'
+ * sensor identifiers. All pairs whose sensor's parents have changed since last iteration are
+ * forwarded. A mapping of an identifier to <code>null</code> means that the corresponding sensor
+ * does not longer exists in the sensor registry.
+ */
+public class ChildParentsFlatMapFunction
+    extends RichFlatMapFunction<SensorRegistry, Tuple2<String, Set<String>>> {
+
+  private static final long serialVersionUID = 3969444219510915221L; // NOPMD
+
+  private transient MapState<String, Set<String>> state;
+
+  @Override
+  public void open(final Configuration parameters) {
+    final MapStateDescriptor<String, Set<String>> descriptor =
+        new MapStateDescriptor<>(
+            "child-parents-state",
+            TypeInformation.of(new TypeHint<String>() {}),
+            TypeInformation.of(new TypeHint<Set<String>>() {}));
+    this.state = this.getRuntimeContext().getMapState(descriptor);
+  }
+
+  @Override
+  public void flatMap(final SensorRegistry value, final Collector<Tuple2<String, Set<String>>> out)
+      throws Exception {
+    final Map<String, Set<String>> childParentsPairs = this.constructChildParentsPairs(value);
+    this.updateChildParentsPairs(childParentsPairs);
+    this.updateState(childParentsPairs);
+    childParentsPairs
+        .entrySet()
+        .stream()
+        .map(e -> new Tuple2<>(e.getKey(), e.getValue()))
+        .forEach(out::collect);
+  }
+
+  private Map<String, Set<String>> constructChildParentsPairs(final SensorRegistry registry) {
+    return this.streamAllChildren(registry.getTopLevelSensor())
+        .collect(Collectors.toMap(
+            Sensor::getIdentifier,
+            child -> child.getParent()
+                .map(p -> Set.of(p.getIdentifier()))
+                .orElseGet(Set::of)));
+  }
+
+  private Stream<Sensor> streamAllChildren(final AggregatedSensor sensor) {
+    return sensor.getChildren().stream()
+        .flatMap(s -> Stream.concat(
+            Stream.of(s),
+            s instanceof AggregatedSensor ? this.streamAllChildren((AggregatedSensor) s)
+                : Stream.empty()));
+  }
+
+  private void updateChildParentsPairs(final Map<String, Set<String>> childParentsPairs)
+      throws Exception { // NOPMD General exception thown by Flink
+    final Iterator<Map.Entry<String, Set<String>>> oldChildParentsPairs = this.state.iterator();
+    while (oldChildParentsPairs.hasNext()) {
+      final Map.Entry<String, Set<String>> oldChildParentPair = oldChildParentsPairs.next();
+      final String identifier = oldChildParentPair.getKey();
+      final Set<String> oldParents = oldChildParentPair.getValue();
+      final Set<String> newParents = childParentsPairs.get(identifier); // null if not exists
+      if (newParents == null) {
+        // Sensor was deleted
+        childParentsPairs.put(identifier, null);
+      } else if (newParents.equals(oldParents)) {
+        // No changes
+        childParentsPairs.remove(identifier);
+      }
+      // Else: Later Perhaps: Mark changed parents
+    }
+  }
+
+  private void updateState(final Map<String, Set<String>> childParentsPairs)
+      throws Exception { // NOPMD General exception thown by Flink
+    for (final Map.Entry<String, Set<String>> childParentPair : childParentsPairs.entrySet()) {
+      if (childParentPair.getValue() == null) {
+        this.state.remove(childParentPair.getKey());
+      } else {
+        this.state.put(childParentPair.getKey(), childParentPair.getValue());
+      }
+    }
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ConfigurationKeys.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ConfigurationKeys.java
new file mode 100644
index 0000000000000000000000000000000000000000..6497f6b055ef115c4a681499c5fa38657bb5d29e
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/ConfigurationKeys.java
@@ -0,0 +1,40 @@
+package theodolite.uc4.application;
+
+/**
+ * Keys to access configuration parameters.
+ */
+public final class ConfigurationKeys {
+  public static final String APPLICATION_NAME = "application.name";
+
+  public static final String APPLICATION_VERSION = "application.version";
+
+  public static final String CONFIGURATION_KAFKA_TOPIC = "configuration.kafka.topic";
+
+  public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers";
+
+  public static final String KAFKA_OUTPUT_TOPIC = "kafka.output.topic";
+
+  public static final String KAFKA_INPUT_TOPIC = "kafka.input.topic";
+  
+  public static final String SCHEMA_REGISTRY_URL = "schema.registry.url";
+
+  public static final String WINDOW_SIZE_MS = "window.size.ms";
+
+  public static final String WINDOW_GRACE_MS = "window.grace.ms";
+
+  public static final String COMMIT_INTERVAL_MS = "commit.interval.ms";
+
+  public static final String FLINK_STATE_BACKEND = "flink.state.backend";
+
+  public static final String FLINK_STATE_BACKEND_PATH = "flink.state.backend.path";
+
+  public static final String FLINK_STATE_BACKEND_MEMORY_SIZE = //NOPMD
+      "flink.state.backend.memory.size";
+
+  public static final String DEBUG = "debug";
+
+  public static final String CHECKPOINTING = "checkpointing";
+
+  private ConfigurationKeys() {}
+
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/JoinAndDuplicateCoFlatMapFunction.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/JoinAndDuplicateCoFlatMapFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..6ef9a72e9695cfccba0bbcca1238f7ebc94fc505
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/JoinAndDuplicateCoFlatMapFunction.java
@@ -0,0 +1,66 @@
+package theodolite.uc4.application;
+
+import java.util.Set;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
+import org.apache.flink.util.Collector;
+import theodolite.uc4.application.util.SensorParentKey;
+import titan.ccp.model.records.ActivePowerRecord;
+
+/**
+ * A {@link RichCoFlatMapFunction} which joins each incoming {@link ActivePowerRecord} with its
+ * corresponding parents. The {@link ActivePowerRecord} is duplicated for each parent. When
+ * receiving a new set of parents for a sensor, this operator updates its internal state and
+ * forwards "tombstone" record if a sensor does no longer have a certain parent.
+ */
+public class JoinAndDuplicateCoFlatMapFunction extends
+    RichCoFlatMapFunction<ActivePowerRecord, Tuple2<String, Set<String>>, Tuple2<SensorParentKey, ActivePowerRecord>> { // NOCS
+
+  private static final long serialVersionUID = -6992783644887835979L; // NOPMD
+
+  private transient MapState<String, Set<String>> state;
+
+  @Override
+  public void open(final Configuration parameters) throws Exception {
+    final MapStateDescriptor<String, Set<String>> descriptor =
+        new MapStateDescriptor<>(
+            "join-and-duplicate-state",
+            TypeInformation.of(new TypeHint<String>() {}),
+            TypeInformation.of(new TypeHint<Set<String>>() {}));
+    this.state = this.getRuntimeContext().getMapState(descriptor);
+  }
+
+  @Override
+  public void flatMap1(final ActivePowerRecord value,
+      final Collector<Tuple2<SensorParentKey, ActivePowerRecord>> out) throws Exception {
+    final Set<String> parents = this.state.get(value.getIdentifier());
+    if (parents == null) {
+      return;
+    }
+    for (final String parent : parents) {
+      out.collect(new Tuple2<>(new SensorParentKey(value.getIdentifier(), parent), value));
+    }
+  }
+
+  @Override
+  public void flatMap2(final Tuple2<String, Set<String>> value,
+      final Collector<Tuple2<SensorParentKey, ActivePowerRecord>> out) throws Exception {
+    final String sensor = value.f0;
+    final Set<String> oldParents = this.state.get(sensor);
+    final Set<String> newParents = value.f1;
+    if (oldParents != null && !newParents.equals(oldParents)) {
+      for (final String oldParent : oldParents) {
+        if (!newParents.contains(oldParent)) {
+          // Parent was deleted, emit tombstone record
+          out.collect(new Tuple2<>(new SensorParentKey(sensor, oldParent), null));
+        }
+      }
+    }
+    this.state.put(sensor, newParents);
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/RecordAggregationProcessWindowFunction.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/RecordAggregationProcessWindowFunction.java
new file mode 100644
index 0000000000000000000000000000000000000000..45d4a09d153881572c949d2af7542f9cffb5622d
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/RecordAggregationProcessWindowFunction.java
@@ -0,0 +1,102 @@
+package theodolite.uc4.application;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.Collector;
+import theodolite.uc4.application.util.SensorParentKey;
+import titan.ccp.model.records.ActivePowerRecord;
+import titan.ccp.model.records.AggregatedActivePowerRecord;
+
+/**
+ * A {@link ProcessWindowFunction} which performs the windowed aggregation of all
+ * {@link ActivePowerRecord} for the same {@link SensorParentKey}. Result of this aggregation is an
+ * {@link AggregatedActivePowerRecord}.
+ */
+public class RecordAggregationProcessWindowFunction extends
+    ProcessWindowFunction<Tuple2<SensorParentKey, ActivePowerRecord>, AggregatedActivePowerRecord, String, TimeWindow> { // NOCS
+
+  private static final long serialVersionUID = 6030159552332624435L; // NOPMD
+
+  private transient MapState<SensorParentKey, ActivePowerRecord> lastValueState;
+  private transient ValueState<AggregatedActivePowerRecord> aggregateState;
+
+  @Override
+  public void open(final Configuration parameters) {
+    final MapStateDescriptor<SensorParentKey, ActivePowerRecord> lastValueStateDescriptor =
+        new MapStateDescriptor<>(
+            "last-value-state",
+            TypeInformation.of(new TypeHint<SensorParentKey>() {}),
+            TypeInformation.of(new TypeHint<ActivePowerRecord>() {}));
+    this.lastValueState = this.getRuntimeContext().getMapState(lastValueStateDescriptor);
+
+    final ValueStateDescriptor<AggregatedActivePowerRecord> aggregateStateDescriptor =
+        new ValueStateDescriptor<>(
+            "aggregation-state",
+            TypeInformation.of(new TypeHint<AggregatedActivePowerRecord>() {}));
+    this.aggregateState = this.getRuntimeContext().getState(aggregateStateDescriptor);
+  }
+
+  @Override
+  public void process(
+      final String key,
+      final Context context,
+      final Iterable<Tuple2<SensorParentKey, ActivePowerRecord>> elements,
+      final Collector<AggregatedActivePowerRecord> out) throws Exception {
+    for (final Tuple2<SensorParentKey, ActivePowerRecord> t : elements) {
+      AggregatedActivePowerRecord currentAggregate = this.aggregateState.value();
+      if (currentAggregate == null) {
+        currentAggregate = new AggregatedActivePowerRecord(key, 0L, 0L, 0.0, 0.0);
+        this.aggregateState.update(currentAggregate);
+      }
+      long count = currentAggregate.getCount();
+
+      final SensorParentKey sensorParentKey = t.f0;
+      ActivePowerRecord newRecord = t.f1;
+      if (newRecord == null) { // sensor was deleted -> decrease count, set newRecord to zero
+        count--;
+        newRecord = new ActivePowerRecord(sensorParentKey.getSensor(), 0L, 0.0);
+      }
+
+      // get last value of this record from state or create 0 valued record
+      ActivePowerRecord previousRecord = this.lastValueState.get(sensorParentKey);
+      if (previousRecord == null) { // sensor was added -> increase count
+        count++;
+        previousRecord = new ActivePowerRecord(sensorParentKey.getSensor(), 0L, 0.0);
+      }
+
+      // if incoming record is older than the last saved record, skip the record
+      if (newRecord.getTimestamp() < previousRecord.getTimestamp()) {
+        continue;
+      }
+
+      // prefer newer timestamp, but use previous if 0 -> sensor was deleted
+      final long timestamp =
+          newRecord.getTimestamp() == 0 ? previousRecord.getTimestamp() : newRecord.getTimestamp();
+      final double sumInW =
+          currentAggregate.getSumInW() - previousRecord.getValueInW() + newRecord.getValueInW();
+      final double avgInW = count == 0 ? 0 : sumInW / count;
+
+      final AggregatedActivePowerRecord newAggregate = new AggregatedActivePowerRecord(
+          sensorParentKey.getParent(),
+          timestamp,
+          count,
+          sumInW,
+          avgInW);
+
+      // update state and aggregateState
+      this.lastValueState.put(sensorParentKey, newRecord);
+      this.aggregateState.update(newAggregate);
+    }
+
+    // emit aggregated record
+    out.collect(this.aggregateState.value());
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSensorRegistrySerializer.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSensorRegistrySerializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..e157f35c8a052d2d4a28526a0d98d56515d586d6
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSensorRegistrySerializer.java
@@ -0,0 +1,28 @@
+package theodolite.uc4.application.util;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.io.Serializable;
+import titan.ccp.model.sensorregistry.ImmutableSensorRegistry;
+
+/**
+ * A {@link Serializer} for {@link ImmutableSensorRegistry}s.
+ */
+public class ImmutableSensorRegistrySerializer extends Serializer<ImmutableSensorRegistry>
+    implements Serializable {
+
+  private static final long serialVersionUID = 1806411056006113017L; // NOPMD
+
+  @Override
+  public void write(final Kryo kryo, final Output output, final ImmutableSensorRegistry object) {
+    output.writeString(object.toJson());
+  }
+
+  @Override
+  public ImmutableSensorRegistry read(final Kryo kryo, final Input input,
+      final Class<ImmutableSensorRegistry> type) {
+    return (ImmutableSensorRegistry) ImmutableSensorRegistry.fromJson(input.readString());
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSetSerializer.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSetSerializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..6b2dbcdfb403705b39815dd31112deab7947d83d
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/ImmutableSetSerializer.java
@@ -0,0 +1,51 @@
+package theodolite.uc4.application.util;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.io.Serializable;
+import java.util.Set;
+
+/**
+ * A {@link Serializer} for serializing arbitrary {@link Set}s of {@link Object}s.
+ */
+public final class ImmutableSetSerializer extends Serializer<Set<Object>> implements Serializable {
+
+  private static final long serialVersionUID = 6919877826110724620L; // NOPMD
+
+  public ImmutableSetSerializer() {
+    super(false, true);
+  }
+
+  @Override
+  public void write(final Kryo kryo, final Output output, final Set<Object> object) {
+    output.writeInt(object.size(), true);
+    for (final Object elm : object) {
+      kryo.writeClassAndObject(output, elm);
+    }
+  }
+
+  @Override
+  public Set<Object> read(final Kryo kryo, final Input input, final Class<Set<Object>> type) {
+    final int size = input.readInt(true);
+    final Object[] list = new Object[size];
+    for (int i = 0; i < size; ++i) {
+      list[i] = kryo.readClassAndObject(input);
+    }
+    return Set.of(list);
+  }
+
+  /**
+   * Creates a new {@link ImmutableSetSerializer} and registers its serializer for the several
+   * related classes.
+   *
+   * @param kryo the {@link Kryo} instance to set the serializer on
+   */
+  public static void registerSerializers(final Kryo kryo) {
+    final ImmutableSetSerializer serializer = new ImmutableSetSerializer();
+    kryo.register(Set.of().getClass(), serializer);
+    kryo.register(Set.of(1).getClass(), serializer);
+    kryo.register(Set.of(1, 2, 3, 4).getClass(), serializer); // NOCS
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKey.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKey.java
new file mode 100644
index 0000000000000000000000000000000000000000..903b66dd12a2864d522fde7eb7cf3fdc2ec73bcd
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKey.java
@@ -0,0 +1,51 @@
+package theodolite.uc4.application.util;
+
+import java.util.Objects;
+
+/**
+ * A key consisting of the identifier of a sensor and an identifier of parent sensor.
+ */
+public class SensorParentKey {
+
+  private final String sensorIdentifier;
+
+  private final String parentIdentifier;
+
+  public SensorParentKey(final String sensorIdentifier, final String parentIdentifier) {
+    this.sensorIdentifier = sensorIdentifier;
+    this.parentIdentifier = parentIdentifier;
+  }
+
+  public String getSensor() {
+    return this.sensorIdentifier;
+  }
+
+  public String getParent() {
+    return this.parentIdentifier;
+  }
+
+  @Override
+  public String toString() {
+    return "{" + this.sensorIdentifier + ", " + this.parentIdentifier + "}";
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.sensorIdentifier, this.parentIdentifier);
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (!(obj instanceof SensorParentKey)) {
+      return false;
+    }
+    final SensorParentKey k = (SensorParentKey) obj;
+    return this.sensorIdentifier.equals(k.sensorIdentifier)
+        && this.parentIdentifier.equals(k.parentIdentifier);
+  }
+
+
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKeySerializer.java b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKeySerializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..bdd403a05de8f54f636568e839f5f48effd43d58
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/java/theodolite/uc4/application/util/SensorParentKeySerializer.java
@@ -0,0 +1,30 @@
+package theodolite.uc4.application.util;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.io.Serializable;
+
+/**
+ * Kryo serializer for {@link SensorParentKey}.
+ */
+public final class SensorParentKeySerializer extends Serializer<SensorParentKey>
+    implements Serializable {
+
+  private static final long serialVersionUID = -867781963471414857L; // NOPMD
+
+  @Override
+  public void write(final Kryo kryo, final Output output, final SensorParentKey object) {
+    output.writeString(object.getSensor());
+    output.writeString(object.getParent());
+  }
+
+  @Override
+  public SensorParentKey read(final Kryo kryo, final Input input,
+      final Class<SensorParentKey> type) {
+    final String sensor = input.readString();
+    final String parent = input.readString();
+    return new SensorParentKey(sensor, parent);
+  }
+}
diff --git a/theodolite-benchmarks/uc4-application-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc4-application-flink/src/main/resources/META-INF/application.properties
new file mode 100644
index 0000000000000000000000000000000000000000..de85fdb88c0462edc9fba58409918470fcb8cb6c
--- /dev/null
+++ b/theodolite-benchmarks/uc4-application-flink/src/main/resources/META-INF/application.properties
@@ -0,0 +1,16 @@
+application.name=theodolite-uc2-application
+application.version=0.0.1
+
+configuration.host=localhost
+configuration.port=8082
+configuration.kafka.topic=configuration
+
+kafka.bootstrap.servers=localhost:9092
+kafka.input.topic=input
+kafka.output.topic=output
+schema.registry.url=http://localhost:8081
+window.size.ms=1000
+window.grace.ms=0
+num.threads=1
+commit.interval.ms=1000
+cache.max.bytes.buffering=-1
diff --git a/theodolite-benchmarks/workload-generator-commons/build.gradle b/theodolite-benchmarks/workload-generator-commons/build.gradle
index 98d820b480ba0b357b74f82ebce5a647ee392461..c42fff0412c332bc8292e175a352c03ada71f659 100644
--- a/theodolite-benchmarks/workload-generator-commons/build.gradle
+++ b/theodolite-benchmarks/workload-generator-commons/build.gradle
@@ -2,4 +2,11 @@ dependencies {
   implementation 'com.google.guava:guava:30.1-jre'
   implementation 'com.hazelcast:hazelcast:4.1.1'
   implementation 'com.hazelcast:hazelcast-kubernetes:2.2.1'
+  implementation 'org.slf4j:slf4j-simple:1.7.25'
+  implementation('org.industrial-devops:titan-ccp-common:0.1.0-SNAPSHOT') { changing = true }
+  implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
+  implementation 'org.apache.kafka:kafka-streams:2.6.0' // TODO required?
+  
+  // Use JUnit test framework
+  testImplementation 'junit:junit:4.12'
 }
\ No newline at end of file