diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 84edb0c1fd9770e37c9dd8510aef2db017edec49..66fb891898a1c57f8d814394a698a17bb7935164 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -204,10 +204,8 @@ spotbugs-benchmarks: - changes: - theodolite-benchmarks/* - theodolite-benchmarks/$JAVA_PROJECT_NAME/**/* - - theodolite-benchmarks/kstreams-commons/**/* - - theodolite-benchmarks/flink-commons/**/* - - theodolite-benchmarks/load-generator-commons/**/* - if: "$CR_HOST && $CR_ORG && $CR_USER && $CR_PW && $IMAGE_NAME && $JAVA_PROJECT_NAME" + - theodolite-benchmarks/{$JAVA_PROJECT_DEPS}/**/* + if: "$CR_HOST && $CR_ORG && $CR_USER && $CR_PW && $IMAGE_NAME && $JAVA_PROJECT_NAME && $JAVA_PROJECT_DEPS" - if: "$CR_HOST && $CR_ORG && $CR_USER && $CR_PW && $IMAGE_NAME && $JAVA_PROJECT_NAME" when: manual allow_failure: true @@ -217,72 +215,140 @@ deploy-uc1-kstreams: variables: IMAGE_NAME: "theodolite-uc1-kstreams-app" JAVA_PROJECT_NAME: "uc1-kstreams" + JAVA_PROJECT_DEPS: "kstreams-commons" deploy-uc2-kstreams: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc2-kstreams-app" JAVA_PROJECT_NAME: "uc2-kstreams" + JAVA_PROJECT_DEPS: "kstreams-commons" deploy-uc3-kstreams: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc3-kstreams-app" JAVA_PROJECT_NAME: "uc3-kstreams" + JAVA_PROJECT_DEPS: "kstreams-commons" deploy-uc4-kstreams: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc4-kstreams-app" JAVA_PROJECT_NAME: "uc4-kstreams" + JAVA_PROJECT_DEPS: "kstreams-commons" deploy-uc1-flink: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc1-flink" JAVA_PROJECT_NAME: "uc1-flink" + JAVA_PROJECT_DEPS: "flink-commons" deploy-uc2-flink: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc2-flink" JAVA_PROJECT_NAME: "uc2-flink" + JAVA_PROJECT_DEPS: "flink-commons" deploy-uc3-flink: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc3-flink" JAVA_PROJECT_NAME: "uc3-flink" + JAVA_PROJECT_DEPS: "flink-commons" deploy-uc4-flink: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc4-flink" JAVA_PROJECT_NAME: "uc4-flink" + JAVA_PROJECT_DEPS: "flink-commons" + +deploy-uc1-beam-flink: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc1-beam-flink" + JAVA_PROJECT_NAME: "uc1-beam-flink" + JAVA_PROJECT_DEPS: "beam-commons,uc1-beam" + +deploy-uc2-beam-flink: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc2-beam-flink" + JAVA_PROJECT_NAME: "uc2-beam-flink" + JAVA_PROJECT_DEPS: "beam-commons,uc2-beam" + +deploy-uc3-beam-flink: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc3-beam-flink" + JAVA_PROJECT_NAME: "uc3-beam-flink" + JAVA_PROJECT_DEPS: "beam-commons,uc3-beam" + +deploy-uc4-beam-flink: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc4-beam-flink" + JAVA_PROJECT_NAME: "uc4-beam-flink" + JAVA_PROJECT_DEPS: "beam-commons,uc4-beam" + +deploy-uc1-beam-samza: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc1-beam-samza" + JAVA_PROJECT_NAME: "uc1-beam-samza" + JAVA_PROJECT_DEPS: "beam-commons,uc1-beam" + +deploy-uc2-beam-samza: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc2-beam-samza" + JAVA_PROJECT_NAME: "uc2-beam-samza" + JAVA_PROJECT_DEPS: "beam-commons,uc2-beam" + +deploy-uc3-beam-samza: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc3-beam-samza" + JAVA_PROJECT_NAME: "uc3-beam-samza" + JAVA_PROJECT_DEPS: "beam-commons,uc3-beam" + +deploy-uc4-beam-samza: + extends: .deploy-benchmarks + variables: + IMAGE_NAME: "theodolite-uc4-beam-samza" + JAVA_PROJECT_NAME: "uc4-beam-samza" + JAVA_PROJECT_DEPS: "beam-commons,uc4-beam" deploy-uc1-load-generator: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc1-workload-generator" JAVA_PROJECT_NAME: "uc1-load-generator" + JAVA_PROJECT_DEPS: "load-generator-commons" deploy-uc2-load-generator: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc2-workload-generator" JAVA_PROJECT_NAME: "uc2-load-generator" + JAVA_PROJECT_DEPS: "load-generator-commons" deploy-uc3-load-generator: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc3-workload-generator" JAVA_PROJECT_NAME: "uc3-load-generator" + JAVA_PROJECT_DEPS: "load-generator-commons" deploy-uc4-load-generator: extends: .deploy-benchmarks variables: IMAGE_NAME: "theodolite-uc4-workload-generator" JAVA_PROJECT_NAME: "uc4-load-generator" + JAVA_PROJECT_DEPS: "load-generator-commons" # Theodolite Framework diff --git a/theodolite-benchmarks/beam-commons/build.gradle b/theodolite-benchmarks/beam-commons/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..66ec44ad715d64458584e71fdd4f49abb9c458f4 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/build.gradle @@ -0,0 +1,33 @@ +plugins { + id 'theodolite.java-commons' +} + +repositories { + jcenter() + maven { + url "https://oss.sonatype.org/content/repositories/snapshots/" + } + maven { + url 'https://packages.confluent.io/maven/' + } +} + +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 'com.google.code.gson:gson:2.8.2' + implementation 'com.google.guava:guava:24.1-jre' + + implementation('org.apache.beam:beam-sdks-java-io-kafka:2.22.0'){ + exclude group: 'org.apache.kafka', module: 'kafka-clients' + } + implementation group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.30' + implementation group: 'org.apache.beam', name: 'beam-sdks-java-core', version: '2.22.0' + + runtimeOnly 'org.slf4j:slf4j-api:1.7.32' + runtimeOnly 'org.slf4j:slf4j-jdk14:1.7.32' + + // Use JUnit test framework + testImplementation 'junit:junit:4.12' +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractBeamService.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractBeamService.java new file mode 100644 index 0000000000000000000000000000000000000000..03c5ca1daa7ffab71a4d08c04f677d7412e3a2be --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractBeamService.java @@ -0,0 +1,45 @@ +package theodolite.commons.beam; + +import org.apache.beam.sdk.options.PipelineOptions; +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; + +/** + * Abstraction of a Beam microservice. + * Encapsulates the corresponding {@link PipelineOptions} and the beam Runner. + */ +public class AbstractBeamService { + + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBeamService.class); + + // Beam Pipeline + protected PipelineOptions options; + + // Application Configurations + private final Configuration config = ServiceConfigurations.createWithDefaults(); + private final String applicationName = + config.getString(ConfigurationKeys.APPLICATION_NAME); + + + /** + * Creates AbstractBeamService with options. + */ + public AbstractBeamService(final String[] args) { //NOPMD + super(); + LOGGER.info("Pipeline options:"); + for (final String s : args) { + LOGGER.info("{}", s); + } + options = PipelineOptionsFactory.fromArgs(args).create(); + options.setJobName(applicationName); + LOGGER.info("Starting BeamService with PipelineOptions {}:", this.options.toString()); + } + + public Configuration getConfig() { + return config; + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractPipeline.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractPipeline.java new file mode 100644 index 0000000000000000000000000000000000000000..c936ce918c10f3c500cdd26f7e057cd7b6c555b6 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/AbstractPipeline.java @@ -0,0 +1,50 @@ +package theodolite.commons.beam; + +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.commons.configuration2.Configuration; +import org.apache.kafka.clients.consumer.ConsumerConfig; + +/** + * Abstraction of a Beam {@link Pipeline}. + */ +public class AbstractPipeline extends Pipeline { + + protected final String inputTopic; + protected final String bootstrapServer; + // Application Configurations + private final Configuration config; + + protected AbstractPipeline(final PipelineOptions options, final Configuration config) { + super(options); + this.config = config; + + inputTopic = config.getString(ConfigurationKeys.KAFKA_INPUT_TOPIC); + bootstrapServer = config.getString(ConfigurationKeys.KAFKA_BOOTSTRAP_SERVERS); + } + + /** + * Builds a simple configuration for a Kafka consumer transformation. + * + * @return the build configuration. + */ + public Map<String, Object> buildConsumerConfig() { + final Map<String, Object> consumerConfig = new HashMap<>(); + consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + config.getString(ConfigurationKeys.ENABLE_AUTO_COMMIT_CONFIG)); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + config + .getString(ConfigurationKeys.AUTO_OFFSET_RESET_CONFIG)); + consumerConfig.put("schema.registry.url", + config.getString(ConfigurationKeys.SCHEMA_REGISTRY_URL)); + + consumerConfig.put("specific.avro.reader", + config.getString(ConfigurationKeys.SPECIFIC_AVRO_READER)); + + final String applicationName = config.getString(ConfigurationKeys.APPLICATION_NAME); + consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, applicationName); + return consumerConfig; + } +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/ConfigurationKeys.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/ConfigurationKeys.java new file mode 100644 index 0000000000000000000000000000000000000000..1e4dc593c627282f5c6735a4d91e963d83af6865 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/ConfigurationKeys.java @@ -0,0 +1,48 @@ +package theodolite.commons.beam; + +/** + * Keys to access configuration parameters. + */ +public final class ConfigurationKeys { + // Common keys + public static final String APPLICATION_NAME = "application.name"; + + public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; + + public static final String SCHEMA_REGISTRY_URL = "schema.registry.url"; + + public static final String KAFKA_INPUT_TOPIC = "kafka.input.topic"; + + // Additional topics + public static final String KAFKA_FEEDBACK_TOPIC = "kafka.feedback.topic"; + + public static final String KAFKA_OUTPUT_TOPIC = "kafka.output.topic"; + + public static final String KAFKA_CONFIGURATION_TOPIC = "kafka.configuration.topic"; + + // UC2 + public static final String KAFKA_WINDOW_DURATION_MINUTES = "kafka.window.duration.minutes"; + + // UC3 + public static final String AGGREGATION_DURATION_DAYS = "aggregation.duration.days"; + + public static final String AGGREGATION_ADVANCE_DAYS = "aggregation.advance.days"; + + // UC4 + public static final String GRACE_PERIOD_MS = "grace.period.ms"; + + + // BEAM + public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit.config"; + + public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset.config"; + + public static final String SPECIFIC_AVRO_READER = "specific.avro.reader"; + + public static final String TRIGGER_INTERVAL = "trigger.interval"; + + + private ConfigurationKeys() { + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/EventTimePolicy.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/EventTimePolicy.java new file mode 100644 index 0000000000000000000000000000000000000000..e568968670b3ea51388f3e2a19da8f64bd7c5391 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/EventTimePolicy.java @@ -0,0 +1,35 @@ +package theodolite.commons.beam.kafka; + +import java.util.Optional; +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; + +/** + * TimeStampPolicy to use event time based on the timestamp of the record value. + */ +public class EventTimePolicy + extends TimestampPolicy<String, ActivePowerRecord> { + protected Instant currentWatermark; + + public EventTimePolicy(final Optional<Instant> previousWatermark) { + super(); + this.currentWatermark = previousWatermark.orElse(BoundedWindow.TIMESTAMP_MIN_VALUE); + } + + + @Override + public Instant getTimestampForRecord(final PartitionContext ctx, + final KafkaRecord<String, ActivePowerRecord> record) { + this.currentWatermark = new Instant(record.getKV().getValue().getTimestamp()); + return this.currentWatermark; + } + + @Override + public Instant getWatermark(final PartitionContext ctx) { + return this.currentWatermark; + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerRecordReader.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerRecordReader.java new file mode 100644 index 0000000000000000000000000000000000000000..f102bee41d66c251ecb66418dd3b90dced32cffb --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerRecordReader.java @@ -0,0 +1,61 @@ +package theodolite.commons.beam.kafka; + +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.PTransform; +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; + +/** + * Simple {@link PTransform} that read from Kafka using {@link KafkaIO}. + */ +public class KafkaActivePowerRecordReader extends + PTransform<PBegin, PCollection<KV<String, ActivePowerRecord>>> { + + private static final long serialVersionUID = 2603286150183186115L; + private final PTransform<PBegin, PCollection<KV<String, ActivePowerRecord>>> reader; + + + /** + * Instantiates a {@link PTransform} that reads from Kafka with the given Configuration. + */ + public KafkaActivePowerRecordReader(final String bootstrapServer, final String inputTopic, + final Map<String, Object> consumerConfig) { + super(); + + if (bootstrapServer == null) { + throw new IllegalArgumentException("bootstrapServer is null"); + } + + if (inputTopic == null) { + throw new IllegalArgumentException("inputTopic is null"); + } + + // Check if boostrap server and inputTopic are defined + if (bootstrapServer.isEmpty() || inputTopic.isEmpty()) { + throw new IllegalArgumentException("bootstrapServer or inputTopic missing"); + } + + + reader = + KafkaIO.<String, ActivePowerRecord>read() + .withBootstrapServers(bootstrapServer) + .withTopic(inputTopic) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializerAndCoder((Class) KafkaAvroDeserializer.class, + AvroCoder.of(ActivePowerRecord.class)) + .withConsumerConfigUpdates(consumerConfig) + .withoutMetadata(); + } + + @Override + public PCollection<KV<String, ActivePowerRecord>> expand(final PBegin input) { + return input.apply(this.reader); + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerTimestampReader.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerTimestampReader.java new file mode 100644 index 0000000000000000000000000000000000000000..732afe9a0c1d4bdfea876025fceea0c5da1310fe --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaActivePowerTimestampReader.java @@ -0,0 +1,56 @@ +package theodolite.commons.beam.kafka; + +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.PTransform; +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; + +/** + * Simple {@link PTransform} that read from Kafka using {@link KafkaIO}. + * Has additional a TimestampPolicy. + */ +public class KafkaActivePowerTimestampReader extends + PTransform<PBegin, PCollection<KV<String, ActivePowerRecord>>> { + + private static final long serialVersionUID = 2603286150183186115L; + private final PTransform<PBegin, PCollection<KV<String, ActivePowerRecord>>> reader; + + + /** + * Instantiates a {@link PTransform} that reads from Kafka with the given Configuration. + */ + public KafkaActivePowerTimestampReader(final String bootstrapServer, final String inputTopic, + final Map<String, Object> consumerConfig) { + super(); + + // Check if boostrap server and inputTopic are defined + if (bootstrapServer.isEmpty() || inputTopic.isEmpty()) { + throw new IllegalArgumentException("bootstrapServer or inputTopic missing"); + } + + reader = + KafkaIO.<String, ActivePowerRecord>read() + .withBootstrapServers(bootstrapServer) + .withTopic(inputTopic) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializerAndCoder((Class) KafkaAvroDeserializer.class, + AvroCoder.of(ActivePowerRecord.class)) + .withConsumerConfigUpdates(consumerConfig) + // Set TimeStampPolicy for event time + .withTimestampPolicyFactory( + (tp, previousWaterMark) -> new EventTimePolicy(previousWaterMark)) + .withoutMetadata(); + } + + @Override + public PCollection<KV<String, ActivePowerRecord>> expand(final PBegin input) { + return input.apply(this.reader); + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaGenericReader.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaGenericReader.java new file mode 100644 index 0000000000000000000000000000000000000000..83336b5a4c2451ef4bffefbd60ad9d52fccd9c17 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaGenericReader.java @@ -0,0 +1,55 @@ +package theodolite.commons.beam.kafka; + +import java.util.Map; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; + +/** + * Simple {@link PTransform} that read from Kafka using {@link KafkaIO}. + * + * @param <K> Type of the Key. + * @param <V> Type of the Value. + */ +public class KafkaGenericReader<K, V> extends + PTransform<PBegin, PCollection<KV<K, V>>> { + + private static final long serialVersionUID = 2603286150183186115L; + private final PTransform<PBegin, PCollection<KV<K, V>>> reader; + + /** + * Instantiates a {@link PTransform} that reads from Kafka with the given Configuration. + */ + public KafkaGenericReader(final String bootstrapServer, final String inputTopic, + final Map<String, Object> consumerConfig, + final Class<? extends + org.apache.kafka.common.serialization.Deserializer<K>> + keyDeserializer, + final Class<? extends + org.apache.kafka.common.serialization.Deserializer<V>> + valueDeserializer) { + super(); + + // Check if boostrap server and inputTopic are defined + if (bootstrapServer.isEmpty() || inputTopic.isEmpty()) { + throw new IllegalArgumentException("bootstrapServer or inputTopic missing"); + } + + reader = + KafkaIO.<K, V>read() + .withBootstrapServers(bootstrapServer) + .withTopic(inputTopic) + .withKeyDeserializer(keyDeserializer) + .withValueDeserializer(valueDeserializer) + .withConsumerConfigUpdates(consumerConfig) + .withoutMetadata(); + } + + @Override + public PCollection<KV<K, V>> expand(final PBegin input) { + return input.apply(this.reader); + } + +} diff --git a/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaWriterTransformation.java b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaWriterTransformation.java new file mode 100644 index 0000000000000000000000000000000000000000..0a3867e71479e36ce30a9f222dfd0a7d473bd209 --- /dev/null +++ b/theodolite-benchmarks/beam-commons/src/main/java/theodolite/commons/beam/kafka/KafkaWriterTransformation.java @@ -0,0 +1,45 @@ +package theodolite.commons.beam.kafka; + +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringSerializer; + +/** + * Wrapper for a Kafka writing Transformation + * where the value type can be generic. + * @param <T> type of the value. + */ +public class KafkaWriterTransformation<T> extends + PTransform<PCollection<KV<String, T>>, PDone> { + + private static final long serialVersionUID = 3171423303843174723L; + private final PTransform<PCollection<KV<String, T>>, PDone> writer; + + /** + * Creates a new kafka writer transformation. + */ + public KafkaWriterTransformation(final String bootstrapServer, final String outputTopic, + final Class<? extends Serializer<T>> valueSerializer) { + super(); + // Check if boostrap server and outputTopic are defined + if (bootstrapServer.isEmpty() || outputTopic.isEmpty()) { + throw new IllegalArgumentException("bootstrapServer or outputTopic missing"); + } + + this.writer = KafkaIO.<String, T>write() + .withBootstrapServers(bootstrapServer) + .withTopic(outputTopic) + .withKeySerializer(StringSerializer.class) + .withValueSerializer(valueSerializer); + + } + + @Override + public PDone expand(final PCollection<KV<String, T>> input) { + return input.apply(this.writer); + } +} diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.flink.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.flink.gradle new file mode 100644 index 0000000000000000000000000000000000000000..eb9bda1f84c4f20568fca1498462dff9082ea1fa --- /dev/null +++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.flink.gradle @@ -0,0 +1,8 @@ +plugins { + id 'theodolite.beam' +} + +dependencies { + implementation group: 'org.apache.beam', name: 'beam-runners-flink-1.12', version: '2.27.0' + implementation group: 'org.apache.flink', name: 'flink-statebackend-rocksdb_2.11', version: '1.12.0' +} \ No newline at end of file diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle new file mode 100644 index 0000000000000000000000000000000000000000..09e36d52171699c61b212b1f64c827933679b6fa --- /dev/null +++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.gradle @@ -0,0 +1,45 @@ +plugins { + // common java conventions + id 'theodolite.java-conventions' + + // make executable + id 'application' +} + +tasks.distZip.enabled = false + +repositories { + jcenter() + maven { + url "https://oss.sonatype.org/content/repositories/snapshots/" + } + maven { + url 'https://packages.confluent.io/maven/' + } +} + +def apacheBeamVersion = '2.22.0' //'2.27.0' // '2.34.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 'com.google.code.gson:gson:2.8.2' + implementation 'com.google.guava:guava:24.1-jre' + implementation 'org.slf4j:slf4j-simple:1.7.25' + implementation project(':beam-commons') + + implementation group: 'org.apache.beam', name: 'beam-sdks-java-core', version: "${apacheBeamVersion}" + + implementation("org.apache.beam:beam-sdks-java-io-kafka:${apacheBeamVersion}"){ + exclude group: 'org.apache.kafka', module: 'kafka-clients' + } + implementation group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.30' + + runtimeOnly "org.apache.beam:beam-runners-direct-java:${apacheBeamVersion}" + runtimeOnly 'org.slf4j:slf4j-api:1.7.32' + runtimeOnly 'org.slf4j:slf4j-jdk14:1.7.32' + + // Use JUnit test framework + testImplementation 'junit:junit:4.12' +} diff --git a/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.samza.gradle b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.samza.gradle new file mode 100644 index 0000000000000000000000000000000000000000..73e916ccc867b9b3316776192f0dab56fa0710f0 --- /dev/null +++ b/theodolite-benchmarks/buildSrc/src/main/groovy/theodolite.beam.samza.gradle @@ -0,0 +1,9 @@ +plugins { + id 'theodolite.beam' +} + +dependencies { + implementation('org.apache.beam:beam-runners-samza:2.22.0') { + exclude group: 'org.apache.samza', module: 'samza-yarn_2.11' + } +} \ No newline at end of file diff --git a/theodolite-benchmarks/docker-test/uc1-beam-flink/docker-compose.yml b/theodolite-benchmarks/docker-test/uc1-beam-flink/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..d8a7b946a9d5e407032ce02838b3ad02892eae73 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc1-beam-flink/docker-compose.yml @@ -0,0 +1,74 @@ +version: '2.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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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: load-generator: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-beam-flink:latest + #ports: + # - "8080:8081" + command: > + standalone-job --job-classname application.Uc1BeamFlink + --disableMetrics=true + --fasterCopy + 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-beam-flink:latest + scale: 1 + command: taskmanager + environment: + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + depends_on: + - schema-registry + - kafka diff --git a/theodolite-benchmarks/docker-test/uc1-beam-samza/docker-compose.yml b/theodolite-benchmarks/docker-test/uc1-beam-samza/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..11cf0c345b417fdda7cedba2f9db1342d2b64634 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc1-beam-samza/docker-compose.yml @@ -0,0 +1,59 @@ +version: '2.2' +services: + zookeeper: + image: confluentinc/cp-zookeeper + expose: + - "2181" + #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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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' + benchmark: + image: ghcr.io/cau-se/theodolite-uc1-beam-samza:latest + scale: 1 + depends_on: + - schema-registry + - kafka + environment: + SAMZA_JOB_COORDINATOR_ZK_CONNECT: zookeeper:2181 + SAMZA_SYSTEMS_KAFKA_PRODUCER_BOOTSTRAP_SERVERS: kafka:9092 + SAMZA_SYSTEMS_KAFKA_CONSUMER_BOOTSTRAP_SERVERS: kafka:9092 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + load-generator: + image: ghcr.io/cau-se/theodolite-uc1-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator:5701 + PORT: 5701 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + NUM_SENSORS: 10 diff --git a/theodolite-benchmarks/docker-test/uc2-beam-flink/docker-compose.yml b/theodolite-benchmarks/docker-test/uc2-beam-flink/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..f8bdfae935a55c8cb60e3fb22b19c471832ca9f4 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc2-beam-flink/docker-compose.yml @@ -0,0 +1,74 @@ +version: '2.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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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-uc2-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator: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-beam-flink:latest + #ports: + # - "8080:8081" + command: > + standalone-job --job-classname application.Uc2BeamFlink + --disableMetrics=true + --fasterCopy + 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-beam-flink:latest + scale: 1 + command: taskmanager + environment: + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + depends_on: + - schema-registry + - kafka diff --git a/theodolite-benchmarks/docker-test/uc2-beam-samza/docker-compose.yml b/theodolite-benchmarks/docker-test/uc2-beam-samza/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..67a5997b66833e33696592285dffe24b03b3d210 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc2-beam-samza/docker-compose.yml @@ -0,0 +1,59 @@ +version: '2.2' +services: + zookeeper: + image: confluentinc/cp-zookeeper + expose: + - "2181" + #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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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' + benchmark: + image: ghcr.io/cau-se/theodolite-uc2-beam-samza:latest + scale: 1 + depends_on: + - schema-registry + - kafka + environment: + SAMZA_JOB_COORDINATOR_ZK_CONNECT: zookeeper:2181 + SAMZA_SYSTEMS_KAFKA_PRODUCER_BOOTSTRAP_SERVERS: kafka:9092 + SAMZA_SYSTEMS_KAFKA_CONSUMER_BOOTSTRAP_SERVERS: kafka:9092 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + load-generator: + image: ghcr.io/cau-se/theodolite-uc2-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator:5701 + PORT: 5701 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + NUM_SENSORS: 10 diff --git a/theodolite-benchmarks/docker-test/uc3-beam-flink/docker-compose.yml b/theodolite-benchmarks/docker-test/uc3-beam-flink/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..9a18ab364463a985b40cd691f6232b9b47ae412e --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc3-beam-flink/docker-compose.yml @@ -0,0 +1,78 @@ +version: '2.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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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-uc3-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator: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-beam-flink:latest + #ports: + # - "8080:8081" + command: > + standalone-job --job-classname application.Uc3BeamFlink + --disableMetrics=true + --fasterCopy + environment: + - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 + - SCHEMA_REGISTRY_URL=http://schema-registry:8081 + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + parallelism.default: 1 + state.backend: rocksdb + state.checkpoints.dir: file:///data/flink/checkpoints + depends_on: + - schema-registry + - kafka + benchmark-taskmanager: + image: ghcr.io/cau-se/theodolite-uc3-beam-flink:latest + scale: 1 + command: taskmanager + environment: + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + state.backend: rocksdb + state.checkpoints.dir: file:///data/flink/checkpoints + depends_on: + - schema-registry + - kafka diff --git a/theodolite-benchmarks/docker-test/uc3-beam-samza/docker-compose.yml b/theodolite-benchmarks/docker-test/uc3-beam-samza/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..a50b32bd8f78678d63f06688821d6dfb5f133138 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc3-beam-samza/docker-compose.yml @@ -0,0 +1,59 @@ +version: '2.2' +services: + zookeeper: + image: confluentinc/cp-zookeeper + expose: + - "2181" + #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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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' + benchmark: + image: ghcr.io/cau-se/theodolite-uc3-beam-samza:latest + scale: 1 + depends_on: + - schema-registry + - kafka + environment: + SAMZA_JOB_COORDINATOR_ZK_CONNECT: zookeeper:2181 + SAMZA_SYSTEMS_KAFKA_PRODUCER_BOOTSTRAP_SERVERS: kafka:9092 + SAMZA_SYSTEMS_KAFKA_CONSUMER_BOOTSTRAP_SERVERS: kafka:9092 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + load-generator: + image: ghcr.io/cau-se/theodolite-uc3-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator:5701 + PORT: 5701 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + NUM_SENSORS: 10 diff --git a/theodolite-benchmarks/docker-test/uc4-beam-flink/docker-compose.yml b/theodolite-benchmarks/docker-test/uc4-beam-flink/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..5169ac551952f992d98c74f7d65d5378ecdcc2a5 --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc4-beam-flink/docker-compose.yml @@ -0,0 +1,80 @@ +version: '2.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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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: load-generator: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-beam-flink:latest + #ports: + # - "8080:8081" + command: > + standalone-job + --job-classname application.Uc4BeamFlink + --disableMetrics=true + --fasterCopy + environment: + - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 + - SCHEMA_REGISTRY_URL=http://schema-registry:8081 + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + parallelism.default: 1 + state.backend: rocksdb + state.checkpoints.dir: file:///data/flink/checkpoints + depends_on: + - schema-registry + - kafka + benchmark-taskmanager: + image: ghcr.io/cau-se/theodolite-uc4-beam-flink:latest + scale: 1 + command: taskmanager + environment: + - | + FLINK_PROPERTIES= + jobmanager.rpc.address: benchmark-jobmanager + state.backend: rocksdb + state.checkpoints.dir: file:///data/flink/checkpoints + depends_on: + - schema-registry + - kafka diff --git a/theodolite-benchmarks/docker-test/uc4-beam-samza/docker-compose.yml b/theodolite-benchmarks/docker-test/uc4-beam-samza/docker-compose.yml new file mode 100644 index 0000000000000000000000000000000000000000..bded9d5d227d0f62cb6cb3f9edac3df383ea3e8a --- /dev/null +++ b/theodolite-benchmarks/docker-test/uc4-beam-samza/docker-compose.yml @@ -0,0 +1,60 @@ +version: '2.2' +services: + zookeeper: + image: confluentinc/cp-zookeeper + expose: + - "2181" + #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_AUTO_CREATE_TOPICS_ENABLE: "false" + 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' + benchmark: + image: ghcr.io/cau-se/theodolite-uc4-beam-samza:latest + scale: 1 + depends_on: + - schema-registry + - kafka + environment: + SAMZA_JOB_COORDINATOR_ZK_CONNECT: zookeeper:2181 + SAMZA_SYSTEMS_KAFKA_PRODUCER_BOOTSTRAP_SERVERS: kafka:9092 + SAMZA_SYSTEMS_KAFKA_CONSUMER_BOOTSTRAP_SERVERS: kafka:9092 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + load-generator: + image: ghcr.io/cau-se/theodolite-uc4-workload-generator:latest + depends_on: + - schema-registry + - kafka + environment: + BOOTSTRAP_SERVER: load-generator:5701 + PORT: 5701 + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_URL: http://schema-registry:8081 + NUM_SENSORS: 4 + NUM_NESTED_GROUPS: 4 diff --git a/theodolite-benchmarks/settings.gradle b/theodolite-benchmarks/settings.gradle index 5602e816bb21dce72162b085de99836b8f9aea1e..ae4254e968a0bc09970752f95c6a40db86ae775c 100644 --- a/theodolite-benchmarks/settings.gradle +++ b/theodolite-benchmarks/settings.gradle @@ -3,19 +3,34 @@ rootProject.name = 'theodolite-benchmarks' include 'load-generator-commons' include 'kstreams-commons' include 'flink-commons' +include 'beam-commons' + +include 'uc1-beam' +include 'uc2-beam' +include 'uc3-beam' +include 'uc4-beam' include 'uc1-load-generator' include 'uc1-kstreams' include 'uc1-flink' +include 'uc1-beam-flink' +include 'uc1-beam-samza' include 'uc2-load-generator' include 'uc2-kstreams' include 'uc2-flink' +include 'uc2-beam-flink' +include 'uc2-beam-samza' include 'uc3-load-generator' include 'uc3-kstreams' include 'uc3-flink' +include 'uc3-beam-flink' +include 'uc3-beam-samza' include 'uc4-load-generator' include 'uc4-kstreams' include 'uc4-flink' +include 'uc4-beam-flink' +include 'uc4-beam-samza' + diff --git a/theodolite-benchmarks/uc1-beam-flink/Dockerfile b/theodolite-benchmarks/uc1-beam-flink/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..0e3c0b3184e4bac8b62b97bc022df88c7701b619 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-flink/Dockerfile @@ -0,0 +1,5 @@ +FROM flink:1.13-java11 + +ADD build/distributions/uc1-beam-flink.tar /opt/flink/usrlib/artifacts/uc1-beam-flink.tar + + diff --git a/theodolite-benchmarks/uc1-beam-flink/build.gradle b/theodolite-benchmarks/uc1-beam-flink/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..f4b6cff8efbcdbcb701f249220643669f0f89626 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-flink/build.gradle @@ -0,0 +1,9 @@ +plugins { + id 'theodolite.beam.flink' +} + +dependencies { + implementation project(':uc1-beam') +} + +mainClassName = "application.Uc1BeamFlink" diff --git a/theodolite-benchmarks/uc1-beam-flink/src/main/java/application/Uc1BeamFlink.java b/theodolite-benchmarks/uc1-beam-flink/src/main/java/application/Uc1BeamFlink.java new file mode 100644 index 0000000000000000000000000000000000000000..fe58369b3c0c19351bcc5cde170df68946af7cbd --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-flink/src/main/java/application/Uc1BeamFlink.java @@ -0,0 +1,40 @@ +package application; + +import org.apache.beam.runners.flink.FlinkRunner; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Database Storage using Apache Beam with the Flink Runner. To + * execute locally in standalone start Kafka, Zookeeper, the schema-registry and the workload + * generator using the delayed_startup.sh script. Start a Flink cluster and pass its REST adress + * using--flinkMaster as run parameter. To persist logs add + * ${workspace_loc:/uc1-application-samza/eclipseConsoleLogs.log} as Output File under Standard + * Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc1BeamFlink extends AbstractBeamService { + + /** + * Private constructor setting specific options for this use case. + */ + private Uc1BeamFlink(final String[] args) { //NOPMD + super(args); + this.options.setRunner(FlinkRunner.class); + } + + /** + * Main method. + */ + public static void main(final String[] args) { + + // Create application via configurations + final Uc1BeamFlink uc1 = new Uc1BeamFlink(args); + + // Create pipeline with configurations + final Uc1BeamPipeline pipeline = new Uc1BeamPipeline(uc1.options, uc1.getConfig()); + + // Submit job and start execution + pipeline.run().waitUntilFinish(); + } + +} + diff --git a/theodolite-benchmarks/uc1-beam-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc1-beam-flink/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..50db1510ab5d7f6b8c9b1a75f112719209c351ce --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-flink/src/main/resources/META-INF/application.properties @@ -0,0 +1,16 @@ +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 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc1-beam-samza/.gitignore b/theodolite-benchmarks/uc1-beam-samza/.gitignore new file mode 100644 index 0000000000000000000000000000000000000000..7bf05dd280fcc888467656ce1fbdeb65322c7ba8 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/.gitignore @@ -0,0 +1 @@ +state \ No newline at end of file diff --git a/theodolite-benchmarks/uc1-beam-samza/Dockerfile b/theodolite-benchmarks/uc1-beam-samza/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..9b729060532ea3a242ac3084ba0bebf88ca2e9b6 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/Dockerfile @@ -0,0 +1,9 @@ +FROM openjdk:11-slim + +ENV MAX_SOURCE_PARALLELISM=1024 + +ADD build/distributions/uc1-beam-samza.tar / +ADD samza-standalone.properties / + +CMD /uc1-beam-samza/bin/uc1-beam-samza --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory --configFilePath=samza-standalone.properties --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=$MAX_SOURCE_PARALLELISM --enableMetrics=false --configOverride="{\"job.coordinator.zk.connect\":\"$SAMZA_JOB_COORDINATOR_ZK_CONNECT\"}" + diff --git a/theodolite-benchmarks/uc1-beam-samza/build.gradle b/theodolite-benchmarks/uc1-beam-samza/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..54c070d967d17ccd41c85f90486655c9fd56b65b --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.samza' +} + +dependencies { + implementation project(':uc1-beam') +} + + +mainClassName = "application.Uc1BeamSamza" diff --git a/theodolite-benchmarks/uc1-beam-samza/samza-standalone.properties b/theodolite-benchmarks/uc1-beam-samza/samza-standalone.properties new file mode 100644 index 0000000000000000000000000000000000000000..02411017e86e274f6057688a7c351567603c0f80 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/samza-standalone.properties @@ -0,0 +1,17 @@ +# Set EnvironmentRewriter +job.config.rewriters=env-config +job.config.rewriter.env-config.class=org.apache.samza.config.EnvironmentConfigRewriter + +# Configure ZooKeeper for coordination +job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory +job.coordinator.zk.connect=localhost:2181 + +# Use GroupByContainerIds +task.name.grouper.factory=org.apache.samza.container.grouper.task.GroupByContainerIdsFactory + +# Configure Kafka as "system" +job.default.system=kafka +systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory +systems.kafka.consumer.bootstrap.servers=localhost:9092 +systems.kafka.producer.bootstrap.servers=localhost:9092 +systems.kafka.default.stream.replication.factor=1 diff --git a/theodolite-benchmarks/uc1-beam-samza/src/main/java/application/Uc1BeamSamza.java b/theodolite-benchmarks/uc1-beam-samza/src/main/java/application/Uc1BeamSamza.java new file mode 100644 index 0000000000000000000000000000000000000000..aaef5c2d6968c4b89059537277a2582ecca70451 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/src/main/java/application/Uc1BeamSamza.java @@ -0,0 +1,43 @@ +package application; + +import org.apache.beam.runners.samza.SamzaRunner; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Database Storage using Apache Beam with the Samza Runner. To + * execute locally in standalone start Kafka, Zookeeper, the schema-registry and the workload + * generator. Add + * --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory + * --configFilePath=samza-standalone.properties + * --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=1024 as program arguments. To + * persist logs add ${workspace_loc:/uc4-application-samza/eclipseConsoleLogs.log} as Output File + * under Standard Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc1BeamSamza extends AbstractBeamService { + + /** + * Private constructor setting specific options for this use case. + */ + private Uc1BeamSamza(final String[] args) { //NOPMD + super(args); + this.options.setRunner(SamzaRunner.class); + } + + /** + * Main method. + */ + public static void main(final String[] args) { + + // Create application via configurations + final Uc1BeamSamza uc1 = new Uc1BeamSamza(args); + + // Create pipeline with configurations + final Uc1BeamPipeline pipeline = new Uc1BeamPipeline(uc1.options, uc1.getConfig()); + + // Submit job and start execution + pipeline.run().waitUntilFinish(); + } +} + + + diff --git a/theodolite-benchmarks/uc1-beam-samza/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc1-beam-samza/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..50db1510ab5d7f6b8c9b1a75f112719209c351ce --- /dev/null +++ b/theodolite-benchmarks/uc1-beam-samza/src/main/resources/META-INF/application.properties @@ -0,0 +1,16 @@ +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 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc1-beam/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc1-beam/.settings/org.eclipse.jdt.ui.prefs new file mode 100644 index 0000000000000000000000000000000000000000..da2db2cefa90c0d974068e22804132eb6c11d824 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam/.settings/org.eclipse.jdt.ui.prefs @@ -0,0 +1,283 @@ +cleanup.add_all=false +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.array_with_curly=false +cleanup.arrays_fill=false +cleanup.bitwise_conditional_expression=false +cleanup.boolean_literal=false +cleanup.boolean_value_rather_than_comparison=true +cleanup.break_loop=false +cleanup.collection_cloning=false +cleanup.comparing_on_criteria=false +cleanup.comparison_statement=false +cleanup.controlflow_merge=false +cleanup.convert_functional_interfaces=false +cleanup.convert_to_enhanced_for_loop=true +cleanup.convert_to_enhanced_for_loop_if_loop_var_used=true +cleanup.convert_to_switch_expressions=false +cleanup.correct_indentation=true +cleanup.do_while_rather_than_while=true +cleanup.double_negation=false +cleanup.else_if=false +cleanup.embedded_if=false +cleanup.evaluate_nullable=false +cleanup.extract_increment=false +cleanup.format_source_code=true +cleanup.format_source_code_changes_only=false +cleanup.hash=false +cleanup.if_condition=false +cleanup.insert_inferred_type_arguments=false +cleanup.instanceof=false +cleanup.instanceof_keyword=false +cleanup.invert_equals=false +cleanup.join=false +cleanup.lazy_logical_operator=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.map_cloning=false +cleanup.merge_conditional_blocks=false +cleanup.multi_catch=false +cleanup.never_use_blocks=false +cleanup.never_use_parentheses_in_expressions=true +cleanup.no_string_creation=false +cleanup.no_super=false +cleanup.number_suffix=false +cleanup.objects_equals=false +cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=true +cleanup.operand_factorization=false +cleanup.organize_imports=true +cleanup.overridden_assignment=false +cleanup.plain_replacement=false +cleanup.precompile_regex=false +cleanup.primitive_comparison=false +cleanup.primitive_parsing=false +cleanup.primitive_rather_than_wrapper=true +cleanup.primitive_serialization=false +cleanup.pull_out_if_from_if_else=false +cleanup.pull_up_assignment=false +cleanup.push_down_negation=false +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.reduce_indentation=false +cleanup.redundant_comparator=false +cleanup.redundant_falling_through_block_end=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_array_creation=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.return_expression=false +cleanup.simplify_lambda_expression_and_method_ref=false +cleanup.single_used_field=false +cleanup.sort_members=false +cleanup.sort_members_all=false +cleanup.standard_comparison=false +cleanup.static_inner_class=false +cleanup.strictly_equal_or_different=false +cleanup.stringbuffer_to_stringbuilder=false +cleanup.stringbuilder=false +cleanup.stringbuilder_for_local_vars=true +cleanup.substring=false +cleanup.switch=false +cleanup.system_property=false +cleanup.system_property_boolean=false +cleanup.system_property_file_encoding=false +cleanup.system_property_file_separator=false +cleanup.system_property_line_separator=false +cleanup.system_property_path_separator=false +cleanup.ternary_operator=false +cleanup.try_with_resource=false +cleanup.unlooped_while=false +cleanup.unreachable_block=false +cleanup.use_anonymous_class_creation=false +cleanup.use_autoboxing=false +cleanup.use_blocks=true +cleanup.use_blocks_only_for_return_and_throw=false +cleanup.use_directly_map_method=false +cleanup.use_lambda=true +cleanup.use_parentheses_in_expressions=true +cleanup.use_string_is_blank=false +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.use_unboxing=false +cleanup.use_var=false +cleanup.useless_continue=false +cleanup.useless_return=false +cleanup.valueof_rather_than_instantiation=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=21 +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_all=false +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.array_with_curly=false +sp_cleanup.arrays_fill=false +sp_cleanup.bitwise_conditional_expression=false +sp_cleanup.boolean_literal=false +sp_cleanup.boolean_value_rather_than_comparison=false +sp_cleanup.break_loop=false +sp_cleanup.collection_cloning=false +sp_cleanup.comparing_on_criteria=false +sp_cleanup.comparison_statement=false +sp_cleanup.controlflow_merge=false +sp_cleanup.convert_functional_interfaces=false +sp_cleanup.convert_to_enhanced_for_loop=false +sp_cleanup.convert_to_enhanced_for_loop_if_loop_var_used=false +sp_cleanup.convert_to_switch_expressions=false +sp_cleanup.correct_indentation=false +sp_cleanup.do_while_rather_than_while=false +sp_cleanup.double_negation=false +sp_cleanup.else_if=false +sp_cleanup.embedded_if=false +sp_cleanup.evaluate_nullable=false +sp_cleanup.extract_increment=false +sp_cleanup.format_source_code=true +sp_cleanup.format_source_code_changes_only=false +sp_cleanup.hash=false +sp_cleanup.if_condition=false +sp_cleanup.insert_inferred_type_arguments=false +sp_cleanup.instanceof=false +sp_cleanup.instanceof_keyword=false +sp_cleanup.invert_equals=false +sp_cleanup.join=false +sp_cleanup.lazy_logical_operator=false +sp_cleanup.make_local_variable_final=true +sp_cleanup.make_parameters_final=false +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.map_cloning=false +sp_cleanup.merge_conditional_blocks=false +sp_cleanup.multi_catch=false +sp_cleanup.never_use_blocks=false +sp_cleanup.never_use_parentheses_in_expressions=true +sp_cleanup.no_string_creation=false +sp_cleanup.no_super=false +sp_cleanup.number_suffix=false +sp_cleanup.objects_equals=false +sp_cleanup.on_save_use_additional_actions=true +sp_cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=false +sp_cleanup.operand_factorization=false +sp_cleanup.organize_imports=true +sp_cleanup.overridden_assignment=false +sp_cleanup.plain_replacement=false +sp_cleanup.precompile_regex=false +sp_cleanup.primitive_comparison=false +sp_cleanup.primitive_parsing=false +sp_cleanup.primitive_rather_than_wrapper=false +sp_cleanup.primitive_serialization=false +sp_cleanup.pull_out_if_from_if_else=false +sp_cleanup.pull_up_assignment=false +sp_cleanup.push_down_negation=false +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.reduce_indentation=false +sp_cleanup.redundant_comparator=false +sp_cleanup.redundant_falling_through_block_end=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=false +sp_cleanup.remove_trailing_whitespaces_all=true +sp_cleanup.remove_trailing_whitespaces_ignore_empty=false +sp_cleanup.remove_unnecessary_array_creation=false +sp_cleanup.remove_unnecessary_casts=true +sp_cleanup.remove_unnecessary_nls_tags=false +sp_cleanup.remove_unused_imports=false +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.return_expression=false +sp_cleanup.simplify_lambda_expression_and_method_ref=false +sp_cleanup.single_used_field=false +sp_cleanup.sort_members=false +sp_cleanup.sort_members_all=false +sp_cleanup.standard_comparison=false +sp_cleanup.static_inner_class=false +sp_cleanup.strictly_equal_or_different=false +sp_cleanup.stringbuffer_to_stringbuilder=false +sp_cleanup.stringbuilder=false +sp_cleanup.stringbuilder_for_local_vars=true +sp_cleanup.substring=false +sp_cleanup.switch=false +sp_cleanup.system_property=false +sp_cleanup.system_property_boolean=false +sp_cleanup.system_property_file_encoding=false +sp_cleanup.system_property_file_separator=false +sp_cleanup.system_property_line_separator=false +sp_cleanup.system_property_path_separator=false +sp_cleanup.ternary_operator=false +sp_cleanup.try_with_resource=true +sp_cleanup.unlooped_while=false +sp_cleanup.unreachable_block=false +sp_cleanup.use_anonymous_class_creation=false +sp_cleanup.use_autoboxing=false +sp_cleanup.use_blocks=true +sp_cleanup.use_blocks_only_for_return_and_throw=false +sp_cleanup.use_directly_map_method=false +sp_cleanup.use_lambda=true +sp_cleanup.use_parentheses_in_expressions=true +sp_cleanup.use_string_is_blank=false +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 +sp_cleanup.use_unboxing=false +sp_cleanup.use_var=false +sp_cleanup.useless_continue=false +sp_cleanup.useless_return=false +sp_cleanup.valueof_rather_than_instantiation=false diff --git a/theodolite-benchmarks/uc1-beam/build.gradle b/theodolite-benchmarks/uc1-beam/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..502e94fa737fb2ae1bab861407b27575cd8766ca --- /dev/null +++ b/theodolite-benchmarks/uc1-beam/build.gradle @@ -0,0 +1,5 @@ +plugins { + id 'theodolite.beam' +} + + diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/application/LogKeyValue.java b/theodolite-benchmarks/uc1-beam/src/main/java/application/LogKeyValue.java new file mode 100644 index 0000000000000000000000000000000000000000..79566fd937b9c100663d426610b6ff476035ef87 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam/src/main/java/application/LogKeyValue.java @@ -0,0 +1,27 @@ +package application; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Logs all Key Value pairs. + */ +@SuppressWarnings({"unused"}) +public class LogKeyValue extends DoFn<KV<String, String>, KV<String, String>> { + private static final long serialVersionUID = 4328743; + private static final Logger LOGGER = LoggerFactory.getLogger(LogKeyValue.class); + + /** + * Logs all key value pairs it processes. + */ + @ProcessElement + public void processElement(@Element final KV<String, String> kv, + final OutputReceiver<KV<String, String>> out) { + if (LOGGER.isInfoEnabled()) { + LOGGER.info("Key: {}, Value: {}", kv.getKey(), kv.getValue()); + } + out.output(kv); + } +} diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/application/MapToGson.java b/theodolite-benchmarks/uc1-beam/src/main/java/application/MapToGson.java new file mode 100644 index 0000000000000000000000000000000000000000..6b0c6bc4ddfe78c22028da5b8cf7dde7ed57fced --- /dev/null +++ b/theodolite-benchmarks/uc1-beam/src/main/java/application/MapToGson.java @@ -0,0 +1,26 @@ +package application; + +import com.google.gson.Gson; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import titan.ccp.model.records.ActivePowerRecord; + +/** + * Converts a Map into a json String. + */ +public class MapToGson extends SimpleFunction<KV<String, ActivePowerRecord>, KV<String, String>> { + private static final long serialVersionUID = 7168356203579050214L; + private transient Gson gsonObj = new Gson(); + + @Override + public KV<String, String> apply( + final KV<String, ActivePowerRecord> kv) { + + if (this.gsonObj == null) { + this.gsonObj = new Gson(); + } + + final String gson = this.gsonObj.toJson(kv.getValue()); + return KV.of(kv.getKey(), gson); + } +} diff --git a/theodolite-benchmarks/uc1-beam/src/main/java/application/Uc1BeamPipeline.java b/theodolite-benchmarks/uc1-beam/src/main/java/application/Uc1BeamPipeline.java new file mode 100644 index 0000000000000000000000000000000000000000..eaff08ac78cd18ddfd47eb2949ca13340ecc27b8 --- /dev/null +++ b/theodolite-benchmarks/uc1-beam/src/main/java/application/Uc1BeamPipeline.java @@ -0,0 +1,52 @@ +package application; + +import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.commons.configuration2.Configuration; +import theodolite.commons.beam.AbstractPipeline; +import theodolite.commons.beam.kafka.KafkaActivePowerTimestampReader; +import titan.ccp.model.records.ActivePowerRecord; + + +/** + * Implementation of the use case Database Storage using Apache Beam with the Flink Runner. To + * execute locally in standalone start Kafka, Zookeeper, the schema-registry and the workload + * generator using the delayed_startup.sh script. Start a Flink cluster and pass its REST adress + * using--flinkMaster as run parameter. To persist logs add + * ${workspace_loc:/uc1-application-samza/eclipseConsoleLogs.log} as Output File under Standard + * Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc1BeamPipeline extends AbstractPipeline { + + protected Uc1BeamPipeline(final PipelineOptions options, final Configuration config) { + super(options, config); + + // Set Coders for Classes that will be distributed + final CoderRegistry cr = this.getCoderRegistry(); + cr.registerCoderForClass(ActivePowerRecord.class, AvroCoder.of(ActivePowerRecord.SCHEMA$)); + + // build KafkaConsumerConfig + final Map<String, Object> consumerConfig = this.buildConsumerConfig(); + + // Create Pipeline transformations + final KafkaActivePowerTimestampReader kafka = + new KafkaActivePowerTimestampReader(this.bootstrapServer, this.inputTopic, consumerConfig); + + final LogKeyValue logKeyValue = new LogKeyValue(); + final MapToGson mapToGson = new MapToGson(); + + // Apply pipeline transformations + // Read from Kafka + this.apply(kafka) + // Map to Gson + .apply(MapElements + .via(mapToGson)) + // Print to console + .apply(ParDo.of(logKeyValue)); + } +} + diff --git a/theodolite-benchmarks/uc2-beam-flink/Dockerfile b/theodolite-benchmarks/uc2-beam-flink/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..b2ad8ff9036eef0f0cd0efad23d4f92abef7cc42 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-flink/Dockerfile @@ -0,0 +1,4 @@ +FROM flink:1.13-java11 + +ADD build/distributions/uc2-beam-flink.tar /opt/flink/usrlib/artifacts/uc2-beam-flink.tar + diff --git a/theodolite-benchmarks/uc2-beam-flink/build.gradle b/theodolite-benchmarks/uc2-beam-flink/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..9ab898cd465abe20e855d06ebf85373e46ab12e2 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-flink/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.flink' +} + + +dependencies { + implementation project(':uc2-beam') +} + +mainClassName = "application.Uc2BeamFlink" diff --git a/theodolite-benchmarks/uc2-beam-flink/src/main/java/application/Uc2BeamFlink.java b/theodolite-benchmarks/uc2-beam-flink/src/main/java/application/Uc2BeamFlink.java new file mode 100644 index 0000000000000000000000000000000000000000..f5bb849e626444929e00b17b1324a08c41cb19a0 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-flink/src/main/java/application/Uc2BeamFlink.java @@ -0,0 +1,35 @@ +package application; + +import org.apache.beam.runners.flink.FlinkRunner; +import org.apache.beam.sdk.Pipeline; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Downsampling using Apache Beam with the Flink Runner. To execute + * locally in standalone start Kafka, Zookeeper, the schema-registry and the workload generator + * using the delayed_startup.sh script. Start a Flink cluster and pass its REST adress + * using--flinkMaster as run parameter. + */ +public final class Uc2BeamFlink extends AbstractBeamService { + + /** + * Private constructor setting specific options for this use case. + */ + private Uc2BeamFlink(final String[] args) { // NOPMD + super(args); + this.options.setRunner(FlinkRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc2BeamFlink uc2BeamFlink = new Uc2BeamFlink(args); + + final Pipeline pipeline = new Uc2BeamPipeline(uc2BeamFlink.options, uc2BeamFlink.getConfig()); + + pipeline.run().waitUntilFinish(); + } +} + diff --git a/theodolite-benchmarks/uc2-beam-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc2-beam-flink/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..1545a0f6630c8ea51d694f4056ca3aa750463f5b --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-flink/src/main/resources/META-INF/application.properties @@ -0,0 +1,17 @@ +application.name=theodolite-uc2-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc2-beam-samza/.gitignore b/theodolite-benchmarks/uc2-beam-samza/.gitignore new file mode 100644 index 0000000000000000000000000000000000000000..7bf05dd280fcc888467656ce1fbdeb65322c7ba8 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/.gitignore @@ -0,0 +1 @@ +state \ No newline at end of file diff --git a/theodolite-benchmarks/uc2-beam-samza/Dockerfile b/theodolite-benchmarks/uc2-beam-samza/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..22855cea279819cacbf6eee253c30c60409fdba3 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/Dockerfile @@ -0,0 +1,8 @@ +FROM openjdk:11-slim + +ENV MAX_SOURCE_PARALLELISM=1024 + +ADD build/distributions/uc2-beam-samza.tar / +ADD samza-standalone.properties / + +CMD /uc2-beam-samza/bin/uc2-beam-samza --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory --configFilePath=samza-standalone.properties --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=$MAX_SOURCE_PARALLELISM --enableMetrics=false --configOverride="{\"job.coordinator.zk.connect\":\"$SAMZA_JOB_COORDINATOR_ZK_CONNECT\"}" diff --git a/theodolite-benchmarks/uc2-beam-samza/build.gradle b/theodolite-benchmarks/uc2-beam-samza/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..29d7f9ac0c22c421072646ca665b3849c558d56f --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.samza' +} + + +dependencies { + implementation project(':uc2-beam') +} + +mainClassName = "application.Uc2BeamSamza" diff --git a/theodolite-benchmarks/uc2-beam-samza/samza-standalone.properties b/theodolite-benchmarks/uc2-beam-samza/samza-standalone.properties new file mode 100644 index 0000000000000000000000000000000000000000..02411017e86e274f6057688a7c351567603c0f80 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/samza-standalone.properties @@ -0,0 +1,17 @@ +# Set EnvironmentRewriter +job.config.rewriters=env-config +job.config.rewriter.env-config.class=org.apache.samza.config.EnvironmentConfigRewriter + +# Configure ZooKeeper for coordination +job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory +job.coordinator.zk.connect=localhost:2181 + +# Use GroupByContainerIds +task.name.grouper.factory=org.apache.samza.container.grouper.task.GroupByContainerIdsFactory + +# Configure Kafka as "system" +job.default.system=kafka +systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory +systems.kafka.consumer.bootstrap.servers=localhost:9092 +systems.kafka.producer.bootstrap.servers=localhost:9092 +systems.kafka.default.stream.replication.factor=1 diff --git a/theodolite-benchmarks/uc2-beam-samza/src/main/java/application/Uc2BeamSamza.java b/theodolite-benchmarks/uc2-beam-samza/src/main/java/application/Uc2BeamSamza.java new file mode 100644 index 0000000000000000000000000000000000000000..d4b3d6d910824a718bffe8dc5f0204d53b9865c1 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/src/main/java/application/Uc2BeamSamza.java @@ -0,0 +1,39 @@ +package application; + +import org.apache.beam.runners.samza.SamzaRunner; +import org.apache.beam.sdk.Pipeline; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Downsampling using Apache Beam with the Samza Runner. To run + * locally in standalone start Kafka, Zookeeper, the schema-registry and the workload generator + * using the delayed_startup.sh script. Add + * --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory + * --configFilePath=${workspace_loc:uc3-application-samza}/config/standalone_local.properties + * --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=1024 --as program arguments. To + * persist logs add ${workspace_loc:/uc3-application-samza/eclipseConsoleLogs.log} as Output File + * under Standard Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc2BeamSamza extends AbstractBeamService { + + /** + * Private constructor setting specific options for this use case. + */ + private Uc2BeamSamza(final String[] args) { //NOPMD + super(args); + this.options.setRunner(SamzaRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc2BeamSamza uc2BeamSamza = new Uc2BeamSamza(args); + + final Pipeline pipeline = new Uc2BeamPipeline(uc2BeamSamza.options, uc2BeamSamza.getConfig()); + + pipeline.run().waitUntilFinish(); + } +} + diff --git a/theodolite-benchmarks/uc2-beam-samza/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc2-beam-samza/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..1545a0f6630c8ea51d694f4056ca3aa750463f5b --- /dev/null +++ b/theodolite-benchmarks/uc2-beam-samza/src/main/resources/META-INF/application.properties @@ -0,0 +1,17 @@ +application.name=theodolite-uc2-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc2-beam/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc2-beam/.settings/org.eclipse.jdt.ui.prefs new file mode 100644 index 0000000000000000000000000000000000000000..43ccd3dbcdf80e49b8920c8fe242b35c3f604281 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam/.settings/org.eclipse.jdt.ui.prefs @@ -0,0 +1,283 @@ +cleanup.add_all=false +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.array_with_curly=false +cleanup.arrays_fill=false +cleanup.bitwise_conditional_expression=false +cleanup.boolean_literal=false +cleanup.boolean_value_rather_than_comparison=true +cleanup.break_loop=false +cleanup.collection_cloning=false +cleanup.comparing_on_criteria=false +cleanup.comparison_statement=false +cleanup.controlflow_merge=false +cleanup.convert_functional_interfaces=false +cleanup.convert_to_enhanced_for_loop=true +cleanup.convert_to_enhanced_for_loop_if_loop_var_used=true +cleanup.convert_to_switch_expressions=false +cleanup.correct_indentation=true +cleanup.do_while_rather_than_while=true +cleanup.double_negation=false +cleanup.else_if=false +cleanup.embedded_if=false +cleanup.evaluate_nullable=false +cleanup.extract_increment=false +cleanup.format_source_code=true +cleanup.format_source_code_changes_only=false +cleanup.hash=false +cleanup.if_condition=false +cleanup.insert_inferred_type_arguments=false +cleanup.instanceof=false +cleanup.instanceof_keyword=false +cleanup.invert_equals=false +cleanup.join=false +cleanup.lazy_logical_operator=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.map_cloning=false +cleanup.merge_conditional_blocks=false +cleanup.multi_catch=false +cleanup.never_use_blocks=false +cleanup.never_use_parentheses_in_expressions=true +cleanup.no_string_creation=false +cleanup.no_super=false +cleanup.number_suffix=false +cleanup.objects_equals=false +cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=true +cleanup.operand_factorization=false +cleanup.organize_imports=true +cleanup.overridden_assignment=false +cleanup.plain_replacement=false +cleanup.precompile_regex=false +cleanup.primitive_comparison=false +cleanup.primitive_parsing=false +cleanup.primitive_rather_than_wrapper=true +cleanup.primitive_serialization=false +cleanup.pull_out_if_from_if_else=false +cleanup.pull_up_assignment=false +cleanup.push_down_negation=false +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.reduce_indentation=false +cleanup.redundant_comparator=false +cleanup.redundant_falling_through_block_end=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_array_creation=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.return_expression=false +cleanup.simplify_lambda_expression_and_method_ref=false +cleanup.single_used_field=false +cleanup.sort_members=false +cleanup.sort_members_all=false +cleanup.standard_comparison=false +cleanup.static_inner_class=false +cleanup.strictly_equal_or_different=false +cleanup.stringbuffer_to_stringbuilder=false +cleanup.stringbuilder=false +cleanup.stringbuilder_for_local_vars=true +cleanup.substring=false +cleanup.switch=false +cleanup.system_property=false +cleanup.system_property_boolean=false +cleanup.system_property_file_encoding=false +cleanup.system_property_file_separator=false +cleanup.system_property_line_separator=false +cleanup.system_property_path_separator=false +cleanup.ternary_operator=false +cleanup.try_with_resource=false +cleanup.unlooped_while=false +cleanup.unreachable_block=false +cleanup.use_anonymous_class_creation=false +cleanup.use_autoboxing=false +cleanup.use_blocks=true +cleanup.use_blocks_only_for_return_and_throw=false +cleanup.use_directly_map_method=false +cleanup.use_lambda=true +cleanup.use_parentheses_in_expressions=true +cleanup.use_string_is_blank=false +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.use_unboxing=false +cleanup.use_var=false +cleanup.useless_continue=false +cleanup.useless_return=false +cleanup.valueof_rather_than_instantiation=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=21 +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_all=false +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=false +sp_cleanup.always_use_this_for_non_static_method_access=false +sp_cleanup.array_with_curly=false +sp_cleanup.arrays_fill=false +sp_cleanup.bitwise_conditional_expression=false +sp_cleanup.boolean_literal=false +sp_cleanup.boolean_value_rather_than_comparison=false +sp_cleanup.break_loop=false +sp_cleanup.collection_cloning=false +sp_cleanup.comparing_on_criteria=false +sp_cleanup.comparison_statement=false +sp_cleanup.controlflow_merge=false +sp_cleanup.convert_functional_interfaces=false +sp_cleanup.convert_to_enhanced_for_loop=true +sp_cleanup.convert_to_enhanced_for_loop_if_loop_var_used=false +sp_cleanup.convert_to_switch_expressions=false +sp_cleanup.correct_indentation=true +sp_cleanup.do_while_rather_than_while=false +sp_cleanup.double_negation=false +sp_cleanup.else_if=true +sp_cleanup.embedded_if=false +sp_cleanup.evaluate_nullable=false +sp_cleanup.extract_increment=false +sp_cleanup.format_source_code=true +sp_cleanup.format_source_code_changes_only=false +sp_cleanup.hash=false +sp_cleanup.if_condition=false +sp_cleanup.insert_inferred_type_arguments=false +sp_cleanup.instanceof=false +sp_cleanup.instanceof_keyword=false +sp_cleanup.invert_equals=false +sp_cleanup.join=false +sp_cleanup.lazy_logical_operator=false +sp_cleanup.make_local_variable_final=true +sp_cleanup.make_parameters_final=false +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.map_cloning=false +sp_cleanup.merge_conditional_blocks=false +sp_cleanup.multi_catch=false +sp_cleanup.never_use_blocks=false +sp_cleanup.never_use_parentheses_in_expressions=true +sp_cleanup.no_string_creation=false +sp_cleanup.no_super=false +sp_cleanup.number_suffix=false +sp_cleanup.objects_equals=false +sp_cleanup.on_save_use_additional_actions=true +sp_cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=false +sp_cleanup.operand_factorization=false +sp_cleanup.organize_imports=true +sp_cleanup.overridden_assignment=false +sp_cleanup.plain_replacement=false +sp_cleanup.precompile_regex=false +sp_cleanup.primitive_comparison=false +sp_cleanup.primitive_parsing=false +sp_cleanup.primitive_rather_than_wrapper=false +sp_cleanup.primitive_serialization=false +sp_cleanup.pull_out_if_from_if_else=false +sp_cleanup.pull_up_assignment=false +sp_cleanup.push_down_negation=false +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.reduce_indentation=false +sp_cleanup.redundant_comparator=false +sp_cleanup.redundant_falling_through_block_end=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_array_creation=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.return_expression=false +sp_cleanup.simplify_lambda_expression_and_method_ref=false +sp_cleanup.single_used_field=false +sp_cleanup.sort_members=false +sp_cleanup.sort_members_all=false +sp_cleanup.standard_comparison=false +sp_cleanup.static_inner_class=false +sp_cleanup.strictly_equal_or_different=false +sp_cleanup.stringbuffer_to_stringbuilder=false +sp_cleanup.stringbuilder=false +sp_cleanup.stringbuilder_for_local_vars=true +sp_cleanup.substring=false +sp_cleanup.switch=false +sp_cleanup.system_property=false +sp_cleanup.system_property_boolean=false +sp_cleanup.system_property_file_encoding=false +sp_cleanup.system_property_file_separator=false +sp_cleanup.system_property_line_separator=false +sp_cleanup.system_property_path_separator=false +sp_cleanup.ternary_operator=false +sp_cleanup.try_with_resource=false +sp_cleanup.unlooped_while=false +sp_cleanup.unreachable_block=false +sp_cleanup.use_anonymous_class_creation=false +sp_cleanup.use_autoboxing=false +sp_cleanup.use_blocks=true +sp_cleanup.use_blocks_only_for_return_and_throw=false +sp_cleanup.use_directly_map_method=false +sp_cleanup.use_lambda=true +sp_cleanup.use_parentheses_in_expressions=true +sp_cleanup.use_string_is_blank=false +sp_cleanup.use_this_for_non_static_field_access=true +sp_cleanup.use_this_for_non_static_field_access_only_if_necessary=true +sp_cleanup.use_this_for_non_static_method_access=true +sp_cleanup.use_this_for_non_static_method_access_only_if_necessary=true +sp_cleanup.use_unboxing=false +sp_cleanup.use_var=false +sp_cleanup.useless_continue=false +sp_cleanup.useless_return=false +sp_cleanup.valueof_rather_than_instantiation=false diff --git a/theodolite-benchmarks/uc2-beam/build.gradle b/theodolite-benchmarks/uc2-beam/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..502e94fa737fb2ae1bab861407b27575cd8766ca --- /dev/null +++ b/theodolite-benchmarks/uc2-beam/build.gradle @@ -0,0 +1,5 @@ +plugins { + id 'theodolite.beam' +} + + diff --git a/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsAggregation.java b/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsAggregation.java new file mode 100644 index 0000000000000000000000000000000000000000..688f6677ec6d74e063a07a20c079b783aa71c399 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsAggregation.java @@ -0,0 +1,45 @@ +package application; + +import com.google.common.math.Stats; +import com.google.common.math.StatsAccumulator; +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; + +/** + * Aggregation Class for ActivePowerRecords. Creates a StatsAccumulator based on the ValueInW. + */ + +@DefaultCoder(AvroCoder.class) +public class StatsAggregation extends CombineFn<ActivePowerRecord, StatsAccumulator, Stats> + implements Serializable { + + private static final long serialVersionUID = 1L; + + @Override + public StatsAccumulator createAccumulator() { + return new StatsAccumulator(); + } + + @Override + public StatsAccumulator addInput(final StatsAccumulator accum, final ActivePowerRecord input) { + accum.add(input.getValueInW()); + return accum; + } + + @Override + public StatsAccumulator mergeAccumulators(final Iterable<StatsAccumulator> accums) { + final StatsAccumulator merged = createAccumulator(); + for (final StatsAccumulator accum : accums) { + merged.addAll(accum.snapshot()); + } + return merged; + } + + @Override + public Stats extractOutput(final StatsAccumulator accum) { + return accum.snapshot(); + } +} diff --git a/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsToString.java b/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsToString.java new file mode 100644 index 0000000000000000000000000000000000000000..9a73ae8e1681b2c350dee328cba7283cf3386fd7 --- /dev/null +++ b/theodolite-benchmarks/uc2-beam/src/main/java/application/StatsToString.java @@ -0,0 +1,18 @@ +package application; + +import com.google.common.math.Stats; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; + +/** + * Transforms a {@code KV<String, Stats>} into a {@code KV<String, String>}. + */ +public class StatsToString extends SimpleFunction<KV<String, Stats>, KV<String, String>> { + private static final long serialVersionUID = 4308991244493097240L; + + @Override + public KV<String, String> apply(final KV<String, Stats> kv) { + return KV.of(kv.getKey(), kv.getValue().toString()); + } + +} diff --git a/theodolite-benchmarks/uc2-beam/src/main/java/application/Uc2BeamPipeline.java b/theodolite-benchmarks/uc2-beam/src/main/java/application/Uc2BeamPipeline.java new file mode 100644 index 0000000000000000000000000000000000000000..02eec9868b0bbfbf6fd45206ff0d4092ac09e1ac --- /dev/null +++ b/theodolite-benchmarks/uc2-beam/src/main/java/application/Uc2BeamPipeline.java @@ -0,0 +1,73 @@ +package application; + +import com.google.common.math.Stats; +import com.google.common.math.StatsAccumulator; +import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.configuration2.Configuration; +import org.apache.kafka.common.serialization.StringSerializer; +import org.joda.time.Duration; +import theodolite.commons.beam.AbstractPipeline; +import theodolite.commons.beam.ConfigurationKeys; +import theodolite.commons.beam.kafka.KafkaActivePowerTimestampReader; +import theodolite.commons.beam.kafka.KafkaWriterTransformation; +import titan.ccp.model.records.ActivePowerRecord; + + +/** + * Implementation of the use case Downsampling using Apache Beam. + */ +public final class Uc2BeamPipeline extends AbstractPipeline { + + protected Uc2BeamPipeline(final PipelineOptions options, final Configuration config) { + super(options, config); + // Additional needed variables + final String outputTopic = config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC); + + final Duration duration = + Duration.standardMinutes(config.getInt(ConfigurationKeys.KAFKA_WINDOW_DURATION_MINUTES)); + + // Build kafka configuration + final Map<String, Object> consumerConfig = buildConsumerConfig(); + + // Set Coders for Classes that will be distributed + final CoderRegistry cr = getCoderRegistry(); + cr.registerCoderForClass(ActivePowerRecord.class, AvroCoder.of(ActivePowerRecord.SCHEMA$)); + cr.registerCoderForClass(StatsAggregation.class, SerializableCoder.of(StatsAggregation.class)); + cr.registerCoderForClass(StatsAccumulator.class, AvroCoder.of(StatsAccumulator.class)); + + // Read from Kafka + final KafkaActivePowerTimestampReader kafkaActivePowerRecordReader = + new KafkaActivePowerTimestampReader(bootstrapServer, inputTopic, consumerConfig); + + // Transform into String + final StatsToString statsToString = new StatsToString(); + + // Write to Kafka + final KafkaWriterTransformation<String> kafkaWriter = + new KafkaWriterTransformation<>(bootstrapServer, outputTopic, StringSerializer.class); + + // Apply pipeline transformations + this.apply(kafkaActivePowerRecordReader) + // Apply a fixed window + .apply(Window.<KV<String, ActivePowerRecord>>into(FixedWindows.of(duration))) + // Aggregate per window for every key + .apply(Combine.<String, ActivePowerRecord, Stats>perKey(new StatsAggregation())) + .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializableCoder.of(Stats.class))) + // Map into correct output format + .apply(MapElements.via(statsToString)) + // Write to Kafka + .apply(kafkaWriter); + } +} + diff --git a/theodolite-benchmarks/uc3-beam-flink/Dockerfile b/theodolite-benchmarks/uc3-beam-flink/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..3c0b4d96b8a59e45da3b85e8d7cd238b98cacfd7 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-flink/Dockerfile @@ -0,0 +1,4 @@ +FROM flink:1.13-java11 + +ADD build/distributions/uc3-beam-flink.tar /opt/flink/usrlib/artifacts/uc3-beam-flink.tar + diff --git a/theodolite-benchmarks/uc3-beam-flink/build.gradle b/theodolite-benchmarks/uc3-beam-flink/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..8f047c6dce50636f01a7cdf645722aa5f7ac9ce9 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-flink/build.gradle @@ -0,0 +1,12 @@ +plugins { + id 'theodolite.beam.flink' +} + + +dependencies { + implementation project(':uc3-beam') +} + + +// This is the path of the main class, stored within ./src/main/java/ +mainClassName = 'application.Uc3BeamFlink' diff --git a/theodolite-benchmarks/uc3-beam-flink/src/main/java/application/Uc3BeamFlink.java b/theodolite-benchmarks/uc3-beam-flink/src/main/java/application/Uc3BeamFlink.java new file mode 100644 index 0000000000000000000000000000000000000000..18532b2655fcc6c24dad5f2fca87607c0b5d2e54 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-flink/src/main/java/application/Uc3BeamFlink.java @@ -0,0 +1,39 @@ +package application; + +import org.apache.beam.runners.flink.FlinkRunner; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Aggregation based on Time Attributes using Apache Beam with the + * Flink Runner. To run locally in standalone start Kafka, Zookeeper, the schema-registry and the + * workload generator using the delayed_startup.sh script. And configure the Kafka, Zookeeper and + * Schema Registry urls accordingly. Start a Flink cluster and pass its REST adress + * using--flinkMaster as run parameter. To persist logs add + * ${workspace_loc:/uc4-application-samza/eclipseConsoleLogs.log} as Output File under Standard + * Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc3BeamFlink extends AbstractBeamService { + + /** + * Private constructor to avoid instantiation. + */ + private Uc3BeamFlink(final String[] args) { //NOPMD + super(args); + this.options.setRunner(FlinkRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc3BeamFlink uc3BeamFlink = new Uc3BeamFlink(args); + + final Uc3BeamPipeline pipeline = + new Uc3BeamPipeline(uc3BeamFlink.options, uc3BeamFlink.getConfig()); + + pipeline.run().waitUntilFinish(); + } + +} + diff --git a/theodolite-benchmarks/uc3-beam-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc3-beam-flink/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..2db723927eaee10d39e02a6b2d369a06af7711fc --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-flink/src/main/resources/META-INF/application.properties @@ -0,0 +1,22 @@ +application.name=theodolite-uc3-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +aggregation.duration.days=30 +aggregation.advance.days=1 + +trigger.interval=15 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc3-beam-samza/.gitignore b/theodolite-benchmarks/uc3-beam-samza/.gitignore new file mode 100644 index 0000000000000000000000000000000000000000..7bf05dd280fcc888467656ce1fbdeb65322c7ba8 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/.gitignore @@ -0,0 +1 @@ +state \ No newline at end of file diff --git a/theodolite-benchmarks/uc3-beam-samza/Dockerfile b/theodolite-benchmarks/uc3-beam-samza/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..d3e860bd72c54121d616bc5562d519e6e1e21dec --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/Dockerfile @@ -0,0 +1,8 @@ +FROM openjdk:11-slim + +ENV MAX_SOURCE_PARALLELISM=1024 + +ADD build/distributions/uc3-beam-samza.tar / +ADD samza-standalone.properties / + +CMD /uc3-beam-samza/bin/uc3-beam-samza --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory --configFilePath=samza-standalone.properties --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=$MAX_SOURCE_PARALLELISM --enableMetrics=false --configOverride="{\"job.coordinator.zk.connect\":\"$SAMZA_JOB_COORDINATOR_ZK_CONNECT\"}" diff --git a/theodolite-benchmarks/uc3-beam-samza/build.gradle b/theodolite-benchmarks/uc3-beam-samza/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..513b850330f4f71f440ad5da4ecea95f092f5ccc --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.samza' +} + + +dependencies { + implementation project(':uc3-beam') +} + +mainClassName = "application.Uc3BeamSamza" \ No newline at end of file diff --git a/theodolite-benchmarks/uc3-beam-samza/samza-standalone.properties b/theodolite-benchmarks/uc3-beam-samza/samza-standalone.properties new file mode 100644 index 0000000000000000000000000000000000000000..02411017e86e274f6057688a7c351567603c0f80 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/samza-standalone.properties @@ -0,0 +1,17 @@ +# Set EnvironmentRewriter +job.config.rewriters=env-config +job.config.rewriter.env-config.class=org.apache.samza.config.EnvironmentConfigRewriter + +# Configure ZooKeeper for coordination +job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory +job.coordinator.zk.connect=localhost:2181 + +# Use GroupByContainerIds +task.name.grouper.factory=org.apache.samza.container.grouper.task.GroupByContainerIdsFactory + +# Configure Kafka as "system" +job.default.system=kafka +systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory +systems.kafka.consumer.bootstrap.servers=localhost:9092 +systems.kafka.producer.bootstrap.servers=localhost:9092 +systems.kafka.default.stream.replication.factor=1 diff --git a/theodolite-benchmarks/uc3-beam-samza/src/main/java/application/Uc3BeamSamza.java b/theodolite-benchmarks/uc3-beam-samza/src/main/java/application/Uc3BeamSamza.java new file mode 100644 index 0000000000000000000000000000000000000000..913293bd02cb16e14ee9d94ea0e161c74853e72a --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/src/main/java/application/Uc3BeamSamza.java @@ -0,0 +1,39 @@ +package application; + +import org.apache.beam.runners.samza.SamzaRunner; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Aggregation based on Time Attributes using Apache Beam with the + * Samza Runner. To run locally in standalone start Kafka, Zookeeper, the schema-registry and the + * workload generator using the delayed_startup.sh script. And configure the Kafka, Zookeeper and + * Schema Registry urls accordingly. Start a Flink cluster and pass its REST adress + * using--flinkMaster as run parameter. To persist logs add + * ${workspace_loc:/uc4-application-samza/eclipseConsoleLogs.log} as Output File under Standard + * Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc3BeamSamza extends AbstractBeamService { + + /** + * Private constructor to avoid instantiation. + */ + private Uc3BeamSamza(final String[] args) { //NOPMD + super(args); + this.options.setRunner(SamzaRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc3BeamSamza uc3BeamSamza = new Uc3BeamSamza(args); + + final Uc3BeamPipeline pipeline = + new Uc3BeamPipeline(uc3BeamSamza.options, uc3BeamSamza.getConfig()); + + pipeline.run().waitUntilFinish(); + } + +} + diff --git a/theodolite-benchmarks/uc3-beam-samza/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc3-beam-samza/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..2db723927eaee10d39e02a6b2d369a06af7711fc --- /dev/null +++ b/theodolite-benchmarks/uc3-beam-samza/src/main/resources/META-INF/application.properties @@ -0,0 +1,22 @@ +application.name=theodolite-uc3-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +aggregation.duration.days=30 +aggregation.advance.days=1 + +trigger.interval=15 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc3-beam/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc3-beam/.settings/org.eclipse.jdt.ui.prefs new file mode 100644 index 0000000000000000000000000000000000000000..d71754b65d8da3cee7e6e440f49aa833ddabae10 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/.settings/org.eclipse.jdt.ui.prefs @@ -0,0 +1,283 @@ +cleanup.add_all=false +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.array_with_curly=false +cleanup.arrays_fill=false +cleanup.bitwise_conditional_expression=false +cleanup.boolean_literal=false +cleanup.boolean_value_rather_than_comparison=true +cleanup.break_loop=false +cleanup.collection_cloning=false +cleanup.comparing_on_criteria=false +cleanup.comparison_statement=false +cleanup.controlflow_merge=false +cleanup.convert_functional_interfaces=false +cleanup.convert_to_enhanced_for_loop=true +cleanup.convert_to_enhanced_for_loop_if_loop_var_used=true +cleanup.convert_to_switch_expressions=false +cleanup.correct_indentation=true +cleanup.do_while_rather_than_while=true +cleanup.double_negation=false +cleanup.else_if=false +cleanup.embedded_if=false +cleanup.evaluate_nullable=false +cleanup.extract_increment=false +cleanup.format_source_code=true +cleanup.format_source_code_changes_only=false +cleanup.hash=false +cleanup.if_condition=false +cleanup.insert_inferred_type_arguments=false +cleanup.instanceof=false +cleanup.instanceof_keyword=false +cleanup.invert_equals=false +cleanup.join=false +cleanup.lazy_logical_operator=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.map_cloning=false +cleanup.merge_conditional_blocks=false +cleanup.multi_catch=false +cleanup.never_use_blocks=false +cleanup.never_use_parentheses_in_expressions=true +cleanup.no_string_creation=false +cleanup.no_super=false +cleanup.number_suffix=false +cleanup.objects_equals=false +cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=true +cleanup.operand_factorization=false +cleanup.organize_imports=true +cleanup.overridden_assignment=false +cleanup.plain_replacement=false +cleanup.precompile_regex=false +cleanup.primitive_comparison=false +cleanup.primitive_parsing=false +cleanup.primitive_rather_than_wrapper=true +cleanup.primitive_serialization=false +cleanup.pull_out_if_from_if_else=false +cleanup.pull_up_assignment=false +cleanup.push_down_negation=false +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.reduce_indentation=false +cleanup.redundant_comparator=false +cleanup.redundant_falling_through_block_end=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_array_creation=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.return_expression=false +cleanup.simplify_lambda_expression_and_method_ref=false +cleanup.single_used_field=false +cleanup.sort_members=false +cleanup.sort_members_all=false +cleanup.standard_comparison=false +cleanup.static_inner_class=false +cleanup.strictly_equal_or_different=false +cleanup.stringbuffer_to_stringbuilder=false +cleanup.stringbuilder=false +cleanup.stringbuilder_for_local_vars=true +cleanup.substring=false +cleanup.switch=false +cleanup.system_property=false +cleanup.system_property_boolean=false +cleanup.system_property_file_encoding=false +cleanup.system_property_file_separator=false +cleanup.system_property_line_separator=false +cleanup.system_property_path_separator=false +cleanup.ternary_operator=false +cleanup.try_with_resource=false +cleanup.unlooped_while=false +cleanup.unreachable_block=false +cleanup.use_anonymous_class_creation=false +cleanup.use_autoboxing=false +cleanup.use_blocks=true +cleanup.use_blocks_only_for_return_and_throw=false +cleanup.use_directly_map_method=false +cleanup.use_lambda=true +cleanup.use_parentheses_in_expressions=true +cleanup.use_string_is_blank=false +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.use_unboxing=false +cleanup.use_var=false +cleanup.useless_continue=false +cleanup.useless_return=false +cleanup.valueof_rather_than_instantiation=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=21 +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_all=false +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.array_with_curly=false +sp_cleanup.arrays_fill=false +sp_cleanup.bitwise_conditional_expression=false +sp_cleanup.boolean_literal=false +sp_cleanup.boolean_value_rather_than_comparison=false +sp_cleanup.break_loop=false +sp_cleanup.collection_cloning=false +sp_cleanup.comparing_on_criteria=false +sp_cleanup.comparison_statement=false +sp_cleanup.controlflow_merge=false +sp_cleanup.convert_functional_interfaces=false +sp_cleanup.convert_to_enhanced_for_loop=false +sp_cleanup.convert_to_enhanced_for_loop_if_loop_var_used=false +sp_cleanup.convert_to_switch_expressions=false +sp_cleanup.correct_indentation=true +sp_cleanup.do_while_rather_than_while=false +sp_cleanup.double_negation=false +sp_cleanup.else_if=false +sp_cleanup.embedded_if=false +sp_cleanup.evaluate_nullable=false +sp_cleanup.extract_increment=false +sp_cleanup.format_source_code=true +sp_cleanup.format_source_code_changes_only=false +sp_cleanup.hash=false +sp_cleanup.if_condition=false +sp_cleanup.insert_inferred_type_arguments=false +sp_cleanup.instanceof=false +sp_cleanup.instanceof_keyword=false +sp_cleanup.invert_equals=false +sp_cleanup.join=false +sp_cleanup.lazy_logical_operator=false +sp_cleanup.make_local_variable_final=true +sp_cleanup.make_parameters_final=false +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.map_cloning=false +sp_cleanup.merge_conditional_blocks=false +sp_cleanup.multi_catch=false +sp_cleanup.never_use_blocks=false +sp_cleanup.never_use_parentheses_in_expressions=true +sp_cleanup.no_string_creation=false +sp_cleanup.no_super=false +sp_cleanup.number_suffix=false +sp_cleanup.objects_equals=false +sp_cleanup.on_save_use_additional_actions=true +sp_cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=false +sp_cleanup.operand_factorization=false +sp_cleanup.organize_imports=true +sp_cleanup.overridden_assignment=false +sp_cleanup.plain_replacement=false +sp_cleanup.precompile_regex=false +sp_cleanup.primitive_comparison=false +sp_cleanup.primitive_parsing=false +sp_cleanup.primitive_rather_than_wrapper=false +sp_cleanup.primitive_serialization=false +sp_cleanup.pull_out_if_from_if_else=false +sp_cleanup.pull_up_assignment=false +sp_cleanup.push_down_negation=false +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.reduce_indentation=false +sp_cleanup.redundant_comparator=false +sp_cleanup.redundant_falling_through_block_end=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_array_creation=false +sp_cleanup.remove_unnecessary_casts=true +sp_cleanup.remove_unnecessary_nls_tags=false +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.return_expression=false +sp_cleanup.simplify_lambda_expression_and_method_ref=false +sp_cleanup.single_used_field=false +sp_cleanup.sort_members=false +sp_cleanup.sort_members_all=false +sp_cleanup.standard_comparison=false +sp_cleanup.static_inner_class=false +sp_cleanup.strictly_equal_or_different=false +sp_cleanup.stringbuffer_to_stringbuilder=false +sp_cleanup.stringbuilder=false +sp_cleanup.stringbuilder_for_local_vars=true +sp_cleanup.substring=false +sp_cleanup.switch=false +sp_cleanup.system_property=false +sp_cleanup.system_property_boolean=false +sp_cleanup.system_property_file_encoding=false +sp_cleanup.system_property_file_separator=false +sp_cleanup.system_property_line_separator=false +sp_cleanup.system_property_path_separator=false +sp_cleanup.ternary_operator=false +sp_cleanup.try_with_resource=true +sp_cleanup.unlooped_while=false +sp_cleanup.unreachable_block=false +sp_cleanup.use_anonymous_class_creation=false +sp_cleanup.use_autoboxing=false +sp_cleanup.use_blocks=true +sp_cleanup.use_blocks_only_for_return_and_throw=false +sp_cleanup.use_directly_map_method=false +sp_cleanup.use_lambda=true +sp_cleanup.use_parentheses_in_expressions=true +sp_cleanup.use_string_is_blank=false +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 +sp_cleanup.use_unboxing=false +sp_cleanup.use_var=false +sp_cleanup.useless_continue=false +sp_cleanup.useless_return=false +sp_cleanup.valueof_rather_than_instantiation=false diff --git a/theodolite-benchmarks/uc3-beam/build.gradle b/theodolite-benchmarks/uc3-beam/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..502e94fa737fb2ae1bab861407b27575cd8766ca --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/build.gradle @@ -0,0 +1,5 @@ +plugins { + id 'theodolite.beam' +} + + diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKey.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKey.java new file mode 100644 index 0000000000000000000000000000000000000000..6db59dd65ee494157400b0f4c6eafbdd7655d402 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKey.java @@ -0,0 +1,34 @@ +package application; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; + + +/** + * Composed key of an hour of the day and a sensor id. + */ +@DefaultCoder(AvroCoder.class) +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; + } + +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKeyFactory.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKeyFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..b993a0199bc13e9b416f9b9cb77a27635d7fe1e1 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKeyFactory.java @@ -0,0 +1,24 @@ +package application; + +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 = 1L; + + @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-beam/src/main/java/application/HourOfDayKeySerde.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKeySerde.java new file mode 100644 index 0000000000000000000000000000000000000000..a0f8e0bbaf959154bfbab69d83da56c81d55802a --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayKeySerde.java @@ -0,0 +1,32 @@ +package application; + +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 implements BufferSerde<HourOfDayKey> { + + @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()); + } + +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayWithStats.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayWithStats.java new file mode 100644 index 0000000000000000000000000000000000000000..46232b3f13601d77f6cb7b13ea0bcdc31290357a --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDayWithStats.java @@ -0,0 +1,19 @@ +package application; + +import com.google.common.math.Stats; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; + +/** + * {@link SimpleFunction} that transforms into the sensorId and the Value. + */ +public class HourOfDayWithStats extends + SimpleFunction<KV<HourOfDayKey, Stats>, KV<String, String>> { + private static final long serialVersionUID = -7411154345437422919L; + private final HourOfDayKeyFactory keyFactory = new HourOfDayKeyFactory(); + + @Override + public KV<String, String> apply(final KV<HourOfDayKey, Stats> kv) { + return KV.of(keyFactory.getSensorId(kv.getKey()), kv.getValue().toString()); + } +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDaykeyCoder.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDaykeyCoder.java new file mode 100644 index 0000000000000000000000000000000000000000..196408263ec29a1ec9b45375dd0b53a18e8f60b3 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/HourOfDaykeyCoder.java @@ -0,0 +1,60 @@ +package application; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Collections; +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; + +/** + * Wrapper Class that encapsulates a HourOfDayKeySerde in a org.apache.beam.sdk.coders.Coder. + */ +public class HourOfDaykeyCoder extends Coder<HourOfDayKey> implements Serializable { + public static final long serialVersionUID = 4444444; + private static final boolean DETERMINISTIC = true; + private static final int VALUE_SIZE = 4; + + private transient Serde<HourOfDayKey> innerSerde = HourOfDayKeySerde.create(); + + @Override + public void encode(final HourOfDayKey value, final OutputStream outStream) + throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = HourOfDayKeySerde.create(); + } + final byte[] bytes = this.innerSerde.serializer().serialize("ser", value); + final byte[] sizeinBytes = ByteBuffer.allocate(VALUE_SIZE).putInt(bytes.length).array(); + outStream.write(sizeinBytes); + outStream.write(bytes); + } + + @Override + public HourOfDayKey decode(final InputStream inStream) throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = HourOfDayKeySerde.create(); + } + final byte[] sizeinBytes = new byte[VALUE_SIZE]; + inStream.read(sizeinBytes); + final int size = ByteBuffer.wrap(sizeinBytes).getInt(); + final byte[] bytes = new byte[size]; + inStream.read(bytes); + return this.innerSerde.deserializer().deserialize("deser", bytes); + } + + @Override + public List<? extends Coder<?>> getCoderArguments() { + return Collections.emptyList(); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!DETERMINISTIC) { + throw new NonDeterministicException(this, "This class is not deterministic!"); + } + } +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/MapTimeFormat.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/MapTimeFormat.java new file mode 100644 index 0000000000000000000000000000000000000000..7d8897fb0fd76cd4eb145da6a7ce031f9f45d396 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/MapTimeFormat.java @@ -0,0 +1,27 @@ +package application; + +import java.time.Instant; +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; + +/** + * Changes the time format to us europe/paris time. + */ +public class MapTimeFormat + extends SimpleFunction<KV<String, ActivePowerRecord>, KV<HourOfDayKey, ActivePowerRecord>> { + private static final long serialVersionUID = -6597391279968647035L; + private final StatsKeyFactory<HourOfDayKey> keyFactory = new HourOfDayKeyFactory(); + private final ZoneId zone = ZoneId.of("Europe/Paris"); + + @Override + public KV<HourOfDayKey, ActivePowerRecord> apply( + final KV<String, ActivePowerRecord> kv) { + final Instant instant = Instant.ofEpochMilli(kv.getValue().getTimestamp()); + final LocalDateTime dateTime = LocalDateTime.ofInstant(instant, this.zone); + return KV.of(this.keyFactory.createKey(kv.getValue().getIdentifier(), dateTime), + kv.getValue()); + } +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsAggregation.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsAggregation.java new file mode 100644 index 0000000000000000000000000000000000000000..ee5cfc48bcd42dec41dd2030ad3f4a730fd6ac85 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsAggregation.java @@ -0,0 +1,45 @@ +package application; + +import com.google.common.math.Stats; +import com.google.common.math.StatsAccumulator; +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; + + +/** + * Aggregation Class for ActivePowerRecords. Creates a StatsAccumulator based on the ValueInW. + */ + +@DefaultCoder(AvroCoder.class) +public class StatsAggregation extends CombineFn<ActivePowerRecord, StatsAccumulator, Stats> + implements Serializable { + private static final long serialVersionUID = 1L; + + @Override + public StatsAccumulator createAccumulator() { + return new StatsAccumulator(); + } + + @Override + public StatsAccumulator addInput(final StatsAccumulator accum, final ActivePowerRecord input) { + accum.add(input.getValueInW()); + return accum; + } + + @Override + public StatsAccumulator mergeAccumulators(final Iterable<StatsAccumulator> accums) { + final StatsAccumulator merged = this.createAccumulator(); + for (final StatsAccumulator accum : accums) { + merged.addAll(accum.snapshot()); + } + return merged; + } + + @Override + public Stats extractOutput(final StatsAccumulator accum) { + return accum.snapshot(); + } +} diff --git a/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsKeyFactory.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsKeyFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..820168058f88ca21f8efcf61c7ebed60c08aa200 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/StatsKeyFactory.java @@ -0,0 +1,17 @@ +package application; + +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-beam/src/main/java/application/Uc3BeamPipeline.java b/theodolite-benchmarks/uc3-beam/src/main/java/application/Uc3BeamPipeline.java new file mode 100644 index 0000000000000000000000000000000000000000..c402271777dd63026e1f1fb36855dad1a72e1136 --- /dev/null +++ b/theodolite-benchmarks/uc3-beam/src/main/java/application/Uc3BeamPipeline.java @@ -0,0 +1,102 @@ +package application; + +import com.google.common.math.Stats; +import com.google.common.math.StatsAccumulator; +import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.configuration2.Configuration; +import org.apache.kafka.common.serialization.StringSerializer; +import org.joda.time.Duration; +import theodolite.commons.beam.AbstractPipeline; +import theodolite.commons.beam.ConfigurationKeys; +import theodolite.commons.beam.kafka.KafkaActivePowerTimestampReader; +import theodolite.commons.beam.kafka.KafkaWriterTransformation; +import titan.ccp.model.records.ActivePowerRecord; + + +/** + * Implementation of the use case Aggregation based on Time Attributes using Apache Beam. + */ +public final class Uc3BeamPipeline extends AbstractPipeline { + + protected Uc3BeamPipeline(final PipelineOptions options, final Configuration config) { + super(options, config); + // Additional needed variables + final String outputTopic = config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC); + + final Duration duration = + Duration.standardDays(config.getInt(ConfigurationKeys.AGGREGATION_DURATION_DAYS)); + final Duration aggregationAdvanceDuration = + Duration.standardDays(config.getInt(ConfigurationKeys.AGGREGATION_ADVANCE_DAYS)); + final Duration triggerDelay = + Duration.standardSeconds(config.getInt(ConfigurationKeys.TRIGGER_INTERVAL)); + + // Build Kafka configuration + final Map<String, Object> consumerConfig = this.buildConsumerConfig(); + + // Set Coders for classes that will be distributed + final CoderRegistry cr = this.getCoderRegistry(); + registerCoders(cr); + + // Read from Kafka + final KafkaActivePowerTimestampReader kafka = + new KafkaActivePowerTimestampReader(this.bootstrapServer, this.inputTopic, consumerConfig); + + // Map the time format + final MapTimeFormat mapTimeFormat = new MapTimeFormat(); + + // Get the stats per HourOfDay + final HourOfDayWithStats hourOfDayWithStats = new HourOfDayWithStats(); + + // Write to Kafka + final KafkaWriterTransformation<String> kafkaWriter = + new KafkaWriterTransformation<>(this.bootstrapServer, outputTopic, StringSerializer.class); + + this.apply(kafka) + // Map to correct time format + .apply(MapElements.via(mapTimeFormat)) + // Apply a sliding window + .apply(Window + .<KV<HourOfDayKey, ActivePowerRecord>>into( + SlidingWindows.of(duration).every(aggregationAdvanceDuration)) + .triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf(triggerDelay))) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()) + + // Aggregate per window for every key + .apply(Combine.<HourOfDayKey, ActivePowerRecord, Stats>perKey(new StatsAggregation())) + .setCoder(KvCoder.of(new HourOfDaykeyCoder(), SerializableCoder.of(Stats.class))) + + // Map into correct output format + .apply(MapElements.via(hourOfDayWithStats)) + // Write to Kafka + .apply(kafkaWriter); + } + + + /** + * Registers all Coders for all needed Coders. + * + * @param cr CoderRegistry. + */ + private static void registerCoders(final CoderRegistry cr) { + cr.registerCoderForClass(ActivePowerRecord.class, AvroCoder.of(ActivePowerRecord.SCHEMA$)); + cr.registerCoderForClass(HourOfDayKey.class, new HourOfDaykeyCoder()); + cr.registerCoderForClass(StatsAggregation.class, SerializableCoder.of(StatsAggregation.class)); + cr.registerCoderForClass(StatsAccumulator.class, AvroCoder.of(StatsAccumulator.class)); + } +} + diff --git a/theodolite-benchmarks/uc4-beam-flink/Dockerfile b/theodolite-benchmarks/uc4-beam-flink/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..9c238ddbaccbe9040571e18ac8ad8eef5b7ecf15 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-flink/Dockerfile @@ -0,0 +1,4 @@ +FROM flink:1.13-java11 + +ADD build/distributions/uc4-beam-flink.tar /opt/flink/usrlib/artifacts/uc4-beam-flink.tar + diff --git a/theodolite-benchmarks/uc4-beam-flink/build.gradle b/theodolite-benchmarks/uc4-beam-flink/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..7ace89a2275e29e22186f7b67dcb7816cc7a85d0 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-flink/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.flink' +} + + +dependencies { + implementation project(':uc4-beam') +} + +mainClassName = 'application.Uc4BeamFlink' \ No newline at end of file diff --git a/theodolite-benchmarks/uc4-beam-flink/src/main/java/application/Uc4BeamFlink.java b/theodolite-benchmarks/uc4-beam-flink/src/main/java/application/Uc4BeamFlink.java new file mode 100644 index 0000000000000000000000000000000000000000..90f9a4a292e99526fa94c7dd512bdcec548fbb4f --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-flink/src/main/java/application/Uc4BeamFlink.java @@ -0,0 +1,34 @@ +package application; + +import org.apache.beam.runners.flink.FlinkRunner; +import org.apache.beam.sdk.Pipeline; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Hierarchical Aggregation using Apache Beam with the Flink + * Runner. + **/ +public final class Uc4BeamFlink extends AbstractBeamService { + + + /** + * Private constructor setting specific options for this use case. + */ + private Uc4BeamFlink(final String[] args) { //NOPMD + super(args); + this.options.setRunner(FlinkRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc4BeamFlink uc4BeamFlink = new Uc4BeamFlink(args); + + final Pipeline pipeline = new Uc4BeamPipeline(uc4BeamFlink.options, uc4BeamFlink.getConfig()); + + pipeline.run().waitUntilFinish(); + } + +} diff --git a/theodolite-benchmarks/uc4-beam-flink/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc4-beam-flink/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..bc679580dadf969e181b6787e8287066426be7e2 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-flink/src/main/resources/META-INF/application.properties @@ -0,0 +1,25 @@ +application.name=theodolite-uc4-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.configuration.topic=configuration +kafka.feedback.topic=aggregation-feedback +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +aggregation.duration.days=30 +aggregation.advance.days=1 + +trigger.interval=15 +grace.period.ms=270 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc4-beam-samza/.gitignore b/theodolite-benchmarks/uc4-beam-samza/.gitignore new file mode 100644 index 0000000000000000000000000000000000000000..7bf05dd280fcc888467656ce1fbdeb65322c7ba8 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/.gitignore @@ -0,0 +1 @@ +state \ No newline at end of file diff --git a/theodolite-benchmarks/uc4-beam-samza/Dockerfile b/theodolite-benchmarks/uc4-beam-samza/Dockerfile new file mode 100644 index 0000000000000000000000000000000000000000..ffb82ee09e99384d4914a0f86b6d9214fc161381 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/Dockerfile @@ -0,0 +1,8 @@ +FROM openjdk:11-slim + +ENV MAX_SOURCE_PARALLELISM=1024 + +ADD build/distributions/uc4-beam-samza.tar / +ADD samza-standalone.properties / + +CMD /uc4-beam-samza/bin/uc4-beam-samza --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory --configFilePath=samza-standalone.properties --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=$MAX_SOURCE_PARALLELISM --enableMetrics=false --configOverride="{\"job.coordinator.zk.connect\":\"$SAMZA_JOB_COORDINATOR_ZK_CONNECT\"}" diff --git a/theodolite-benchmarks/uc4-beam-samza/build.gradle b/theodolite-benchmarks/uc4-beam-samza/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..90dcc76b0cae5aeca61f76d0cb73347b2407408b --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/build.gradle @@ -0,0 +1,10 @@ +plugins { + id 'theodolite.beam.samza' +} + +dependencies { + implementation project(':uc4-beam') +} + + +mainClassName = "application.Uc4BeamSamza" diff --git a/theodolite-benchmarks/uc4-beam-samza/samza-standalone.properties b/theodolite-benchmarks/uc4-beam-samza/samza-standalone.properties new file mode 100644 index 0000000000000000000000000000000000000000..812a9784c25ea84922b8a0f96f17b0377cedf925 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/samza-standalone.properties @@ -0,0 +1,23 @@ +# Set EnvironmentRewriter +job.config.rewriters=env-config +job.config.rewriter.env-config.class=org.apache.samza.config.EnvironmentConfigRewriter + +# Configure ZooKeeper for coordination +job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory +job.coordinator.zk.connect=localhost:2181 + +# Use GroupByContainerIds +task.name.grouper.factory=org.apache.samza.container.grouper.task.GroupByContainerIdsFactory + +# Configure Kafka as "system" +job.default.system=kafka +systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory +systems.kafka.consumer.bootstrap.servers=localhost:9092 +systems.kafka.producer.bootstrap.servers=localhost:9092 +systems.kafka.default.stream.replication.factor=1 + +# Configure serialization and stores +serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory +stores.my-store.factory=org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory +stores.my-store.key.serde=string +stores.my-store.msg.serde=string diff --git a/theodolite-benchmarks/uc4-beam-samza/src/main/java/application/Uc4BeamSamza.java b/theodolite-benchmarks/uc4-beam-samza/src/main/java/application/Uc4BeamSamza.java new file mode 100644 index 0000000000000000000000000000000000000000..3894fa95f16253e0a165dde70bf25d4a4bee96cb --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/src/main/java/application/Uc4BeamSamza.java @@ -0,0 +1,40 @@ +package application; + +import org.apache.beam.runners.samza.SamzaRunner; +import org.apache.beam.sdk.Pipeline; +import theodolite.commons.beam.AbstractBeamService; + +/** + * Implementation of the use case Hierarchical Aggregation using Apache Beam with the Samza + * Runner. To run locally in standalone start Kafka, Zookeeper, the schema-registry and the + * workload generator using the delayed_startup.sh script. Add + * --configFactory=org.apache.samza.config.factories.PropertiesConfigFactory + * --configFilePath=${workspace_loc:uc4-application-samza}/config/standalone_local.properties + * --samzaExecutionEnvironment=STANDALONE --maxSourceParallelism=1024 --as program arguments. To + * persist logs add ${workspace_loc:/uc4-application-samza/eclipseConsoleLogs.log} as Output File + * under Standard Input Output in Common in the Run Configuration Start via Eclipse Run. + */ +public final class Uc4BeamSamza extends AbstractBeamService { + + + /** + * Private constructor setting specific options for this use case. + */ + private Uc4BeamSamza(final String[] args) { //NOPMD + super(args); + this.options.setRunner(SamzaRunner.class); + } + + /** + * Start running this microservice. + */ + public static void main(final String[] args) { + + final Uc4BeamSamza uc4BeamSamza = new Uc4BeamSamza(args); + + final Pipeline pipeline = new Uc4BeamPipeline(uc4BeamSamza.options, uc4BeamSamza.getConfig()); + + pipeline.run().waitUntilFinish(); + } + +} diff --git a/theodolite-benchmarks/uc4-beam-samza/src/main/resources/META-INF/application.properties b/theodolite-benchmarks/uc4-beam-samza/src/main/resources/META-INF/application.properties new file mode 100644 index 0000000000000000000000000000000000000000..bc679580dadf969e181b6787e8287066426be7e2 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam-samza/src/main/resources/META-INF/application.properties @@ -0,0 +1,25 @@ +application.name=theodolite-uc4-application +application.version=0.0.1 + +kafka.bootstrap.servers=localhost:9092 +kafka.input.topic=input +kafka.output.topic=output +kafka.configuration.topic=configuration +kafka.feedback.topic=aggregation-feedback +kafka.window.duration.minutes=1 + +schema.registry.url=http://localhost:8081 + +aggregation.duration.days=30 +aggregation.advance.days=1 + +trigger.interval=15 +grace.period.ms=270 + +num.threads=1 +commit.interval.ms=1000 +cache.max.bytes.buffering=-1 + +specific.avro.reader=True +enable.auto.commit.config=True +auto.offset.reset.config=earliest \ No newline at end of file diff --git a/theodolite-benchmarks/uc4-beam/.settings/org.eclipse.jdt.ui.prefs b/theodolite-benchmarks/uc4-beam/.settings/org.eclipse.jdt.ui.prefs new file mode 100644 index 0000000000000000000000000000000000000000..32e18c393f53a0b0b3207bb896ec0e4211b27bf0 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/.settings/org.eclipse.jdt.ui.prefs @@ -0,0 +1,284 @@ +cleanup.add_all=false +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.array_with_curly=false +cleanup.arrays_fill=false +cleanup.bitwise_conditional_expression=false +cleanup.boolean_literal=false +cleanup.boolean_value_rather_than_comparison=true +cleanup.break_loop=false +cleanup.collection_cloning=false +cleanup.comparing_on_criteria=false +cleanup.comparison_statement=false +cleanup.controlflow_merge=false +cleanup.convert_functional_interfaces=false +cleanup.convert_to_enhanced_for_loop=true +cleanup.convert_to_enhanced_for_loop_if_loop_var_used=true +cleanup.convert_to_switch_expressions=false +cleanup.correct_indentation=true +cleanup.do_while_rather_than_while=true +cleanup.double_negation=false +cleanup.else_if=false +cleanup.embedded_if=false +cleanup.evaluate_nullable=false +cleanup.extract_increment=false +cleanup.format_source_code=true +cleanup.format_source_code_changes_only=false +cleanup.hash=false +cleanup.if_condition=false +cleanup.insert_inferred_type_arguments=false +cleanup.instanceof=false +cleanup.instanceof_keyword=false +cleanup.invert_equals=false +cleanup.join=false +cleanup.lazy_logical_operator=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.map_cloning=false +cleanup.merge_conditional_blocks=false +cleanup.multi_catch=false +cleanup.never_use_blocks=false +cleanup.never_use_parentheses_in_expressions=true +cleanup.no_string_creation=false +cleanup.no_super=false +cleanup.number_suffix=false +cleanup.objects_equals=false +cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=true +cleanup.operand_factorization=false +cleanup.organize_imports=true +cleanup.overridden_assignment=false +cleanup.plain_replacement=false +cleanup.precompile_regex=false +cleanup.primitive_comparison=false +cleanup.primitive_parsing=false +cleanup.primitive_rather_than_wrapper=true +cleanup.primitive_serialization=false +cleanup.pull_out_if_from_if_else=false +cleanup.pull_up_assignment=false +cleanup.push_down_negation=false +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.reduce_indentation=false +cleanup.redundant_comparator=false +cleanup.redundant_falling_through_block_end=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_array_creation=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.return_expression=false +cleanup.simplify_lambda_expression_and_method_ref=false +cleanup.single_used_field=false +cleanup.sort_members=false +cleanup.sort_members_all=false +cleanup.standard_comparison=false +cleanup.static_inner_class=false +cleanup.strictly_equal_or_different=false +cleanup.stringbuffer_to_stringbuilder=false +cleanup.stringbuilder=false +cleanup.stringbuilder_for_local_vars=true +cleanup.substring=false +cleanup.switch=false +cleanup.system_property=false +cleanup.system_property_boolean=false +cleanup.system_property_file_encoding=false +cleanup.system_property_file_separator=false +cleanup.system_property_line_separator=false +cleanup.system_property_path_separator=false +cleanup.ternary_operator=false +cleanup.try_with_resource=false +cleanup.unlooped_while=false +cleanup.unreachable_block=false +cleanup.use_anonymous_class_creation=false +cleanup.use_autoboxing=false +cleanup.use_blocks=true +cleanup.use_blocks_only_for_return_and_throw=false +cleanup.use_directly_map_method=false +cleanup.use_lambda=true +cleanup.use_parentheses_in_expressions=true +cleanup.use_string_is_blank=false +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.use_unboxing=false +cleanup.use_var=false +cleanup.useless_continue=false +cleanup.useless_return=false +cleanup.valueof_rather_than_instantiation=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=21 +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_all=false +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.array_with_curly=false +sp_cleanup.arrays_fill=false +sp_cleanup.bitwise_conditional_expression=false +sp_cleanup.boolean_literal=false +sp_cleanup.boolean_value_rather_than_comparison=false +sp_cleanup.break_loop=false +sp_cleanup.collection_cloning=false +sp_cleanup.comparing_on_criteria=false +sp_cleanup.comparison_statement=false +sp_cleanup.controlflow_merge=false +sp_cleanup.convert_functional_interfaces=false +sp_cleanup.convert_to_enhanced_for_loop=false +sp_cleanup.convert_to_enhanced_for_loop_if_loop_var_used=false +sp_cleanup.convert_to_switch_expressions=false +sp_cleanup.correct_indentation=true +sp_cleanup.do_while_rather_than_while=false +sp_cleanup.double_negation=false +sp_cleanup.else_if=false +sp_cleanup.embedded_if=false +sp_cleanup.evaluate_nullable=false +sp_cleanup.extract_increment=false +sp_cleanup.format_source_code=true +sp_cleanup.format_source_code_changes_only=false +sp_cleanup.hash=false +sp_cleanup.if_condition=false +sp_cleanup.insert_inferred_type_arguments=false +sp_cleanup.instanceof=false +sp_cleanup.instanceof_keyword=false +sp_cleanup.invert_equals=false +sp_cleanup.join=false +sp_cleanup.lazy_logical_operator=false +sp_cleanup.make_local_variable_final=true +sp_cleanup.make_parameters_final=false +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.map_cloning=false +sp_cleanup.merge_conditional_blocks=false +sp_cleanup.multi_catch=false +sp_cleanup.never_use_blocks=false +sp_cleanup.never_use_parentheses_in_expressions=true +sp_cleanup.no_string_creation=false +sp_cleanup.no_super=false +sp_cleanup.number_suffix=false +sp_cleanup.objects_equals=false +sp_cleanup.on_save_use_additional_actions=true +sp_cleanup.one_if_rather_than_duplicate_blocks_that_fall_through=false +sp_cleanup.operand_factorization=false +sp_cleanup.organize_imports=true +sp_cleanup.overridden_assignment=false +sp_cleanup.plain_replacement=false +sp_cleanup.precompile_regex=false +sp_cleanup.primitive_comparison=false +sp_cleanup.primitive_parsing=false +sp_cleanup.primitive_rather_than_wrapper=false +sp_cleanup.primitive_serialization=false +sp_cleanup.pull_out_if_from_if_else=false +sp_cleanup.pull_up_assignment=false +sp_cleanup.push_down_negation=false +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.reduce_indentation=false +sp_cleanup.redundant_comparator=false +sp_cleanup.redundant_falling_through_block_end=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=false +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_array_creation=false +sp_cleanup.remove_unnecessary_casts=true +sp_cleanup.remove_unnecessary_nls_tags=false +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.return_expression=false +sp_cleanup.simplify_lambda_expression_and_method_ref=false +sp_cleanup.single_used_field=false +sp_cleanup.sort_members=false +sp_cleanup.sort_members_all=false +sp_cleanup.standard_comparison=false +sp_cleanup.static_inner_class=false +sp_cleanup.strictly_equal_or_different=false +sp_cleanup.stringbuffer_to_stringbuilder=false +sp_cleanup.stringbuilder=false +sp_cleanup.stringbuilder_for_local_vars=true +sp_cleanup.substring=false +sp_cleanup.switch=false +sp_cleanup.system_property=false +sp_cleanup.system_property_boolean=false +sp_cleanup.system_property_file_encoding=false +sp_cleanup.system_property_file_separator=false +sp_cleanup.system_property_line_separator=false +sp_cleanup.system_property_path_separator=false +sp_cleanup.ternary_operator=false +sp_cleanup.try_with_resource=false +sp_cleanup.unlooped_while=false +sp_cleanup.unreachable_block=false +sp_cleanup.use_anonymous_class_creation=false +sp_cleanup.use_autoboxing=false +sp_cleanup.use_blocks=true +sp_cleanup.use_blocks_only_for_return_and_throw=false +sp_cleanup.use_directly_map_method=false +sp_cleanup.use_lambda=true +sp_cleanup.use_parentheses_in_expressions=true +sp_cleanup.use_string_is_blank=false +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 +sp_cleanup.use_unboxing=false +sp_cleanup.use_var=false +sp_cleanup.useless_continue=false +sp_cleanup.useless_return=false +sp_cleanup.valueof_rather_than_instantiation=false diff --git a/theodolite-benchmarks/uc4-beam/build.gradle b/theodolite-benchmarks/uc4-beam/build.gradle new file mode 100644 index 0000000000000000000000000000000000000000..502e94fa737fb2ae1bab861407b27575cd8766ca --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/build.gradle @@ -0,0 +1,5 @@ +plugins { + id 'theodolite.beam' +} + + diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedActivePowerRecordEventTimePolicy.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedActivePowerRecordEventTimePolicy.java new file mode 100644 index 0000000000000000000000000000000000000000..dad9eca7d8d50d1b85932ddaa7ffc99418a4b759 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedActivePowerRecordEventTimePolicy.java @@ -0,0 +1,34 @@ +package application; + +import java.util.Optional; +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; + +/** + * TimeStampPolicy to use event time based on the timestamp of the record value. + */ +public class AggregatedActivePowerRecordEventTimePolicy + extends TimestampPolicy<String, AggregatedActivePowerRecord> { + protected Instant currentWatermark; + + public AggregatedActivePowerRecordEventTimePolicy(final Optional<Instant> previousWatermark) { + super(); + this.currentWatermark = previousWatermark.orElse(BoundedWindow.TIMESTAMP_MIN_VALUE); + } + + @Override + public Instant getTimestampForRecord(final PartitionContext ctx, + final KafkaRecord<String, AggregatedActivePowerRecord> record) { + this.currentWatermark = new Instant(record.getKV().getValue().getTimestamp()); + return this.currentWatermark; + } + + @Override + public Instant getWatermark(final PartitionContext ctx) { + return this.currentWatermark; + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedToActive.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedToActive.java new file mode 100644 index 0000000000000000000000000000000000000000..bddd43e1e09c54bcfc85e5cbb65d1a6487f53438 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/AggregatedToActive.java @@ -0,0 +1,22 @@ +package application; + +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; + +/** + * Converts AggregatedActivePowerRecord to ActivePowerRecord. + */ +public class AggregatedToActive + extends SimpleFunction<KV<String, AggregatedActivePowerRecord>, KV<String, ActivePowerRecord>> { + + private static final long serialVersionUID = -8275252527964065889L; + + @Override + public KV<String, ActivePowerRecord> apply( + final KV<String, AggregatedActivePowerRecord> kv) { + return KV.of(kv.getKey(), new ActivePowerRecord(kv.getValue().getIdentifier(), + kv.getValue().getTimestamp(), kv.getValue().getSumInW())); + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/DuplicateAsFlatMap.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/DuplicateAsFlatMap.java new file mode 100644 index 0000000000000000000000000000000000000000..7b66082c91b87c246d8c834249d2bc82545766f5 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/DuplicateAsFlatMap.java @@ -0,0 +1,69 @@ +package application; + +import com.google.common.base.MoreObjects; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +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; + + +/** + * Duplicates the Kv containing the (Children,Parents) pair as a flat map. + */ +public class DuplicateAsFlatMap extends DoFn + <KV<String, ActivePowerRecord>, KV<SensorParentKey, ActivePowerRecord>> { + private static final long serialVersionUID = -5132355515723961647L; + @StateId("parents") + private final StateSpec<ValueState<Set<String>>> parents = StateSpecs.value();//NOPMD + private final PCollectionView<Map<String, Set<String>>> childParentPairMap; + + public DuplicateAsFlatMap(final PCollectionView<Map<String, Set<String>>> childParentPairMap) { + super(); + this.childParentPairMap = childParentPairMap; + } + + + /** + * Generate a KV-pair for every child-parent match. + */ + @ProcessElement + public void processElement(@Element final KV<String, ActivePowerRecord> kv, + final OutputReceiver<KV<SensorParentKey, ActivePowerRecord>> out, + @StateId("parents") final ValueState<Set<String>> state, + final ProcessContext c) { + + final ActivePowerRecord record = kv.getValue() == null ? null : kv.getValue(); + final Set<String> newParents = + c.sideInput(childParentPairMap).get(kv.getKey()) == null + ? Collections.emptySet() + : c.sideInput(childParentPairMap).get(kv.getKey()); + final Set<String> oldParents = + MoreObjects.firstNonNull(state.read(), Collections.emptySet()); + // Forward new Pairs if they exist + if (!newParents.isEmpty()) { + for (final String parent : newParents) { + + // Forward flat mapped record + final SensorParentKey key = new SensorParentKey(kv.getKey(), parent); + out.output(KV.of(key, record)); + } + } + if (!newParents.equals(oldParents)) { + for (final String oldParent : oldParents) { + if (!newParents.contains(oldParent)) { + // Forward Delete + final SensorParentKey key = new SensorParentKey(kv.getKey(), oldParent); + out.output(KV.of(key, null)); + } + } + state.write(newParents); + } + } +} + diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterEvents.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterEvents.java new file mode 100644 index 0000000000000000000000000000000000000000..3588443393fdef2e0fd1bf5f1e7c497e5030cf77 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterEvents.java @@ -0,0 +1,19 @@ +package application; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import titan.ccp.configuration.events.Event; + +/** + * Filters for {@code Event.SENSOR_REGISTRY_CHANGED} and {@code Event.SENSOR_REGISTRY_STATUS} + * events. + */ +public class FilterEvents implements SerializableFunction<KV<Event, String>, Boolean> { + private static final long serialVersionUID = -2233447357614891559L; + + @Override + public Boolean apply(final KV<Event, String> kv) { + return kv.getKey() == Event.SENSOR_REGISTRY_CHANGED + || kv.getKey() == Event.SENSOR_REGISTRY_STATUS; + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterNullValues.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterNullValues.java new file mode 100644 index 0000000000000000000000000000000000000000..143294f1ff2bfeea77c40ce38cd10ce3eb44be49 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/FilterNullValues.java @@ -0,0 +1,18 @@ +package application; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import titan.ccp.model.records.ActivePowerRecord; + +/** + * Filters {@code null} Values. + */ +public class FilterNullValues implements + SerializableFunction<KV<SensorParentKey, ActivePowerRecord>, Boolean> { + private static final long serialVersionUID = -6197352369880867482L; + + @Override + public Boolean apply(final KV<SensorParentKey, ActivePowerRecord> kv) { + return kv.getValue() != null; + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/GenerateParentsFn.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/GenerateParentsFn.java new file mode 100644 index 0000000000000000000000000000000000000000..68cf551af9c681c586ebc6026c043ae8c9befbc5 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/GenerateParentsFn.java @@ -0,0 +1,60 @@ +package application; + +import java.util.Collections; +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.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +// import theodolite.uc2.streamprocessing.KeyValue; +// import theodolite.uc2.streamprocessing.KeyValueIterator; +import titan.ccp.configuration.events.Event; +import titan.ccp.model.sensorregistry.AggregatedSensor; +import titan.ccp.model.sensorregistry.Sensor; +import titan.ccp.model.sensorregistry.SensorRegistry; + +/** + * DoFn class to generate a child-parent pair for every sensor in the hierarchy. + */ +public class GenerateParentsFn extends DoFn<KV<Event, String>, KV<String, Set<String>>> { + + private static final long serialVersionUID = 958270648688932091L; + + /** + * Transforms a parent [children] map of sensors to a child [parents] map. + * + * @param kv input map. + * @param out outputstream. + */ + @ProcessElement + public void processElement(@Element final KV<Event, String> kv, + final OutputReceiver<KV<String, Set<String>>> out) { + final Map<String, Set<String>> childParentsPairs = + this.constructChildParentsPairs(SensorRegistry.fromJson(kv.getValue())); + final Iterator<Map.Entry<String, Set<String>>> it = childParentsPairs.entrySet().iterator(); + while (it.hasNext()) { + final Map.Entry<String, Set<String>> pair = it.next(); + out.output(KV.of(pair.getKey(), pair.getValue())); + } + + } + + private Map<String, Set<String>> constructChildParentsPairs(final SensorRegistry registry) { + return this.streamAllChildren(registry.getTopLevelSensor()) + .collect(Collectors.<Sensor, String, Set<String>>toMap( + child -> child.getIdentifier(), + child -> child.getParent() + .map(p -> Stream.of(p.getIdentifier()).collect(Collectors.toSet())) + .orElse(Collections.<String>emptySet()))); + } + + 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())); + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/RecordAggregation.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/RecordAggregation.java new file mode 100644 index 0000000000000000000000000000000000000000..16fd411b44cc1f955b255be870215ac120bce193 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/RecordAggregation.java @@ -0,0 +1,63 @@ +package application; + +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; + + + +/** + * CombineFn to aggregate ActivePowerRecords into AggregatedActivePowerRecords. + */ +public class RecordAggregation + extends CombineFn<ActivePowerRecord, RecordAggregation.Accum, AggregatedActivePowerRecord> { + + private static final long serialVersionUID = 4362213539553233529L; + + /** + * Wrapper for an accumulation of records. + */ + @DefaultCoder(AvroCoder.class) + public static class Accum implements Serializable { + private static final long serialVersionUID = 3701311203919534376L; + private long count; + private Double sum = 0.0; + private long timestamp; + } + + @Override + public Accum createAccumulator() { + return new Accum(); + } + + @Override + public Accum addInput(final Accum mutableAccumulator, final ActivePowerRecord input) { + mutableAccumulator.count += 1; + mutableAccumulator.sum += input.getValueInW(); + mutableAccumulator.timestamp = input.getTimestamp(); + return mutableAccumulator; + } + + @Override + public Accum mergeAccumulators(final Iterable<Accum> accumulators) { + final Accum merged = this.createAccumulator(); + for (final Accum accumulator : accumulators) { + merged.count += accumulator.count; + merged.sum += accumulator.sum; + merged.timestamp = accumulator.timestamp; + } + + return merged; + } + + @Override + public AggregatedActivePowerRecord extractOutput(final Accum accumulator) { + final double average = accumulator.count == 0 ? 0.0 : accumulator.sum / accumulator.count; + return new AggregatedActivePowerRecord("", accumulator.timestamp, accumulator.count, + accumulator.sum, average); + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/SensorParentKey.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/SensorParentKey.java new file mode 100644 index 0000000000000000000000000000000000000000..546fc04c2de089a28d8f0fba86a7fbcd5c1cc0a8 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/SensorParentKey.java @@ -0,0 +1,30 @@ +package application; + +/** + * 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 + "}"; + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/SetIdForAggregated.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/SetIdForAggregated.java new file mode 100644 index 0000000000000000000000000000000000000000..0279a26ed925408b5383be50a202f5c88cea53aa --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/SetIdForAggregated.java @@ -0,0 +1,22 @@ +package application; + +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import titan.ccp.model.records.AggregatedActivePowerRecord; + +/** + * Sets the identifier for new {@link AggregatedActivePowerRecord}. + */ +public class SetIdForAggregated extends + SimpleFunction<KV<String, AggregatedActivePowerRecord>, KV<String, AggregatedActivePowerRecord>> { // NOCS + private static final long serialVersionUID = 2148522605294086982L; + + @Override + public KV<String, AggregatedActivePowerRecord> apply( + final KV<String, AggregatedActivePowerRecord> kv) { + final AggregatedActivePowerRecord record = new AggregatedActivePowerRecord( + kv.getKey(), kv.getValue().getTimestamp(), kv.getValue().getCount(), + kv.getValue().getSumInW(), kv.getValue().getAverageInW()); + return KV.of(kv.getKey(), record); + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/SetKeyToGroup.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/SetKeyToGroup.java new file mode 100644 index 0000000000000000000000000000000000000000..7d8dc70583fb45e02a5a8091b92d724bb22b4a78 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/SetKeyToGroup.java @@ -0,0 +1,20 @@ +package application; + +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import titan.ccp.model.records.ActivePowerRecord; + +/** + * Set the Key for a group of {@code ActivePowerRecords} to their Parent. + */ +public class SetKeyToGroup + extends SimpleFunction<KV<SensorParentKey, ActivePowerRecord>, KV<String, ActivePowerRecord>> { + + private static final long serialVersionUID = 790215050768527L; + + @Override + public KV<String, ActivePowerRecord> apply( + final KV<SensorParentKey, ActivePowerRecord> kv) { + return KV.of(kv.getKey().getParent(), kv.getValue()); + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/Uc4BeamPipeline.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/Uc4BeamPipeline.java new file mode 100644 index 0000000000000000000000000000000000000000..7179fe5da937280d5baf72cd73cc392ef15a60e0 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/Uc4BeamPipeline.java @@ -0,0 +1,248 @@ +package application; // NOPMD + +import com.google.common.math.StatsAccumulator; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.Latest; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.commons.configuration2.Configuration; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.joda.time.Duration; +import serialization.AggregatedActivePowerRecordCoder; +import serialization.AggregatedActivePowerRecordDeserializer; +import serialization.AggregatedActivePowerRecordSerializer; +import serialization.EventCoder; +import serialization.EventDeserializer; +import serialization.SensorParentKeyCoder; +import theodolite.commons.beam.AbstractPipeline; +import theodolite.commons.beam.ConfigurationKeys; +import theodolite.commons.beam.kafka.KafkaActivePowerTimestampReader; +import theodolite.commons.beam.kafka.KafkaGenericReader; +import theodolite.commons.beam.kafka.KafkaWriterTransformation; +import titan.ccp.configuration.events.Event; +import titan.ccp.model.records.ActivePowerRecord; +import titan.ccp.model.records.AggregatedActivePowerRecord; + +/** + * Implementation of the use case Hierarchical Aggregation using Apache Beam. + */ +public final class Uc4BeamPipeline extends AbstractPipeline { + + protected Uc4BeamPipeline(final PipelineOptions options, final Configuration config) { // NOPMD + super(options, config); + + // Additional needed variables + final String feedbackTopic = config.getString(ConfigurationKeys.KAFKA_FEEDBACK_TOPIC); + final String outputTopic = config.getString(ConfigurationKeys.KAFKA_OUTPUT_TOPIC); + final String configurationTopic = config.getString(ConfigurationKeys.KAFKA_CONFIGURATION_TOPIC); + + final Duration duration = + Duration.standardSeconds(config.getInt(ConfigurationKeys.KAFKA_WINDOW_DURATION_MINUTES)); + final Duration triggerDelay = + Duration.standardSeconds(config.getInt(ConfigurationKeys.TRIGGER_INTERVAL)); + final Duration gracePeriod = + Duration.standardSeconds(config.getInt(ConfigurationKeys.GRACE_PERIOD_MS)); + + // Build kafka configuration + final Map<String, Object> consumerConfig = this.buildConsumerConfig(); + final Map<String, Object> configurationConfig = this.configurationConfig(config); + + // Set Coders for Classes that will be distributed + final CoderRegistry cr = this.getCoderRegistry(); + registerCoders(cr); + + // Read from Kafka + // ActivePowerRecords + final KafkaActivePowerTimestampReader kafkaActivePowerRecordReader = + new KafkaActivePowerTimestampReader(this.bootstrapServer, this.inputTopic, consumerConfig); + + // Configuration Events + final KafkaGenericReader<Event, String> kafkaConfigurationReader = + new KafkaGenericReader<>( + this.bootstrapServer, configurationTopic, configurationConfig, + EventDeserializer.class, StringDeserializer.class); + + // Transform into AggregatedActivePowerRecords into ActivePowerRecords + final AggregatedToActive aggregatedToActive = new AggregatedToActive(); + + // Write to Kafka + final KafkaWriterTransformation<AggregatedActivePowerRecord> kafkaOutput = + new KafkaWriterTransformation<>( + this.bootstrapServer, outputTopic, AggregatedActivePowerRecordSerializer.class); + + final KafkaWriterTransformation<AggregatedActivePowerRecord> kafkaFeedback = + new KafkaWriterTransformation<>( + this.bootstrapServer, feedbackTopic, AggregatedActivePowerRecordSerializer.class); + + // Apply pipeline transformations + final PCollection<KV<String, ActivePowerRecord>> values = this + .apply("Read from Kafka", kafkaActivePowerRecordReader) + .apply("Read Windows", Window.into(FixedWindows.of(duration))) + .apply("Set trigger for input", Window + .<KV<String, ActivePowerRecord>>configure() + .triggering(Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggerDelay))) + .withAllowedLateness(gracePeriod) + .discardingFiredPanes()); + + // Read the results of earlier aggregations. + final PCollection<KV<String, ActivePowerRecord>> aggregationsInput = this + .apply("Read aggregation results", KafkaIO.<String, AggregatedActivePowerRecord>read() + .withBootstrapServers(this.bootstrapServer) + .withTopic(feedbackTopic) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializer(AggregatedActivePowerRecordDeserializer.class) + .withTimestampPolicyFactory( + (tp, previousWaterMark) -> new AggregatedActivePowerRecordEventTimePolicy( + previousWaterMark)) + .withoutMetadata()) + .apply("Apply Windows", Window.into(FixedWindows.of(duration))) + // Convert into the correct data format + .apply("Convert AggregatedActivePowerRecord to ActivePowerRecord", + MapElements.via(aggregatedToActive)) + .apply("Set trigger for feedback", Window + .<KV<String, ActivePowerRecord>>configure() + .triggering(Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggerDelay))) + .withAllowedLateness(gracePeriod) + .discardingFiredPanes()); + + // Prepare flatten + final PCollectionList<KV<String, ActivePowerRecord>> collections = + PCollectionList.of(values).and(aggregationsInput); + + // Create a single PCollection out of the input and already computed results + final PCollection<KV<String, ActivePowerRecord>> inputCollection = + collections.apply("Flatten sensor data and aggregation results", + Flatten.pCollections()); + + // Build the configuration stream from a changelog. + final PCollection<KV<String, Set<String>>> configurationStream = this + .apply("Read sensor groups", kafkaConfigurationReader) + // Only forward relevant changes in the hierarchy + .apply("Filter changed and status events", + Filter.by(new FilterEvents())) + // Build the changelog + .apply("Generate Parents for every Sensor", ParDo.of(new GenerateParentsFn())) + .apply("Update child and parent pairs", ParDo.of(new UpdateChildParentPairs())) + .apply("Set trigger for configuration", Window + .<KV<String, Set<String>>>configure() + .triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings( + AfterPane.elementCountAtLeast(1))) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + + final PCollectionView<Map<String, Set<String>>> childParentPairMap = + configurationStream.apply(Latest.perKey()) + // Reset trigger to avoid synchronized processing time + .apply("Reset trigger for configurations", Window + .<KV<String, Set<String>>>configure() + .triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings( + AfterPane.elementCountAtLeast(1))) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()) + .apply(View.asMap()); + + final FilterNullValues filterNullValues = new FilterNullValues(); + + // Build pairs of every sensor reading and parent + final PCollection<KV<SensorParentKey, ActivePowerRecord>> flatMappedValues = + inputCollection.apply( + "Duplicate as flatMap", + ParDo.of(new DuplicateAsFlatMap(childParentPairMap)) + .withSideInputs(childParentPairMap)) + .apply("Filter only latest changes", Latest.perKey()) + .apply("Filter out null values", + Filter.by(filterNullValues)); + + final SetIdForAggregated setIdForAggregated = new SetIdForAggregated(); + final SetKeyToGroup setKeyToGroup = new SetKeyToGroup(); + + // Aggregate for every sensor group of the current level + final PCollection<KV<String, AggregatedActivePowerRecord>> aggregations = flatMappedValues + .apply("Set key to group", MapElements.via(setKeyToGroup)) + // Reset trigger to avoid synchronized processing time + .apply("Reset trigger for aggregations", Window + .<KV<String, ActivePowerRecord>>configure() + .triggering(Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggerDelay))) + .withAllowedLateness(gracePeriod) + .discardingFiredPanes()) + .apply( + "Aggregate per group", + Combine.perKey(new RecordAggregation())) + .apply("Set the Identifier in AggregatedActivePowerRecord", + MapElements.via(setIdForAggregated)); + + aggregations.apply("Write to aggregation results", kafkaOutput); + + aggregations + .apply("Write to feedback topic", kafkaFeedback); + + } + + + /** + * Builds a simple configuration for a Kafka consumer transformation. + * + * @return the build configuration. + */ + public Map<String, Object> configurationConfig(final Configuration config) { + final Map<String, Object> consumerConfig = new HashMap<>(); + consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + config.getString(ConfigurationKeys.ENABLE_AUTO_COMMIT_CONFIG)); + consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + config + .getString(ConfigurationKeys.AUTO_OFFSET_RESET_CONFIG)); + + consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, config + .getString(ConfigurationKeys.APPLICATION_NAME) + "-configuration"); + return consumerConfig; + } + + + /** + * Registers all Coders for all needed Coders. + * + * @param cr CoderRegistry. + */ + private static void registerCoders(final CoderRegistry cr) { + cr.registerCoderForClass(ActivePowerRecord.class, + AvroCoder.of(ActivePowerRecord.class)); + cr.registerCoderForClass(AggregatedActivePowerRecord.class, + new AggregatedActivePowerRecordCoder()); + cr.registerCoderForClass(Set.class, SetCoder.of(StringUtf8Coder.of())); + cr.registerCoderForClass(Event.class, new EventCoder()); + cr.registerCoderForClass(SensorParentKey.class, new SensorParentKeyCoder()); + cr.registerCoderForClass(StatsAccumulator.class, AvroCoder.of(StatsAccumulator.class)); + } +} + diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/application/UpdateChildParentPairs.java b/theodolite-benchmarks/uc4-beam/src/main/java/application/UpdateChildParentPairs.java new file mode 100644 index 0000000000000000000000000000000000000000..8692be5ae6637ebda86f10d66b43c6071264e099 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/application/UpdateChildParentPairs.java @@ -0,0 +1,36 @@ +package application; + +import java.util.Set; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; + +/** + * Forward changes or tombstone values for deleted records. + */ +public class UpdateChildParentPairs extends DoFn<KV<String, Set<String>>, KV<String, Set<String>>> { + + private static final long serialVersionUID = 1L; + + @StateId("parents") + private final StateSpec<ValueState<Set<String>>> parents = // NOPMD + StateSpecs.value(); + + /** + * Match the changes accordingly. + * + * @param kv the sensor parents set that contains the changes. + */ + @ProcessElement + public void processElement(@Element final KV<String, Set<String>> kv, + final OutputReceiver<KV<String, Set<String>>> out, + @StateId("parents") final ValueState<Set<String>> state) { + if (kv.getValue() == null || !kv.getValue().equals(state.read())) { + out.output(kv); + state.write(kv.getValue()); + } + + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordCoder.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordCoder.java new file mode 100644 index 0000000000000000000000000000000000000000..d2b484f5ab30be63f311d6dbcf495baebbd5e2b4 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordCoder.java @@ -0,0 +1,57 @@ +package serialization; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +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; + +/** + * Wrapper Class that encapsulates a AggregatedActivePowerRecord Serde in a + * org.apache.beam.sdk.coders.Coder. + */ +@SuppressWarnings("serial") +public class AggregatedActivePowerRecordCoder extends Coder<AggregatedActivePowerRecord> + implements Serializable { + + private static final boolean DETERMINISTIC = true; + + private transient AvroCoder<AggregatedActivePowerRecord> avroEnCoder = + AvroCoder.of(AggregatedActivePowerRecord.class); + + @Override + public void encode(final AggregatedActivePowerRecord value, final OutputStream outStream) + throws CoderException, IOException { + if (this.avroEnCoder == null) { + this.avroEnCoder = AvroCoder.of(AggregatedActivePowerRecord.class); + } + this.avroEnCoder.encode(value, outStream); + + } + + @Override + public AggregatedActivePowerRecord decode(final InputStream inStream) + throws CoderException, IOException { + if (this.avroEnCoder == null) { + this.avroEnCoder = AvroCoder.of(AggregatedActivePowerRecord.class); + } + return this.avroEnCoder.decode(inStream); + + } + + @Override + public List<? extends Coder<?>> getCoderArguments() { + return null; + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!DETERMINISTIC) { + throw new NonDeterministicException(this, "This class should be deterministic!"); + } + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordDeserializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordDeserializer.java new file mode 100644 index 0000000000000000000000000000000000000000..6e2f2765ff65d3bca2a127be36db0854f15afebc --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordDeserializer.java @@ -0,0 +1,34 @@ +package serialization; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.kafka.common.serialization.Deserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import titan.ccp.model.records.AggregatedActivePowerRecord; + +/** + * Wrapper Class that encapsulates a IMonitoringRecordSerde.serializer in a Deserializer + */ +public class AggregatedActivePowerRecordDeserializer + implements Deserializer<AggregatedActivePowerRecord> { + + private static final Logger LOGGER = + LoggerFactory.getLogger(AggregatedActivePowerRecordDeserializer.class); + + private final transient AvroCoder<AggregatedActivePowerRecord> avroEnCoder = + AvroCoder.of(AggregatedActivePowerRecord.class); + + @Override + public AggregatedActivePowerRecord deserialize(final String topic, final byte[] data) { + AggregatedActivePowerRecord value = null; + try { + value = this.avroEnCoder.decode(new ByteArrayInputStream(data)); + } catch (final IOException e) { + LOGGER.error("Could not deserialize AggregatedActivePowerRecord", e); + } + return value; + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordSerializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordSerializer.java new file mode 100644 index 0000000000000000000000000000000000000000..77b79d5465f1d561870bf5b04f8fa20f87076adb --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/AggregatedActivePowerRecordSerializer.java @@ -0,0 +1,45 @@ +package serialization; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import titan.ccp.model.records.AggregatedActivePowerRecord; + +/** + * Wrapper Class that encapsulates a IMonitoringRecordSerde.serializer in a Serializer + */ +public class AggregatedActivePowerRecordSerializer + implements Serializer<AggregatedActivePowerRecord> { + + private static final Logger LOGGER = + LoggerFactory.getLogger(AggregatedActivePowerRecordSerializer.class); + + private final transient AvroCoder<AggregatedActivePowerRecord> avroEnCoder = + AvroCoder.of(AggregatedActivePowerRecord.class); + + @Override + public byte[] serialize(final String topic, final AggregatedActivePowerRecord data) { + final ByteArrayOutputStream out = new ByteArrayOutputStream(); + try { + this.avroEnCoder.encode(data, out); + } catch (final IOException e) { + LOGGER.error("Could not serialize AggregatedActivePowerRecord", e); + } + final byte[] result = out.toByteArray(); + try { + out.close(); + } catch (final IOException e) { + LOGGER.error( + "Could not close output stream after serialization of AggregatedActivePowerRecord", e); + } + return result; + } + + @Override + public void close() { + Serializer.super.close(); + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventCoder.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventCoder.java new file mode 100644 index 0000000000000000000000000000000000000000..710beb71dc8776e6309028327b05307aa590a7f6 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventCoder.java @@ -0,0 +1,63 @@ +package serialization; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Collections; +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; + +/** + * Wrapper Class that encapsulates a Event Serde in a org.apache.beam.sdk.coders.Coder. + */ +public class EventCoder extends Coder<Event> implements Serializable { + + private static final long serialVersionUID = 8403045343970659100L; + private static final int VALUE_SIZE = 4; + private static final boolean DETERMINISTIC = true; + + private transient Serde<Event> innerSerde = EventSerde.serde(); + + @Override + public void encode(final Event value, final OutputStream outStream) + throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = EventSerde.serde(); + } + final byte[] bytes = this.innerSerde.serializer().serialize("ser", value); + final byte[] sizeinBytes = ByteBuffer.allocate(VALUE_SIZE).putInt(bytes.length).array(); + outStream.write(sizeinBytes); + outStream.write(bytes); + } + + @Override + public Event decode(final InputStream inStream) throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = EventSerde.serde(); + } + final byte[] sizeinBytes = new byte[VALUE_SIZE]; + inStream.read(sizeinBytes); + final int size = ByteBuffer.wrap(sizeinBytes).getInt(); + final byte[] bytes = new byte[size]; + inStream.read(bytes); + return this.innerSerde.deserializer().deserialize("deser", bytes); + } + + @Override + public List<? extends Coder<?>> getCoderArguments() { + return Collections.emptyList(); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!DETERMINISTIC) { + throw new NonDeterministicException(this, "This class should be deterministic!"); + } + } +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventDeserializer.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventDeserializer.java new file mode 100644 index 0000000000000000000000000000000000000000..34e31a3059d0749848a30979f32e6df6651c1b47 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/EventDeserializer.java @@ -0,0 +1,36 @@ +package 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; + +/** + * Deserializer for Events(SensorRegistry changes). + */ +public class EventDeserializer implements Deserializer<Event> { + + private final ByteBufferDeserializer byteBufferDeserializer = new ByteBufferDeserializer(); + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + this.byteBufferDeserializer.configure(configs, isKey); + } + + @Override + public Event deserialize(final String topic, final byte[] data) { + final int ordinal = this.byteBufferDeserializer.deserialize(topic, data).getInt(); + for (final Event event : Event.values()) { + if (ordinal == event.ordinal()) { + return event; + } + } + throw new IllegalArgumentException("Deserialized data is not a valid event."); + } + + @Override + public void close() { + this.byteBufferDeserializer.close(); + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeyCoder.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeyCoder.java new file mode 100644 index 0000000000000000000000000000000000000000..3e85c3242fb854bef514787c92bb58ad76526cb4 --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeyCoder.java @@ -0,0 +1,67 @@ +package serialization; + +import application.SensorParentKey; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Collections; +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; + +/** + * Wrapper Class that encapsulates a SensorParentKey Serde in a org.apache.beam.sdk.coders.Coder. + */ +public class SensorParentKeyCoder extends Coder<SensorParentKey> implements Serializable { + + private static final long serialVersionUID = -3480141901035692398L; + private static final boolean DETERMINISTIC = true; + private static final int VALUE_SIZE = 4; + + private transient Serde<SensorParentKey> innerSerde = SensorParentKeySerde.serde(); + + @Override + public void encode(final SensorParentKey value, final OutputStream outStream) + throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = SensorParentKeySerde.serde(); + + } + final byte[] bytes = this.innerSerde.serializer().serialize("ser", value); + final byte[] sizeinBytes = ByteBuffer.allocate(VALUE_SIZE).putInt(bytes.length).array(); + outStream.write(sizeinBytes); + outStream.write(bytes); + + } + + @Override + public SensorParentKey decode(final InputStream inStream) throws CoderException, IOException { + if (this.innerSerde == null) { + this.innerSerde = SensorParentKeySerde.serde(); + + } + final byte[] sizeinBytes = new byte[VALUE_SIZE]; + inStream.read(sizeinBytes); + final int size = ByteBuffer.wrap(sizeinBytes).getInt(); + final byte[] bytes = new byte[size]; + inStream.read(bytes); + return this.innerSerde.deserializer().deserialize("deser", bytes); + + } + + @Override + public List<? extends Coder<?>> getCoderArguments() { + return Collections.emptyList(); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!DETERMINISTIC) { + throw new NonDeterministicException(this, "This class should be deterministic!"); + } + } + +} diff --git a/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeySerde.java b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeySerde.java new file mode 100644 index 0000000000000000000000000000000000000000..468adb3947439c11c4fd9b289f41b68e606bdb1d --- /dev/null +++ b/theodolite-benchmarks/uc4-beam/src/main/java/serialization/SensorParentKeySerde.java @@ -0,0 +1,34 @@ +package serialization; + +import application.SensorParentKey; +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 Serde} factory for {@link SensorParentKey}. + */ +public final class SensorParentKeySerde implements BufferSerde<SensorParentKey> { + + private SensorParentKeySerde() {} + + @Override + public void serialize(final WriteBuffer buffer, final SensorParentKey key) { + buffer.putString(key.getSensor()); + buffer.putString(key.getParent()); + } + + @Override + public SensorParentKey deserialize(final ReadBuffer buffer) { + final String sensor = buffer.getString(); + final String parent = buffer.getString(); + return new SensorParentKey(sensor, parent); + } + + public static Serde<SensorParentKey> serde() { + return SimpleSerdes.create(new SensorParentKeySerde()); + } + +}