diff --git a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/BeamService.java b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/BeamService.java
index 0165fa644e1853353e73caeaf0b9d2df0f8e9aea..bf4bb9397ff808fbd016eb3aaa34c8a6db35d3d1 100644
--- a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/BeamService.java
+++ b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/BeamService.java
@@ -10,7 +10,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.commons.configuration2.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import titan.ccp.common.configuration.ServiceConfigurations;
+import rocks.theodolite.commons.commons.configuration.ServiceConfigurations;
 
 /**
  * A general Apache Beam-based microservice. It is configured by Beam pipeline, a Beam runner and
diff --git a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/ActivePowerRecordDeserializer.java b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/ActivePowerRecordDeserializer.java
index 0c8d3a4a847cf9422c4e364a31024fb3d0c3f87a..7f0f20c46f199f216db5a533eebeccf8a36e260e 100644
--- a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/ActivePowerRecordDeserializer.java
+++ b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/ActivePowerRecordDeserializer.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.commons.beam.kafka;
 
 import io.confluent.kafka.streams.serdes.avro.SpecificAvroDeserializer;
 import org.apache.kafka.common.serialization.Deserializer;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A Kafka {@link Deserializer} for typed Schema Registry {@link ActivePowerRecord}.
diff --git a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/EventTimePolicy.java b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/EventTimePolicy.java
index 62e56341518839b96ad059e1c496ea1babb4674d..f4cd67f247a5b2b2ab8cc96ef5f7c523386bdacb 100644
--- a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/EventTimePolicy.java
+++ b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/EventTimePolicy.java
@@ -5,7 +5,7 @@ import org.apache.beam.sdk.io.kafka.KafkaRecord;
 import org.apache.beam.sdk.io.kafka.TimestampPolicy;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.joda.time.Instant;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * TimeStampPolicy to use event time based on the timestamp of the record value.
diff --git a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/KafkaActivePowerTimestampReader.java b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/KafkaActivePowerTimestampReader.java
index e22d5c8eedcd545364511a1461208f30bcb0a75c..d7d25ae91af795fbfabf7b754aaaea44b7ecf617 100644
--- a/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/KafkaActivePowerTimestampReader.java
+++ b/theodolite-benchmarks/beam-commons/src/main/java/rocks/theodolite/benchmarks/commons/beam/kafka/KafkaActivePowerTimestampReader.java
@@ -8,7 +8,7 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.kafka.common.serialization.StringDeserializer;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Simple {@link PTransform} that reads from Kafka using {@link KafkaIO} with event time.
diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle
index 5849bd93221794d135f1c6cb3bcb62d2174724b5..7783baaa215a12bba79f26e70d8ac8073252a780 100644
--- a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle
+++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle
@@ -22,8 +22,7 @@ def apacheBeamVersion =  '2.35.0'
 
 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 }
-    implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
+    implementation project(':commons')
     implementation 'com.google.guava:guava:24.1-jre'
     implementation 'org.slf4j:slf4j-simple:1.7.25'
     implementation project(':beam-commons')
diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.flink.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.flink.gradle
index 7671e602211b6d9e923a3b2a4c87f40fff84c6ec..5665de9d8ea42b88ef915189234ead634c66215c 100644
--- a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.flink.gradle
+++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.flink.gradle
@@ -35,9 +35,7 @@ repositories {
 }
 
 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 project(':commons')
 
     implementation 'org.apache.kafka:kafka-clients:2.2.0'
     implementation 'com.google.guava:guava:30.1-jre'
diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.load-generator.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.load-generator.gradle
index fb4fd89d1fe8a6d625a3ba7b459e9b0961befdbc..b4927d53d0ed976a0f0dcecd6d096e3d6c7d9273 100644
--- a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.load-generator.gradle
+++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.load-generator.gradle
@@ -20,8 +20,7 @@ repositories {
 
 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 }
-  implementation('org.industrial-devops:titan-ccp-common-kafka:0.1.0-SNAPSHOT') { changing = true }
+  implementation project(':commons')
   implementation 'org.slf4j:slf4j-simple:1.7.25'
 
   // These dependencies are used for the workload-generator-commmon
diff --git a/theodolite-benchmarks/commons/build.gradle b/theodolite-benchmarks/commons/build.gradle
index 67860c1ac7f76d1fdbb383acd1aae09505e9ce01..b12eaf4e1c807cb194eb377c9436bfa69c309bb6 100644
--- a/theodolite-benchmarks/commons/build.gradle
+++ b/theodolite-benchmarks/commons/build.gradle
@@ -22,6 +22,7 @@ dependencies {
     implementation 'org.apache.kafka:kafka-clients:2.4.0'
     api 'org.apache.commons:commons-configuration2:2.0'
     api "org.apache.avro:avro:1.11.0"
+    api 'org.apache.kafka:kafka-streams:2.4.0' // needed in flink uc3
 
     // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
     implementation 'commons-beanutils:commons-beanutils:1.9.2' // necessary for commons-configuration2
@@ -53,7 +54,8 @@ sourceSets {
 //// Local avro creation
 avro {
     fieldVisibility = "PRIVATE"
-    createSetters = false
+    // Setters required for flink because of https://issues.apache.org/jira/browse/FLINK-13703
+    createSetters = true
 }
 
 task("generateAvroProtocol", type: com.github.davidmc24.gradle.plugin.avro.GenerateAvroProtocolTask) {
diff --git a/theodolite-benchmarks/http-bridge/build.gradle b/theodolite-benchmarks/http-bridge/build.gradle
index fa98d9fdd602174a945df95321f5e32b8c64052f..7714f9947b1361badb8a17d3eb6f063b1dc39aab 100644
--- a/theodolite-benchmarks/http-bridge/build.gradle
+++ b/theodolite-benchmarks/http-bridge/build.gradle
@@ -17,8 +17,7 @@ repositories {
 }
 
 dependencies {
-  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 project(':commons')
   implementation project(':load-generator-commons')
   
   implementation 'io.javalin:javalin:4.3.0'
diff --git a/theodolite-benchmarks/http-bridge/src/main/java/rocks/theodolite/benchmarks/httpbridge/EnvVarHttpBridgeFactory.java b/theodolite-benchmarks/http-bridge/src/main/java/rocks/theodolite/benchmarks/httpbridge/EnvVarHttpBridgeFactory.java
index d3c172ac87221ab03f0171883df26802bf5a3aa9..25b39b30c062958c9dbd5d05888e86fbfa292db0 100644
--- a/theodolite-benchmarks/http-bridge/src/main/java/rocks/theodolite/benchmarks/httpbridge/EnvVarHttpBridgeFactory.java
+++ b/theodolite-benchmarks/http-bridge/src/main/java/rocks/theodolite/benchmarks/httpbridge/EnvVarHttpBridgeFactory.java
@@ -5,7 +5,7 @@ import java.util.Objects;
 import java.util.Optional;
 import rocks.theodolite.benchmarks.loadgenerator.ConfigurationKeys;
 import rocks.theodolite.benchmarks.loadgenerator.TitanKafkaSenderFactory;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 class EnvVarHttpBridgeFactory {
 
diff --git a/theodolite-benchmarks/kstreams-commons/build.gradle b/theodolite-benchmarks/kstreams-commons/build.gradle
index 167a75327b251af20b1142fe42c82b3bbedfe62b..e443e3af19d8b51e230d6f57a865aef9a4d90e30 100644
--- a/theodolite-benchmarks/kstreams-commons/build.gradle
+++ b/theodolite-benchmarks/kstreams-commons/build.gradle
@@ -14,9 +14,8 @@ repositories {
 
 dependencies {
   // These dependencies are used internally, and not exposed to consumers on their own compile classpath.
+  implementation project(':commons')
   // 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:3.1.0'
 
   // Use JUnit test framework
diff --git a/theodolite-benchmarks/load-generator-commons/build.gradle b/theodolite-benchmarks/load-generator-commons/build.gradle
index 62542eeabc1cccb35fa9f71d1929b72956a56999..082fb8eb9f1df15fb7b054588bb9728940eb442f 100644
--- a/theodolite-benchmarks/load-generator-commons/build.gradle
+++ b/theodolite-benchmarks/load-generator-commons/build.gradle
@@ -18,8 +18,7 @@ dependencies {
   implementation 'org.slf4j:slf4j-simple:1.7.25'
   implementation 'com.google.guava:guava:30.1-jre'
   implementation 'com.google.code.gson:gson:2.8.2'
-  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 project(':commons')
   implementation 'org.apache.kafka:kafka-streams:2.6.0' // TODO required?
   implementation platform('com.google.cloud:libraries-bom:24.2.0')
   implementation 'com.google.protobuf:protobuf-java-util'
diff --git a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/EnvVarLoadGeneratorFactory.java b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/EnvVarLoadGeneratorFactory.java
index ae9a6d4220ceaec091a0a2fb49fb82f16fdbb42e..ff3905e824b1671cd638d39501b61b3ec6c194b3 100644
--- a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/EnvVarLoadGeneratorFactory.java
+++ b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/EnvVarLoadGeneratorFactory.java
@@ -7,7 +7,7 @@ import java.util.Properties;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 class EnvVarLoadGeneratorFactory {
 
diff --git a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/KafkaRecordSender.java b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/KafkaRecordSender.java
index 56b1946ad78d888fe6e5140fdc373bb2cd3a4ed4..914d2881774610ba008189b77e1e7ec1585919f8 100644
--- a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/KafkaRecordSender.java
+++ b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/KafkaRecordSender.java
@@ -8,7 +8,7 @@ import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.StringSerializer;
-import titan.ccp.common.kafka.avro.SchemaRegistryAvroSerdeFactory;
+import rocks.theodolite.commons.kafka.avro.SchemaRegistryAvroSerdeFactory;
 
 /**
  * Sends records to Kafka.
diff --git a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanKafkaSenderFactory.java b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanKafkaSenderFactory.java
index ee7d416513439a5d0ba7bad7bcdb09e1baf5e4c7..7f0a1289e2fe3956ec898c3fcf7ab6dc4989dfad 100644
--- a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanKafkaSenderFactory.java
+++ b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanKafkaSenderFactory.java
@@ -1,7 +1,7 @@
 package rocks.theodolite.benchmarks.loadgenerator;
 
 import java.util.Properties;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A factory for creating {@link KafkaRecordSender}s that sends Titan {@link ActivePowerRecord}s.
diff --git a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanPubSubSenderFactory.java b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanPubSubSenderFactory.java
index 569d98fb9dcd235d12a0c415fdae495eeb6abdd4..41a76a62a5c6df45bc8820a2a99ff708ed849d75 100644
--- a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanPubSubSenderFactory.java
+++ b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanPubSubSenderFactory.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.loadgenerator;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A factory for creating {@link PubSubRecordSender}s that sends Titan {@link ActivePowerRecord}s.
diff --git a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGenerator.java b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGenerator.java
index 34b17db4220e60e1fd273b1acf0a0c1543a0742b..ba4d6d125746110e9af158d037a34ec01f534b63 100644
--- a/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGenerator.java
+++ b/theodolite-benchmarks/load-generator-commons/src/main/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGenerator.java
@@ -1,7 +1,7 @@
 package rocks.theodolite.benchmarks.loadgenerator;
 
 import java.time.Clock;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A factory for creating {@link RecordGenerator}s that creates Titan {@link ActivePowerRecord}s.
diff --git a/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/HttpRecordSenderTest.java b/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/HttpRecordSenderTest.java
index 731dda6c74fd3cd6d74771f95896c2260ce6df29..cbb3906331d62146acc1f3f8814d614ad972afbe 100644
--- a/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/HttpRecordSenderTest.java
+++ b/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/HttpRecordSenderTest.java
@@ -14,7 +14,7 @@ import java.net.URI;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 public class HttpRecordSenderTest {
 
diff --git a/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGeneratorTest.java b/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGeneratorTest.java
index eb7c7f2a403dfb7138bdfd8e0855930001a4488b..d1d66829e25d15a69dabaf214102206e46c5c226 100644
--- a/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGeneratorTest.java
+++ b/theodolite-benchmarks/load-generator-commons/src/test/java/rocks/theodolite/benchmarks/loadgenerator/TitanRecordGeneratorTest.java
@@ -7,7 +7,7 @@ import java.time.ZoneId;
 import java.time.ZoneOffset;
 import org.junit.Assert;
 import org.junit.Test;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 public class TitanRecordGeneratorTest {
 
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/ConverterAdapter.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/ConverterAdapter.java
index 08834a47223e8b4209da79c4cdcbb6a60e027418..fb3bd8055e6e6b2f1433bce11b2de49547159bf5 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/ConverterAdapter.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/ConverterAdapter.java
@@ -3,7 +3,7 @@ package rocks.theodolite.benchmarks.uc1.beam;
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import rocks.theodolite.benchmarks.uc1.commons.RecordConverter;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * {@link SimpleFunction} which wraps a {@link RecordConverter} to be used with Beam.
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/GenericSink.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/GenericSink.java
index 04eae799837b2e5278842d248d135e479f84086b..fe33450ea7901ff23ebc6bd63bfcfc6f8a980015 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/GenericSink.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/GenericSink.java
@@ -5,7 +5,7 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import rocks.theodolite.benchmarks.uc1.commons.DatabaseAdapter;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A {@link PTransform} for a generic {@link DatabaseAdapter}.
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/PipelineFactory.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/PipelineFactory.java
index d95d9b3343835f8348af15c3d00c34ef807d4501..4cdd0cfc60bcfd30dd590a857928ba891937f5e6 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/PipelineFactory.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/PipelineFactory.java
@@ -10,7 +10,7 @@ import org.apache.commons.configuration2.Configuration;
 import rocks.theodolite.benchmarks.commons.beam.AbstractPipelineFactory;
 import rocks.theodolite.benchmarks.commons.beam.kafka.KafkaActivePowerTimestampReader;
 import rocks.theodolite.benchmarks.uc1.beam.firestore.FirestoreOptionsExpander;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * {@link AbstractPipelineFactory} for UC1.
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkFactory.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkFactory.java
index d4854293b0c26804f0204e58c09a45f13dbf171f..a45b4bc3e3fd915f8447267fce279131e671abb2 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkFactory.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkFactory.java
@@ -3,7 +3,7 @@ package rocks.theodolite.benchmarks.uc1.beam;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.configuration2.Configuration;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Interface for a class that creates sinks (i.e., {@link PTransform}s that map and store
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkType.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkType.java
index da836815e09631e2ebc071badc02618171e0792a..a1c9570411315be91d4826f2e5f1275b9b65d4e8 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkType.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/SinkType.java
@@ -6,7 +6,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.configuration2.Configuration;
 import rocks.theodolite.benchmarks.uc1.beam.firestore.FirestoreSink;
 import rocks.theodolite.benchmarks.uc1.commons.logger.LogWriterFactory;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Supported Sink types, i.e., {@link PTransform} for converting and storing
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/DocumentMapper.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/DocumentMapper.java
index 1abf847250779150bb48b45c162afaeac1130044..b72bedce2a7b2c43c935a89a7b798d4dada2e411 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/DocumentMapper.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/DocumentMapper.java
@@ -4,7 +4,7 @@ import com.google.firestore.v1.Document;
 import com.google.firestore.v1.Value;
 import java.io.IOException;
 import org.apache.beam.sdk.transforms.SimpleFunction;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 final class DocumentMapper extends SimpleFunction<ActivePowerRecord, Document> {
 
diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/FirestoreSink.java b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/FirestoreSink.java
index dfe3f240b9727d0fa5027ea5f29cd67def3323ba..ba6542f4a63d1f47adf821dda241b1f79f24d5ef 100644
--- a/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/FirestoreSink.java
+++ b/theodolite-benchmarks/uc1-beam/src/main/java/rocks/theodolite/benchmarks/uc1/beam/firestore/FirestoreSink.java
@@ -7,7 +7,7 @@ import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.configuration2.Configuration;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A {@link PTransform} mapping {@link ActivePowerRecord}s to {@link Document}s, followed by storing
diff --git a/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/ConverterAdapter.java b/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/ConverterAdapter.java
index 064b8afbb03dc16262ca7fcf90a0fdd8af4419a9..05bd1432b3fbb4e83588c772f019ef2f8bdf4704 100644
--- a/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/ConverterAdapter.java
+++ b/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/ConverterAdapter.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc1.flink;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import rocks.theodolite.benchmarks.uc1.commons.RecordConverter;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * {@link MapFunction} which wraps a {@link RecordConverter} to be used with Flink.
diff --git a/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/HistoryServiceFlinkJob.java
index d674effac653cb1613a1b218f381ec3c6c910673..dce779382843fca81c60841ec1c6f07331aa0605 100644
--- a/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/HistoryServiceFlinkJob.java
+++ b/theodolite-benchmarks/uc1-flink/src/main/java/rocks/theodolite/benchmarks/uc1/flink/HistoryServiceFlinkJob.java
@@ -10,8 +10,8 @@ import org.slf4j.LoggerFactory;
 import rocks.theodolite.benchmarks.commons.flink.KafkaConnectorFactory;
 import rocks.theodolite.benchmarks.uc1.commons.DatabaseAdapter;
 import rocks.theodolite.benchmarks.uc1.commons.logger.LogWriterFactory;
-import titan.ccp.common.configuration.ServiceConfigurations;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.commons.configuration.ServiceConfigurations;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * The History microservice implemented as a Flink job.
diff --git a/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/PipelineFactory.java b/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/PipelineFactory.java
index 375b2a6cba5256e0644b6beaf26d41e010089250..868ebfa79770d4d19217b5821b77e85fcd8ea8b5 100644
--- a/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/PipelineFactory.java
+++ b/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/PipelineFactory.java
@@ -22,7 +22,7 @@ import rocks.theodolite.benchmarks.commons.beam.AbstractPipelineFactory;
 import rocks.theodolite.benchmarks.commons.beam.ConfigurationKeys;
 import rocks.theodolite.benchmarks.commons.beam.kafka.KafkaActivePowerTimestampReader;
 import rocks.theodolite.benchmarks.commons.beam.kafka.KafkaWriterTransformation;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * {@link AbstractPipelineFactory} for UC2.
diff --git a/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/StatsAggregation.java b/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/StatsAggregation.java
index a8956ee4b55c1e545e2c25ce38e2911b7c961337..aeb614fd0f973628f1e052ee50ed0d7e92a8feac 100644
--- a/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/StatsAggregation.java
+++ b/theodolite-benchmarks/uc2-beam/src/main/java/rocks/theodolite/benchmarks/uc2/beam/StatsAggregation.java
@@ -6,7 +6,7 @@ import java.io.Serializable;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Aggregation Class for ActivePowerRecords. Creates a StatsAccumulator based on the ValueInW.
diff --git a/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/HistoryServiceFlinkJob.java
index 7e67be897ce06f9f12e3fbcefb61d44a0775eea5..ace81d5e36d67694628b874ca34501e7a4e61402 100644
--- a/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/HistoryServiceFlinkJob.java
+++ b/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/HistoryServiceFlinkJob.java
@@ -16,8 +16,8 @@ import org.slf4j.LoggerFactory;
 import rocks.theodolite.benchmarks.commons.flink.KafkaConnectorFactory;
 import rocks.theodolite.benchmarks.commons.flink.StateBackends;
 import rocks.theodolite.benchmarks.commons.flink.serialization.StatsSerializer;
-import titan.ccp.common.configuration.ServiceConfigurations;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.commons.configuration.ServiceConfigurations;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 
 /**
diff --git a/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/StatsAggregateFunction.java b/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/StatsAggregateFunction.java
index b2a9e5f538c9e92ba777dbcd61caaa3199ebb383..a43d9b4efb35b7d4178d82b2e3bb6e0c5ed000ee 100644
--- a/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/StatsAggregateFunction.java
+++ b/theodolite-benchmarks/uc2-flink/src/main/java/rocks/theodolite/benchmarks/uc2/flink/StatsAggregateFunction.java
@@ -4,7 +4,7 @@ import com.google.common.math.Stats;
 import com.google.common.math.StatsAccumulator;
 import org.apache.flink.api.common.functions.AggregateFunction;
 import rocks.theodolite.benchmarks.uc2.flink.util.StatsFactory;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Statistical aggregation of {@link ActivePowerRecord}s using {@link Stats}.
diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/HourOfDayKeySerde.java b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/HourOfDayKeySerde.java
index 1c09d7d508888b48c2a509d83f55ff49ca967f17..8632572f0b1f84dedd76fee744d6001c366a7f38 100644
--- a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/HourOfDayKeySerde.java
+++ b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/HourOfDayKeySerde.java
@@ -1,10 +1,10 @@
 package rocks.theodolite.benchmarks.uc3.beam;
 
 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;
+import rocks.theodolite.commons.kafka.simpleserdes.BufferSerde;
+import rocks.theodolite.commons.kafka.simpleserdes.ReadBuffer;
+import rocks.theodolite.commons.kafka.simpleserdes.SimpleSerdes;
+import rocks.theodolite.commons.kafka.simpleserdes.WriteBuffer;
 
 /**
  * {@link BufferSerde} for a {@link HourOfDayKey}. Use the {@link #create()} method to create a new
diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/MapTimeFormat.java b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/MapTimeFormat.java
index 3c0d7acdbeccfaf03aac70df478e3db6dd1378e4..f183878a0422461a661fa7d8186459e7825feab3 100644
--- a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/MapTimeFormat.java
+++ b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/MapTimeFormat.java
@@ -5,7 +5,7 @@ import java.time.LocalDateTime;
 import java.time.ZoneId;
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Changes the time format to us Europe/Paris time.
diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/PipelineFactory.java b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/PipelineFactory.java
index 9c766e41254555647dd7ef1eed0417613b7c1629..c8b0220bf10e66c2c077c7f5dad9625552dfb93b 100644
--- a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/PipelineFactory.java
+++ b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/PipelineFactory.java
@@ -23,7 +23,7 @@ import rocks.theodolite.benchmarks.commons.beam.AbstractPipelineFactory;
 import rocks.theodolite.benchmarks.commons.beam.ConfigurationKeys;
 import rocks.theodolite.benchmarks.commons.beam.kafka.KafkaActivePowerTimestampReader;
 import rocks.theodolite.benchmarks.commons.beam.kafka.KafkaWriterTransformation;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * {@link AbstractPipelineFactory} for UC3.
diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/StatsAggregation.java b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/StatsAggregation.java
index e479c0ab5e192f7e5239c100a05df454bd2973ad..1e2130846fdfd97e40bae160065ce56d7dd3461a 100644
--- a/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/StatsAggregation.java
+++ b/theodolite-benchmarks/uc3-beam/src/main/java/rocks/theodolite/benchmarks/uc3/beam/StatsAggregation.java
@@ -6,7 +6,7 @@ import java.io.Serializable;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 
 /**
diff --git a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/HistoryServiceFlinkJob.java b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/HistoryServiceFlinkJob.java
index 4cf7ed080882e42bf488a8e6490c3eab463882b4..f630b44285de9c084434bf13f3b90848fa9cbb37 100644
--- a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/HistoryServiceFlinkJob.java
+++ b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/HistoryServiceFlinkJob.java
@@ -24,8 +24,8 @@ import rocks.theodolite.benchmarks.uc3.flink.util.HourOfDayKey;
 import rocks.theodolite.benchmarks.uc3.flink.util.HourOfDayKeyFactory;
 import rocks.theodolite.benchmarks.uc3.flink.util.HourOfDayKeySerde;
 import rocks.theodolite.benchmarks.uc3.flink.util.StatsKeyFactory;
-import titan.ccp.common.configuration.ServiceConfigurations;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.commons.configuration.ServiceConfigurations;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * The History microservice implemented as a Flink job.
diff --git a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/StatsAggregateFunction.java b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/StatsAggregateFunction.java
index 38d277dac28e88c82a38a1b56ef15cab2c00db14..f58bda804f95482bfe53792dbc1fa6753153178e 100644
--- a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/StatsAggregateFunction.java
+++ b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/StatsAggregateFunction.java
@@ -4,7 +4,7 @@ import com.google.common.math.Stats;
 import com.google.common.math.StatsAccumulator;
 import org.apache.flink.api.common.functions.AggregateFunction;
 import rocks.theodolite.benchmarks.uc3.flink.util.StatsFactory;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Statistical aggregation of {@link ActivePowerRecord}s using {@link Stats}.
diff --git a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayKeySerde.java b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayKeySerde.java
index 2c5bd40fe29f683693fb57bc679dd288e7d3bfb9..f9756bdaa75e429b175a23dc121f6cb299cb1397 100644
--- a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayKeySerde.java
+++ b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayKeySerde.java
@@ -6,10 +6,10 @@ 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;
+import rocks.theodolite.commons.kafka.simpleserdes.BufferSerde;
+import rocks.theodolite.commons.kafka.simpleserdes.ReadBuffer;
+import rocks.theodolite.commons.kafka.simpleserdes.SimpleSerdes;
+import rocks.theodolite.commons.kafka.simpleserdes.WriteBuffer;
 
 /**
  * {@link BufferSerde} for a {@link HourOfDayKey}. Use the {@link #create()} method to create a new
diff --git a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayRecordFactory.java b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayRecordFactory.java
index 5c52a446eac7f7ba39dfe12c247744054fd735d5..69aaa25f011186a74fbd845eba6c124685cf7282 100644
--- a/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayRecordFactory.java
+++ b/theodolite-benchmarks/uc3-flink/src/main/java/rocks/theodolite/benchmarks/uc3/flink/util/HourOfDayRecordFactory.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc3.flink.util;
 
 import com.google.common.math.Stats;
 import org.apache.kafka.streams.kstream.Windowed;
-import titan.ccp.model.records.HourOfDayActivePowerRecord;
+import rocks.theodolite.commons.model.records.HourOfDayActivePowerRecord;
 
 /**
  * {@link StatsRecordFactory} to create an {@link HourOfDayActivePowerRecord}.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedActivePowerRecordEventTimePolicy.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedActivePowerRecordEventTimePolicy.java
index 3af4957aa210a5f6a184bf4b513202dd4087567f..d55c3c78ba5369a3b19bfee9096553d9ee535a9c 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedActivePowerRecordEventTimePolicy.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedActivePowerRecordEventTimePolicy.java
@@ -5,7 +5,7 @@ import org.apache.beam.sdk.io.kafka.KafkaRecord;
 import org.apache.beam.sdk.io.kafka.TimestampPolicy;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.joda.time.Instant;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * TimeStampPolicy to use event time based on the timestamp of the record value.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedToActive.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedToActive.java
index 63f914dca0e6536d52ce225e791d2e3b89107394..cf34ce816dbac36d1ceb52fee06fb6fea79aacc0 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedToActive.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/AggregatedToActive.java
@@ -2,8 +2,8 @@ package rocks.theodolite.benchmarks.uc4.beam;
 
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.model.records.ActivePowerRecord;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * Converts AggregatedActivePowerRecord to ActivePowerRecord.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/DuplicateAsFlatMap.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/DuplicateAsFlatMap.java
index 1b430c99bb127f0746772571443f625ae69be54d..bc3fbdc02bec6e5236796e397844c16fce0a17b8 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/DuplicateAsFlatMap.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/DuplicateAsFlatMap.java
@@ -10,7 +10,7 @@ import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionView;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 
 /**
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterEvents.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterEvents.java
index 6a96d7270640c1cfa3a162e0d506792d577103c5..1325f9f4527568e009387231eb0ffc2bed29f8aa 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterEvents.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterEvents.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam;
 
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.configuration.events.Event;
+import rocks.theodolite.commons.configuration.events.Event;
 
 /**
  * Filters for {@code Event.SENSOR_REGISTRY_CHANGED} and {@code Event.SENSOR_REGISTRY_STATUS}
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterNullValues.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterNullValues.java
index 9aa48d7b8f6fdc64aad205821ec4db25f606156b..858262bf49f4122c5ba642ce6a2f6d6154d3dbde 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterNullValues.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/FilterNullValues.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam;
 
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Filters {@code null} Values.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/GenerateParentsFn.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/GenerateParentsFn.java
index e6f01c3d59c5c92f5d0c51ac807a76f0a37612b1..226f0fdb7512819e6d26bf8b88e406d0c37fbc52 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/GenerateParentsFn.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/GenerateParentsFn.java
@@ -8,10 +8,10 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.configuration.events.Event;
-import titan.ccp.model.sensorregistry.AggregatedSensor;
-import titan.ccp.model.sensorregistry.Sensor;
-import titan.ccp.model.sensorregistry.SensorRegistry;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.model.sensorregistry.AggregatedSensor;
+import rocks.theodolite.commons.model.sensorregistry.Sensor;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 /**
  * DoFn class to generate a child-parent pair for every sensor in the hierarchy.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/PipelineFactory.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/PipelineFactory.java
index 42d12d82026df0682f771b0cec5c1705ead83b2e..69ab5c5bdf01b641659b03afc07eda7b0640d61d 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/PipelineFactory.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/PipelineFactory.java
@@ -46,9 +46,9 @@ import rocks.theodolite.benchmarks.uc4.beam.serialization.AggregatedActivePowerR
 import rocks.theodolite.benchmarks.uc4.beam.serialization.EventCoder;
 import rocks.theodolite.benchmarks.uc4.beam.serialization.EventDeserializer;
 import rocks.theodolite.benchmarks.uc4.beam.serialization.SensorParentKeyCoder;
-import titan.ccp.configuration.events.Event;
-import titan.ccp.model.records.ActivePowerRecord;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * {@link AbstractPipelineFactory} for UC4.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/RecordAggregation.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/RecordAggregation.java
index 6fbf6ff3f3abcdfd4f5ca0de93d68c2532655b4d..2e36901cf7ef166d9e046cdf7e69bd8ff71a83a1 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/RecordAggregation.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/RecordAggregation.java
@@ -4,8 +4,8 @@ import java.io.Serializable;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import titan.ccp.model.records.ActivePowerRecord;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 
 
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetIdForAggregated.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetIdForAggregated.java
index 23ebb6d0104ef3992b4e2a4763dd23e722fe30f9..2fb648e0512a3bc0c7f36f8dd484ede1440df971 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetIdForAggregated.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetIdForAggregated.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam;
 
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * Sets the identifier for new {@link AggregatedActivePowerRecord}.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetKeyToGroup.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetKeyToGroup.java
index 3ffba12823aca2da88250eb601615de1ac5177e2..ea90ae32d93679b89528b5856e66468ceb0e8d46 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetKeyToGroup.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/SetKeyToGroup.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam;
 
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.KV;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * Set the Key for a group of {@code ActivePowerRecords} to their Parent.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordCoder.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordCoder.java
index f460d42de2042064952434f6bf044920f217138a..d44e4740717a45854ab6323e43f55d7eacfd753f 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordCoder.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordCoder.java
@@ -8,7 +8,7 @@ import java.util.List;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * {@link Coder} for an {@link AggregatedActivePowerRecord}.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordDeserializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordDeserializer.java
index 2c481f8e6f68a3a0decbcb73c3751f464646b7cf..2f345328a5f8c622702526d95a77c0baa004e10e 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordDeserializer.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordDeserializer.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam.serialization;
 
 import io.confluent.kafka.streams.serdes.avro.SpecificAvroDeserializer;
 import org.apache.kafka.common.serialization.Deserializer;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * {@link Deserializer} for an {@link AggregatedActivePowerRecord}.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordSerializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordSerializer.java
index 073c3d0f94c24872460ae58c6236a0c4e19e0d5d..5dff392cdc22af3b43f6500abaa76f261aceb1e1 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordSerializer.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/AggregatedActivePowerRecordSerializer.java
@@ -2,7 +2,7 @@ package rocks.theodolite.benchmarks.uc4.beam.serialization;
 
 import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerializer;
 import org.apache.kafka.common.serialization.Serializer;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * {@link Serializer} for an {@link AggregatedActivePowerRecord}.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventCoder.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventCoder.java
index b96398398e61f3db0f0632f8384d11f30ccc7aca..a8d7b37d9eb1c3aaafb092ef7c8eac6b4e4c67e5 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventCoder.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventCoder.java
@@ -10,8 +10,8 @@ import java.util.List;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.kafka.common.serialization.Serde;
-import titan.ccp.configuration.events.Event;
-import titan.ccp.configuration.events.EventSerde;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.configuration.events.EventSerde;
 
 /**
  * Wrapper Class that encapsulates a Event Serde in a org.apache.beam.sdk.coders.Coder.
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventDeserializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventDeserializer.java
index ae9c480e1c76949f8cfa5d27c73282e270232d0a..b57177d2e1794b50ecc716bfd3e0599cfa228a8f 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventDeserializer.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/EventDeserializer.java
@@ -3,7 +3,7 @@ package rocks.theodolite.benchmarks.uc4.beam.serialization;
 import java.util.Map;
 import org.apache.kafka.common.serialization.ByteBufferDeserializer;
 import org.apache.kafka.common.serialization.Deserializer;
-import titan.ccp.configuration.events.Event;
+import rocks.theodolite.commons.configuration.events.Event;
 
 /**
  * Deserializer for Events(SensorRegistry changes).
diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/SensorParentKeySerde.java b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/SensorParentKeySerde.java
index 64242b3fd2618b154e47cc46ce2fe6f9375be209..b5c197c9ba4de3fad3f45073e8e99029ee400579 100644
--- a/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/SensorParentKeySerde.java
+++ b/theodolite-benchmarks/uc4-beam/src/main/java/rocks/theodolite/benchmarks/uc4/beam/serialization/SensorParentKeySerde.java
@@ -2,10 +2,10 @@ package rocks.theodolite.benchmarks.uc4.beam.serialization;
 
 import org.apache.kafka.common.serialization.Serde;
 import rocks.theodolite.benchmarks.uc4.beam.SensorParentKey;
-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;
+import rocks.theodolite.commons.kafka.simpleserdes.BufferSerde;
+import rocks.theodolite.commons.kafka.simpleserdes.ReadBuffer;
+import rocks.theodolite.commons.kafka.simpleserdes.SimpleSerdes;
+import rocks.theodolite.commons.kafka.simpleserdes.WriteBuffer;
 
 /**
  * {@link Serde} factory for {@link SensorParentKey}.
diff --git a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/AggregationServiceFlinkJob.java b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/AggregationServiceFlinkJob.java
index 726a7f590aaa5be4ccb6e817076da5ede21cf272..76ac8b3a0f0466b49507195defc9bfbc23c50c65 100644
--- a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/AggregationServiceFlinkJob.java
+++ b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/AggregationServiceFlinkJob.java
@@ -26,14 +26,14 @@ import rocks.theodolite.benchmarks.uc4.flink.util.ImmutableSensorRegistrySeriali
 import rocks.theodolite.benchmarks.uc4.flink.util.ImmutableSetSerializer;
 import rocks.theodolite.benchmarks.uc4.flink.util.SensorParentKey;
 import rocks.theodolite.benchmarks.uc4.flink.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;
+import rocks.theodolite.commons.commons.configuration.ServiceConfigurations;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.configuration.events.EventSerde;
+import rocks.theodolite.commons.kafka.avro.SchemaRegistryAvroSerdeFactory;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.sensorregistry.ImmutableSensorRegistry;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 /**
  * The Aggregation microservice implemented as a Flink job.
diff --git a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/ChildParentsFlatMapFunction.java b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/ChildParentsFlatMapFunction.java
index c439df1e49381f8779a64814ae056635bd408c64..d1ef7258ac9e9b8e817ad5b8158f885a23836b34 100644
--- a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/ChildParentsFlatMapFunction.java
+++ b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/ChildParentsFlatMapFunction.java
@@ -13,9 +13,9 @@ 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;
+import rocks.theodolite.commons.model.sensorregistry.AggregatedSensor;
+import rocks.theodolite.commons.model.sensorregistry.Sensor;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 /**
  * Transforms a {@link SensorRegistry} into key value pairs of Sensor identifiers and their parents'
diff --git a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/JoinAndDuplicateCoFlatMapFunction.java b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/JoinAndDuplicateCoFlatMapFunction.java
index 624327a69feb8bce2f1b85af3e57cf5db9511e74..f5edbd8a3f52542182965bec0d9990e9480604b1 100644
--- a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/JoinAndDuplicateCoFlatMapFunction.java
+++ b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/JoinAndDuplicateCoFlatMapFunction.java
@@ -10,7 +10,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
 import org.apache.flink.util.Collector;
 import rocks.theodolite.benchmarks.uc4.flink.util.SensorParentKey;
-import titan.ccp.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
 
 /**
  * A {@link RichCoFlatMapFunction} which joins each incoming {@link ActivePowerRecord} with its
diff --git a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/RecordAggregationProcessWindowFunction.java b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/RecordAggregationProcessWindowFunction.java
index 29e1ea32235ae0789e72f4931932b2697c60759b..7c42d8f545d89737d53ca60b4ef0d1cb5d8b5972 100644
--- a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/RecordAggregationProcessWindowFunction.java
+++ b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/RecordAggregationProcessWindowFunction.java
@@ -12,8 +12,8 @@ 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 rocks.theodolite.benchmarks.uc4.flink.util.SensorParentKey;
-import titan.ccp.model.records.ActivePowerRecord;
-import titan.ccp.model.records.AggregatedActivePowerRecord;
+import rocks.theodolite.commons.model.records.ActivePowerRecord;
+import rocks.theodolite.commons.model.records.AggregatedActivePowerRecord;
 
 /**
  * A {@link ProcessWindowFunction} which performs the windowed aggregation of all
diff --git a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/util/ImmutableSensorRegistrySerializer.java b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/util/ImmutableSensorRegistrySerializer.java
index 54d8826900cc70a5eb86d71df7fcf9c4a4da4a7f..e235e91855f1c442e37dacf9bde5ddc70871143f 100644
--- a/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/util/ImmutableSensorRegistrySerializer.java
+++ b/theodolite-benchmarks/uc4-flink/src/main/java/rocks/theodolite/benchmarks/uc4/flink/util/ImmutableSensorRegistrySerializer.java
@@ -5,7 +5,7 @@ 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;
+import rocks.theodolite.commons.model.sensorregistry.ImmutableSensorRegistry;
 
 /**
  * A {@link Serializer} for {@link ImmutableSensorRegistry}s.
diff --git a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/ConfigPublisher.java b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/ConfigPublisher.java
index 98f470b0df1873766c3bf56fb7e6a8eae0019ce4..c7601ef3b389b38316a1e54735a091cf0e60c37d 100644
--- a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/ConfigPublisher.java
+++ b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/ConfigPublisher.java
@@ -7,8 +7,8 @@ import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringSerializer;
-import titan.ccp.configuration.events.Event;
-import titan.ccp.configuration.events.EventSerde;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.configuration.events.EventSerde;
 
 /**
  * Class to publish a configuration to Kafka.
diff --git a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/LoadGenerator.java b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/LoadGenerator.java
index 2077de2d9918d46a7a3e671ae9820a7c7abadbfc..79f4f21b069cdac618529f59355c8a688c557373 100644
--- a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/LoadGenerator.java
+++ b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/LoadGenerator.java
@@ -4,8 +4,8 @@ import java.util.Objects;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import rocks.theodolite.benchmarks.loadgenerator.KeySpace;
-import titan.ccp.configuration.events.Event;
-import titan.ccp.model.sensorregistry.SensorRegistry;
+import rocks.theodolite.commons.configuration.events.Event;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 /**
  * Load generator for Theodolite use case UC4.
diff --git a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilder.java b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilder.java
index c69dffb6093f914111b8c74bc25f3ca3a0a34ae6..b8b89a71a47a0a7cf5bc19f018643248aa2e88a5 100644
--- a/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilder.java
+++ b/theodolite-benchmarks/uc4-load-generator/src/main/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilder.java
@@ -1,8 +1,8 @@
 package rocks.theodolite.benchmarks.uc4.loadgenerator;
 
-import titan.ccp.model.sensorregistry.MutableAggregatedSensor;
-import titan.ccp.model.sensorregistry.MutableSensorRegistry;
-import titan.ccp.model.sensorregistry.SensorRegistry;
+import rocks.theodolite.commons.model.sensorregistry.MutableAggregatedSensor;
+import rocks.theodolite.commons.model.sensorregistry.MutableSensorRegistry;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 /**
  * Builder for creating a nested {@link SensorRegistry} with {@code numNestedGroups} levels and
diff --git a/theodolite-benchmarks/uc4-load-generator/src/test/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilderTest.java b/theodolite-benchmarks/uc4-load-generator/src/test/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilderTest.java
index a169eddb10ac34b91b814a657bf327a79ae00ac4..8c695f6e5ad1755e31034d3ce19ae05e16f1065d 100644
--- a/theodolite-benchmarks/uc4-load-generator/src/test/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilderTest.java
+++ b/theodolite-benchmarks/uc4-load-generator/src/test/java/rocks/theodolite/benchmarks/uc4/loadgenerator/SensorRegistryBuilderTest.java
@@ -5,10 +5,10 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import org.junit.Assert;
 import org.junit.Test;
-import titan.ccp.model.sensorregistry.AggregatedSensor;
-import titan.ccp.model.sensorregistry.MachineSensor;
-import titan.ccp.model.sensorregistry.Sensor;
-import titan.ccp.model.sensorregistry.SensorRegistry;
+import rocks.theodolite.commons.model.sensorregistry.AggregatedSensor;
+import rocks.theodolite.commons.model.sensorregistry.MachineSensor;
+import rocks.theodolite.commons.model.sensorregistry.Sensor;
+import rocks.theodolite.commons.model.sensorregistry.SensorRegistry;
 
 public class SensorRegistryBuilderTest {