diff --git a/docker-compose.yml b/docker-compose.yml index 07a2a09..491475a 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -39,8 +39,40 @@ services: ports: - "0.0.0.0:8082:8082" + kafka-create-topics: + image: confluentinc/cp-kafka:5.4.0 + depends_on: + - kafka + command: "bash -c 'echo Waiting for Kafka to be ready... && \ + cub kafka-ready -b kafka:29092 1 20 && \ + kafka-topics --create --topic bench1_feature --if-not-exists --zookeeper zookeeper:2181 --partitions 37 --replication-factor 1 && \ + kafka-topics --create --topic bench1_makinage --if-not-exists --zookeeper zookeeper:2181 --partitions 37 --replication-factor 1 && \ + kafka-topics --create --topic bench1_kafka_dsl --if-not-exists --zookeeper zookeeper:2181 --partitions 37 --replication-factor 1 && \ + sleep infinity'" + environment: + KAFKA_BROKER_ID: ignored + KAFKA_ZOOKEEPER_CONNECT: ignored + makinage-benchmark: image: makinage-benchmark:latest depends_on: - kafka - command: makinage --config /opt/makinage/config.yaml \ No newline at end of file + - kafka-create-topics + command: makinage --config /opt/makinage/config.yaml + + kafka-dsl-benchmark1: + image: kafka-dsl-benchmark1:latest + depends_on: + - kafka + - kafka-create-topics + environment: + APPLICATION_ID: kafka-dsl-benchmark1 + + kafka-dsl-benchmark1-scaled: + image: kafka-dsl-benchmark1:latest + depends_on: + - kafka + - kafka-create-topics + environment: + APPLICATION_ID: kafka-dsl-benchmark1-scaled + diff --git a/java/.gitignore b/java/.gitignore new file mode 100644 index 0000000..02cf2e1 --- /dev/null +++ b/java/.gitignore @@ -0,0 +1,4 @@ +.idea/ +kafka-dsl-benchmark1.iml +target/ + diff --git a/java/kafka-dsl-benchmark1/Dockerfile b/java/kafka-dsl-benchmark1/Dockerfile new file mode 100644 index 0000000..3d84023 --- /dev/null +++ b/java/kafka-dsl-benchmark1/Dockerfile @@ -0,0 +1,5 @@ +FROM openjdk:8-jre + +ADD ./target/kafka-dsl-benchmark1-*-with-dependencies.jar /usr/src/kafka-dsl-benchmark1.jar +WORKDIR /usr/src +ENTRYPOINT exec java -jar /usr/src/kafka-dsl-benchmark1.jar ${APPLICATION_ID} diff --git a/java/kafka-dsl-benchmark1/pom.xml b/java/kafka-dsl-benchmark1/pom.xml new file mode 100644 index 0000000..d26c61a --- /dev/null +++ b/java/kafka-dsl-benchmark1/pom.xml @@ -0,0 +1,78 @@ + + + 4.0.0 + com.makinage.kafka.dsl.benchmark + kafka-dsl-benchmark1 + 1.0-SNAPSHOT + + UTF-8 + 1.8 + 1.8 + + + + com.google.code.gson + gson + 2.8.6 + + + org.apache.logging.log4j + log4j-core + 2.13.3 + + + org.apache.logging.log4j + log4j-api + 2.13.3 + + + org.slf4j + slf4j-simple + 1.7.21 + + + org.apache.kafka + kafka-clients + 2.4.0 + + + org.apache.kafka + kafka-streams + 2.4.0 + + + junit + junit + 4.12 + test + + + + + + maven-assembly-plugin + + + + com.makinage.kafka.dsl.benchmark.Bench1 + + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Bench1.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Bench1.java new file mode 100644 index 0000000..0c71799 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Bench1.java @@ -0,0 +1,121 @@ +package com.makinage.kafka.dsl.benchmark; + +import com.google.gson.Gson; +import com.makinage.kafka.dsl.benchmark.utils.FeatureTimestampExtractor; +import com.makinage.kafka.dsl.benchmark.utils.JsonSerde; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.*; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.time.Duration; +import java.util.Date; +import java.util.Properties; + + +public class Bench1 { + + static final Logger LOG = LogManager.getLogger(Bench1.class.getName()); + + static final DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS"); + + static final Gson gson = new Gson(); + + public static void main(final String[] args) throws Exception, StreamsException { + + String applicationId = "kafka-dsl-benchmark1"; + if (args.length == 1 && !args[0].isEmpty()) + applicationId = args[0]; + + final Properties streamsConfiguration = new Properties(); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + streamsConfiguration.put(StreamsConfig.CLIENT_ID_CONFIG, applicationId + "-client"); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka:29092"); + streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, FeatureTimestampExtractor.class); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, "/tmp/kafka-streams/bench1"); + streamsConfiguration.put(ProducerConfig.ACKS_CONFIG, "all"); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 50); + + final KafkaStreams streams = createStreams(streamsConfiguration); + streams.cleanUp(); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + try { + streams.close(); + } catch (final Exception e) { + // ignored + } + })); + } + + static KafkaStreams createStreams(final Properties streamsConfiguration) { + + final StreamsBuilder builder = new StreamsBuilder(); + final KStream featuresStream = + builder.stream("bench1_feature", Consumed.with(Serdes.String(), JsonSerde.FeatureSerde())); + + featuresStream.groupByKey() + .windowedBy(TimeWindows.of(Duration.ofHours(2)).grace(Duration.ofMinutes(0))) + .aggregate( + Mean::new, + (group_id, newValue, mean) -> { + + if (mean.event_id == -1L) { + // only store the first one + mean.event_id = newValue.event_id; + } + mean.value = ((mean.value * mean.count) + newValue.data.field2) / (mean.count+1); + mean.count++; + + LOG.debug(group_id + + " / " + dateFormat.format(new Date(newValue.getEventTime().toEpochMilli())) + + " / new value=" + newValue.data.field2 + + " / " + mean.toString() + "\n"); + + return mean; + }, + Materialized.with(Serdes.String(), JsonSerde.MeanSerde()) + ) + .suppress(Suppressed.untilWindowCloses(Suppressed.BufferConfig.unbounded())) + .toStream() + .map((groupIddWindowed, mean) -> { + + if (mean == null) + return null; + + if (LOG.isDebugEnabled()) { + LOG.debug(groupIddWindowed.key() + + " / win[" + dateFormat.format(new Date(groupIddWindowed.window().start())) + + "," + dateFormat.format(new Date(groupIddWindowed.window().end())) + "]" + + " / " + mean.toString() + "\n"); + + } + + FeatureOutput feature = new FeatureOutput(mean.event_id, + groupIddWindowed.window().start(), + groupIddWindowed.key(), + mean.value); + + LOG.info("Stream to bench1_kafka_dsl" + " - " + gson.toJson(groupIddWindowed.key()) + "/ " + + gson.toJson(feature) + "\n"); + + return new KeyValue<>(groupIddWindowed.key(), gson.toJson(feature)); + }) + .to("bench1_kafka_dsl"); + + return new KafkaStreams(builder.build(), streamsConfiguration); + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Data.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Data.java new file mode 100644 index 0000000..0920928 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Data.java @@ -0,0 +1,23 @@ +package com.makinage.kafka.dsl.benchmark; + +public class Data { + + public Data(Long field1, Double field2, Boolean field3) { + this.field1 = field1; + this.field2 = field2; + this.field3 = field3; + } + + public Long field1; + public Double field2; + public Boolean field3; + + @Override + public String toString() { + return "Data{" + + "field1=" + field1 + + ", field2=" + field2 + + ", field3=" + field3 + + '}'; + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Feature.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Feature.java new file mode 100644 index 0000000..285a640 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Feature.java @@ -0,0 +1,43 @@ +package com.makinage.kafka.dsl.benchmark; + +import java.text.DateFormat; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.time.Instant; + +public class Feature { + + static DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); + + public Feature(Long event_id, String event_time, String group_id, Data data) { + this.event_id = event_id; + this.event_time = event_time; + this.group_id = group_id; + this.data = data; + } + + public Long event_id; + public String event_time; + public String group_id; + public Data data; + + public Instant getEventTime() { + + try { + Instant instant = df.parse(event_time).toInstant(); + return instant; + } catch (ParseException exception) { + return Instant.EPOCH; + } + } + + @Override + public String toString() { + return "Feature{" + + "event_id=" + event_id + + ", event_time='" + event_time + '\'' + + ", group_id='" + group_id + '\'' + + ", data=" + data + + '}'; + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/FeatureOutput.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/FeatureOutput.java new file mode 100644 index 0000000..28f99d8 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/FeatureOutput.java @@ -0,0 +1,26 @@ +package com.makinage.kafka.dsl.benchmark; + +public class FeatureOutput { + + public FeatureOutput(Long event_id, Long event_time, String group_id, Double field2_mean) { + this.event_id = event_id; + this.event_time = event_time; + this.group_id = group_id; + this.field2_mean = field2_mean; + } + + public Long event_id; + public Long event_time; + public String group_id; + public Double field2_mean; + + @Override + public String toString() { + return "FeatureOutput{" + + "event_id=" + event_id + + ", event_time=" + event_time + + ", group_id='" + group_id + '\'' + + ", field2_mean=" + field2_mean + + '}'; + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Mean.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Mean.java new file mode 100644 index 0000000..b6dd33e --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/Mean.java @@ -0,0 +1,31 @@ +package com.makinage.kafka.dsl.benchmark; + +public class Mean { + + public Mean() { + value = new Double(0); + count = new Long(0); + event_id = new Long(-1); + + } + + public Mean(Double mean, Long count) { + this.value = mean; + this.count = count; + this.event_id = -1L; + } + + public Double value; + public Long count; + public Long event_id; + + + @Override + public String toString() { + return "Mean{" + + "value=" + value + + ", count=" + count + + ", event_id=" + event_id + + '}'; + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/FeatureTimestampExtractor.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/FeatureTimestampExtractor.java new file mode 100644 index 0000000..c37100e --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/FeatureTimestampExtractor.java @@ -0,0 +1,21 @@ +package com.makinage.kafka.dsl.benchmark.utils; + +import com.makinage.kafka.dsl.benchmark.Feature; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; + +public class FeatureTimestampExtractor implements TimestampExtractor { + + static final DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS"); + + @Override + public long extract(ConsumerRecord record, long l) { + if (record.value() instanceof Feature) { + return ((Feature)record.value()).getEventTime().toEpochMilli(); + } + return 0; + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJODeserializer.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJODeserializer.java new file mode 100644 index 0000000..f1feaa9 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJODeserializer.java @@ -0,0 +1,63 @@ +package com.makinage.kafka.dsl.benchmark.utils; + +import com.google.gson.Gson; +import com.google.gson.JsonSyntaxException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.LogManager; + +import java.util.Map; + +/** + * The type Json pojo deserializer. + * + * @param the type parameter + */ +public class JsonPOJODeserializer implements Deserializer { + + /** + * The Log. + */ + static final Logger LOG = LogManager.getLogger(JsonPOJODeserializer.class.getName()); + + /** + * The Gson. + */ + static final Gson gson = new Gson(); + + private Class tClass; + + /** + * Default constructor needed by Kafka + */ + public JsonPOJODeserializer() { + } + + @SuppressWarnings("unchecked") + @Override + public void configure(Map props, boolean isKey) { + tClass = (Class) props.get("JsonPOJOClass"); + } + + @Override + public T deserialize(String topic, byte[] bytes) { + if (bytes == null) + return null; + + T data; + try { + data = gson.fromJson(new String(bytes), tClass); + } catch (JsonSyntaxException e) { + LOG.error("Failed to create classOfT from Json (" + new String(bytes) + ")\n"); + throw new SerializationException(e); + } + + return data; + } + + @Override + public void close() { + + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJOSerializer.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJOSerializer.java new file mode 100644 index 0000000..009f4c4 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonPOJOSerializer.java @@ -0,0 +1,54 @@ +package com.makinage.kafka.dsl.benchmark.utils; + + +import com.google.gson.Gson; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.LogManager; + +import java.util.Map; + +/** + * The type Json pojo serializer. + * + * @param the type parameter + */ +public class JsonPOJOSerializer implements Serializer { + + /** + * The Log. + */ + static final Logger LOG = LogManager.getLogger(JsonPOJOSerializer.class.getName()); + + /** + * The Gson. + */ + static final Gson gson = new Gson(); + + /** + * Default constructor needed by Kafka + */ + public JsonPOJOSerializer() { + } + + @Override + public void configure(Map props, boolean isKey) { + } + + @Override + public byte[] serialize(String topic, T data) { + if (data == null) + return null; + + try { + return gson.toJson(data).getBytes(); + } catch (Exception e) { + throw new SerializationException("Error serializing JSON message", e); + } + } + + @Override + public void close() { + } +} diff --git a/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonSerde.java b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonSerde.java new file mode 100644 index 0000000..750c671 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/java/com/makinage/kafka/dsl/benchmark/utils/JsonSerde.java @@ -0,0 +1,48 @@ +package com.makinage.kafka.dsl.benchmark.utils; + +import com.makinage.kafka.dsl.benchmark.Feature; +import com.makinage.kafka.dsl.benchmark.Mean; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.HashMap; +import java.util.Map; + +public class JsonSerde { + + static Map serdeProps = new HashMap<>(); + + public JsonSerde() { + } + + static Serializer FeatureSerializer = new JsonPOJOSerializer<>(); + static Deserializer FeatureDeserializer = new JsonPOJODeserializer<>(); + static Serde FeatureSerde = null; + public static Serde FeatureSerde() { + if (FeatureSerde != null) + return FeatureSerde; + serdeProps.put("JsonPOJOClass", Feature.class); + FeatureSerializer.configure(serdeProps, false); + serdeProps.put("JsonPOJOClass", Feature.class); + FeatureDeserializer.configure(serdeProps, false); + FeatureSerde = Serdes.serdeFrom(FeatureSerializer, FeatureDeserializer); + return FeatureSerde; + } + + static Serializer MeanSerializer = new JsonPOJOSerializer<>(); + static Deserializer MeanDeserializer = new JsonPOJODeserializer<>(); + static Serde MeanSerde = null; + public static Serde MeanSerde() { + if (MeanSerde != null) + return MeanSerde; + serdeProps.put("JsonPOJOClass", Mean.class); + MeanSerializer.configure(serdeProps, false); + serdeProps.put("JsonPOJOClass", Mean.class); + MeanDeserializer.configure(serdeProps, false); + MeanSerde = Serdes.serdeFrom(MeanSerializer, MeanDeserializer); + return MeanSerde; + } + +} diff --git a/java/kafka-dsl-benchmark1/src/main/resources/log4j2.properties b/java/kafka-dsl-benchmark1/src/main/resources/log4j2.properties new file mode 100644 index 0000000..12c0b4d --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/main/resources/log4j2.properties @@ -0,0 +1,17 @@ +name=PropertiesConfig +monitorInterval=1 +appender = console + +appender.console.type = Console +appender.console.name = LogToConsole +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{UNIX_MILLIS} %highlight{[%-5level]} %c{1}: %msg + +rootLogger.level = error +rootLogger.appenderRefs = stdout +rootLogger.appenderRef.stdout.ref = LogToConsole + +logger.Bench1.name = com.makinage.kafka.dsl.benchmark.Bench1 +logger.Bench1.level = info +logger.Bench1.additivity = false +logger.Bench1.appenderRef.console.ref = LogToConsole diff --git a/java/kafka-dsl-benchmark1/src/test/java/com/makinage/kafka/dsl/benchmark/DataParseTest.java b/java/kafka-dsl-benchmark1/src/test/java/com/makinage/kafka/dsl/benchmark/DataParseTest.java new file mode 100644 index 0000000..df2c743 --- /dev/null +++ b/java/kafka-dsl-benchmark1/src/test/java/com/makinage/kafka/dsl/benchmark/DataParseTest.java @@ -0,0 +1,33 @@ +package com.makinage.kafka.dsl.benchmark; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.text.DateFormat; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.time.Instant; + +public class DataParseTest { + + + static final Logger LOG = LogManager.getLogger(DataParseTest.class.getName()); + + @BeforeClass + public static void setUp() throws IOException { + } + + @Test + public void testDateISO8601Parsing() throws IOException, URISyntaxException, ParseException { + + DateFormat df1 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); + Instant instant = df1.parse("2020-07-12T00:00:00").toInstant(); + + // Default parser can't be used, missing timezone (Z) + //Instant instant = Instant.parse("2020-07-12T00:00:00Z"); + } +} diff --git a/monitor.py b/monitor.py index f257bad..aebaaae 100644 --- a/monitor.py +++ b/monitor.py @@ -1,14 +1,17 @@ import subprocess import time import statistics +import sys +import getopt +def consumer_groups(group_id): def consumer_groups(): res = subprocess.check_output( - "docker exec -it benchmark_kafka_1 " - "kafka-consumer-groups --describe " - "--group makinage_benchmark " - "--bootstrap-server localhost:9092 ", + "docker exec -it benchmark_kafka_1" + " kafka-consumer-groups --describe" + " --group " + group_id + + " --bootstrap-server localhost:9092 ", shell=True ) @@ -20,20 +23,43 @@ def compute_total_lag(stats): total_lag = 0 for line in stats: line = line.split() - total_lag += int(line[5]) + if line[5] == '-': + total_lag = -1 + else : + total_lag += int(line[5]) return total_lag -def main(): +def main(argv): step = 0 rps = [] prev_total_lag = -1 + group_id = '' + + try: + opts, args = getopt.getopt(argv,"hg:",["groupid="]) + except getopt.GetoptError: + print('monitor.py -g ') + sys.exit(2) + for opt, arg in opts: + if opt == '-h': + print('monitor.py -g ') + sys.exit() + elif opt in ("-g", "--groupid"): + group_id = arg + + print('Group id minitored: ' + group_id) + while True: time.sleep(10.0) - stats = consumer_groups() + stats = consumer_groups(group_id) total_lag = compute_total_lag(stats) - if total_lag == 0: + + if total_lag == -1: + # not ready yet + continue + elif total_lag == 0: break if step >= 2: @@ -49,4 +75,4 @@ def main(): if __name__ == '__main__': - main() + main(sys.argv[1:]) diff --git a/run.sh b/run.sh index 6322a93..17c15c4 100755 --- a/run.sh +++ b/run.sh @@ -3,19 +3,35 @@ echo "Initializing datasets and benchmark..." python3 create_dataset.py docker build -t makinage-benchmark ./python + +(cd ./java/kafka-dsl-benchmark1 && mvn clean install -U) +docker build -t kafka-dsl-benchmark1 ./java/kafka-dsl-benchmark1 + docker-compose up -d kafka sleep 10 + docker exec -it benchmark_kafka_1 /tmp/inject_dataset.sh echo "running python-bench1 single node..." docker-compose up -d makinage-benchmark -python3 monitor.py +python3 monitor.py -g makinage_benchmark docker-compose stop makinage-benchmark +echo "running kafka-dsl-bench1 single node..." +docker-compose up -d kafka-dsl-benchmark1 +python3 monitor.py -g kafka-dsl-benchmark1 +docker-compose stop kafka-dsl-benchmark1 + + echo "running python-bench1 with 5 nodes..." -docker-compose up --scale makinage-benchmark=5 -d -python3 monitor.py +docker-compose up --scale makinage-benchmark=5 -d makinage-benchmark +python3 monitor.py -g makinage_benchmark docker-compose stop makinage-benchmark +echo "running kafka-dsl-bench1(-scaled) with 5 nodes..." +docker-compose up --scale kafka-dsl-benchmark1-scaled=5 -d kafka-dsl-benchmark1-scaled +python3 monitor.py -g kafka-dsl-benchmark1-scaled +docker-compose stop kafka-dsl-benchmark1-scaled + echo "stopping benchmark services..." docker-compose down -v