From 6fff010f8edcd1bf6530973792bd619a584a109e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Sep 2022 11:52:11 +0100 Subject: [PATCH 1/3] feat(producer-datagen): adopt converters --- producer-datagen/pom.xml | 31 +++++++++ .../cli/producer/datagen/IntervalRunner.java | 18 +---- .../producer/datagen/PayloadGenerator.java | 65 ++++++++++++++++--- .../producer/datagen/PerformanceRunner.java | 18 ++--- .../command/ProduceIntervalCommand.java | 6 +- .../datagen/command/ProduceOnceCommand.java | 6 +- .../datagen/command/ProducePerfCommand.java | 5 +- .../datagen/command/SampleCommand.java | 4 +- 8 files changed, 105 insertions(+), 48 deletions(-) diff --git a/producer-datagen/pom.xml b/producer-datagen/pom.xml index e278081..a4ed69c 100644 --- a/producer-datagen/pom.xml +++ b/producer-datagen/pom.xml @@ -27,6 +27,16 @@ org.apache.kafka kafka-clients + + org.apache.kafka + connect-api + ${kafka.version} + + + org.apache.kafka + connect-json + ${kafka.version} + info.picocli @@ -54,6 +64,7 @@ com.fasterxml.jackson.core jackson-databind + io.confluent kafka-avro-serializer @@ -64,6 +75,26 @@ avro-random-generator ${confluent.avro.generator.version} + + io.confluent + kafka-connect-avro-data + ${confluent.version} + + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + + + io.confluent + kafka-connect-json-schema-converter + ${confluent.version} + + + io.confluent + kafka-connect-protobuf-converter + ${confluent.version} + org.graalvm.nativeimage diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/IntervalRunner.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/IntervalRunner.java index 59d7a9f..9e0cc1e 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/IntervalRunner.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/IntervalRunner.java @@ -1,20 +1,18 @@ package kafka.cli.producer.datagen; import java.io.IOException; -import kafka.cli.producer.datagen.PayloadGenerator.Format; import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; public class IntervalRunner { final Config config; - final KafkaProducer producer; + final KafkaProducer producer; final PayloadGenerator payloadGenerator; final Stats stats; public IntervalRunner( Config config, - KafkaProducer producer, + KafkaProducer producer, PayloadGenerator payloadGenerator, Stats stats ) { @@ -53,19 +51,9 @@ void loop(int count) { } void runOnce() { - var payload = payloadGenerator.get(); - var key = payloadGenerator.key(payload); - Object value; - - if (payloadGenerator.format.equals(Format.AVRO)) { - value = payload; - } else { - value = payloadGenerator.toJson(payload); - } - var sendStartMs = System.currentTimeMillis(); var cb = stats.nextCompletion(sendStartMs, sample.length, stats); - var record = new ProducerRecord<>(config.topicName(), key, value); + var record = payloadGenerator.record(config.topicName()); producer.send(record, cb); diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java index f88f7f6..a360129 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java @@ -1,6 +1,10 @@ package kafka.cli.producer.datagen; import io.confluent.avro.random.generator.Generator; +import io.confluent.connect.avro.AvroConverter; +import io.confluent.connect.avro.AvroData; +import io.confluent.connect.json.JsonSchemaConverter; +import io.confluent.connect.protobuf.ProtobufConverter; import io.confluent.kafka.serializers.KafkaAvroSerializer; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -9,6 +13,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Random; @@ -24,6 +29,8 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.Converter; /** Datagen side */ public class PayloadGenerator { @@ -31,9 +38,13 @@ public class PayloadGenerator { final Format format; final Random random; final Generator generator; + final Converter converter; + final AvroData avroData; + final Schema avroSchema; + final org.apache.kafka.connect.data.Schema connectSchema; final String keyFieldName; - public PayloadGenerator(Config config) { + public PayloadGenerator(Config config, Properties producerConfig) { this.format = config.format(); this.random = new Random(); config @@ -45,6 +56,38 @@ public PayloadGenerator(Config config) { this.generator = new Generator.Builder().random(random).generation(config.count()).schema(config.schema()).build(); this.keyFieldName = config.keyFieldName(); + this.avroData = new AvroData(1); + this.avroSchema = config.schema(); + this.connectSchema = avroData.toConnectSchema(config.schema()); + this.converter = switch (this.format) { + case JSON -> { + var jsonConverter = new JsonConverter(); + var schemasEnabled = producerConfig.getProperty("schemas.enabled", "false"); + jsonConverter.configure(Map.of("schemas.enable", schemasEnabled, "converter.type", "value")); + yield jsonConverter; + } + case AVRO -> { + var avroConverter = new AvroConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false); + yield avroConverter; + } + case PROTOBUF -> { + var avroConverter = new ProtobufConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false); + yield avroConverter; + } + case JSON_SCHEMA -> { + var avroConverter = new JsonSchemaConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false); + yield avroConverter; + } + }; } public GenericRecord get() { @@ -60,16 +103,11 @@ public GenericRecord get() { return (GenericRecord) generatedObject; } - public ProducerRecord record(String topicName) { + public ProducerRecord record(String topicName) { final var record = get(); - - final Object value; - if (format.equals(Format.JSON)) { - value = toJson(record); - } else { - value = record; - } - return new ProducerRecord<>(topicName, key(record), value); + final var key = key(record); + final var value = value(topicName, record); + return new ProducerRecord<>(topicName, key, value); } String toJson(GenericRecord record) { @@ -120,6 +158,11 @@ public String keyFieldName() { return keyFieldName; } + public byte[] value(String topicName, GenericRecord payload) { + final var schemaAndValue = avroData.toConnectData(avroSchema, payload); + return converter.fromConnectData(topicName, schemaAndValue.schema(), schemaAndValue.value()); + } + public record Config( Optional randomSeed, Optional quickstart, @@ -175,6 +218,8 @@ public String keyFieldName() { public enum Format { JSON, AVRO, + JSON_SCHEMA, + PROTOBUF } @SuppressWarnings("unchecked") diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PerformanceRunner.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PerformanceRunner.java index faadb40..1381459 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PerformanceRunner.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PerformanceRunner.java @@ -2,7 +2,6 @@ import java.util.Map; import java.util.TreeMap; -import kafka.cli.producer.datagen.PayloadGenerator.Format; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -14,14 +13,14 @@ public class PerformanceRunner { final Config config; - final KafkaProducer producer; + final KafkaProducer producer; final PayloadGenerator payloadGenerator; final ThroughputThrottler throttler; final Stats stats; public PerformanceRunner( final Config config, - final KafkaProducer producer, + final KafkaProducer producer, final PayloadGenerator payloadGenerator, final ThroughputThrottler throughputThrottler, final Stats stats @@ -37,7 +36,7 @@ public void start() { GenericRecord payload; Object value; String key; - ProducerRecord record; + ProducerRecord record; int currentTransactionSize = 0; long transactionStartTime = 0; @@ -45,21 +44,12 @@ public void start() { var sample = payloadGenerator.sample(); for (long i = 0; i < config.records(); i++) { - payload = payloadGenerator.get(); - key = payloadGenerator.key(payload); - - if (payloadGenerator.format.equals(Format.AVRO)) { - value = payload; - } else { - value = payloadGenerator.toJson(payload); - } - if (config.transactionsEnabled() && currentTransactionSize == 0) { producer.beginTransaction(); transactionStartTime = System.currentTimeMillis(); } - record = new ProducerRecord<>(config.topicName(), key, value); + record = payloadGenerator.record(config.topicName()); var sendStartMs = System.currentTimeMillis(); var cb = stats.nextCompletion(sendStartMs, sample.length, stats); diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceIntervalCommand.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceIntervalCommand.java index af788b7..362dcac 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceIntervalCommand.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceIntervalCommand.java @@ -10,7 +10,7 @@ import kafka.cli.producer.datagen.PayloadGenerator; import kafka.cli.producer.datagen.Stats; import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import picocli.CommandLine; @@ -48,10 +48,10 @@ public Integer call() throws Exception { producerConfig.putAll(additionalProperties); var keySerializer = new StringSerializer(); - Serializer valueSerializer = PayloadGenerator.valueSerializer(schemaOpts.format(), producerConfig); + var valueSerializer = new ByteArraySerializer(); try (var producer = new KafkaProducer<>(producerConfig, keySerializer, valueSerializer)) { - final var payloadGenerator = new PayloadGenerator(schemaOpts.config()); + final var payloadGenerator = new PayloadGenerator(schemaOpts.config(), producerConfig); final var stats = new Stats(numRecords, reportingIntervalMs); final var config = new IntervalRunner.Config(topicName, numRecords, intervalMs); diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceOnceCommand.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceOnceCommand.java index 46075e9..56c1025 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceOnceCommand.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProduceOnceCommand.java @@ -8,7 +8,7 @@ import kafka.cli.producer.datagen.Cli; import kafka.cli.producer.datagen.PayloadGenerator; import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import picocli.CommandLine; @@ -33,10 +33,10 @@ public Integer call() throws Exception { if (producerConfig == null) return 1; producerConfig.putAll(additionalProperties); var keySerializer = new StringSerializer(); - Serializer valueSerializer = PayloadGenerator.valueSerializer(schemaOpts.format(), producerConfig); + var valueSerializer = new ByteArraySerializer(); try (var producer = new KafkaProducer<>(producerConfig, keySerializer, valueSerializer)) { - var pg = new PayloadGenerator(schemaOpts.config()); + var pg = new PayloadGenerator(schemaOpts.config(), producerConfig); out.println("Avro Schema used to generate records:"); out.println(pg.schema()); diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProducePerfCommand.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProducePerfCommand.java index 97a2000..9df3d00 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProducePerfCommand.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/ProducePerfCommand.java @@ -11,6 +11,7 @@ import kafka.cli.producer.datagen.Stats; import kafka.cli.producer.datagen.ThroughputThrottler; import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import picocli.CommandLine; @@ -52,7 +53,7 @@ public Integer call() { producerConfig.putAll(additionalProperties); var keySerializer = new StringSerializer(); - var valueSerializer = PayloadGenerator.valueSerializer(schemaOpts.format(), producerConfig); + var valueSerializer = new ByteArraySerializer(); try (var producer = new KafkaProducer<>(producerConfig, keySerializer, valueSerializer)) { final var config = new PerformanceRunner.Config( @@ -62,7 +63,7 @@ public Integer call() { transactionDurationMs, shouldPrintMetrics ); - final var payloadGenerator = new PayloadGenerator(schemaOpts.config()); + final var payloadGenerator = new PayloadGenerator(schemaOpts.config(), producerConfig); final var throughputThrottler = new ThroughputThrottler(System.currentTimeMillis(), throughput); final var stats = new Stats(numRecords, reportingIntervalMs); diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/SampleCommand.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/SampleCommand.java index 0892978..b727103 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/SampleCommand.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/command/SampleCommand.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Optional; +import java.util.Properties; import java.util.concurrent.Callable; import kafka.cli.producer.datagen.Cli; import kafka.cli.producer.datagen.PayloadGenerator; @@ -33,7 +34,8 @@ public Integer call() throws Exception { 1, PayloadGenerator.Format.JSON, null - ) + ), + new Properties() ); if (schema) { final var schema = json.readTree(payloadGenerator.schema()); From 268ab4d86e307dad63638ba646612a3b04e2746a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Sep 2022 11:58:25 +0100 Subject: [PATCH 2/3] fix format --- .../producer/datagen/PayloadGenerator.java | 64 ++++++++++--------- 1 file changed, 34 insertions(+), 30 deletions(-) diff --git a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java index a360129..8ed034a 100644 --- a/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java +++ b/producer-datagen/src/main/java/kafka/cli/producer/datagen/PayloadGenerator.java @@ -59,35 +59,39 @@ public PayloadGenerator(Config config, Properties producerConfig) { this.avroData = new AvroData(1); this.avroSchema = config.schema(); this.connectSchema = avroData.toConnectSchema(config.schema()); - this.converter = switch (this.format) { - case JSON -> { - var jsonConverter = new JsonConverter(); - var schemasEnabled = producerConfig.getProperty("schemas.enabled", "false"); - jsonConverter.configure(Map.of("schemas.enable", schemasEnabled, "converter.type", "value")); - yield jsonConverter; - } - case AVRO -> { - var avroConverter = new AvroConverter(); - avroConverter.configure( - producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), - false); - yield avroConverter; - } - case PROTOBUF -> { - var avroConverter = new ProtobufConverter(); - avroConverter.configure( - producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), - false); - yield avroConverter; - } - case JSON_SCHEMA -> { - var avroConverter = new JsonSchemaConverter(); - avroConverter.configure( - producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), - false); - yield avroConverter; - } - }; + this.converter = + switch (this.format) { + case JSON -> { + var jsonConverter = new JsonConverter(); + var schemasEnabled = producerConfig.getProperty("schemas.enabled", "false"); + jsonConverter.configure(Map.of("schemas.enable", schemasEnabled, "converter.type", "value")); + yield jsonConverter; + } + case AVRO -> { + var avroConverter = new AvroConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false + ); + yield avroConverter; + } + case PROTOBUF -> { + var avroConverter = new ProtobufConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false + ); + yield avroConverter; + } + case JSON_SCHEMA -> { + var avroConverter = new JsonSchemaConverter(); + avroConverter.configure( + producerConfig.keySet().stream().collect(Collectors.toMap(String::valueOf, producerConfig::get)), + false + ); + yield avroConverter; + } + }; } public GenericRecord get() { @@ -219,7 +223,7 @@ public enum Format { JSON, AVRO, JSON_SCHEMA, - PROTOBUF + PROTOBUF, } @SuppressWarnings("unchecked") From 03e67a677c83e633dc6be17643cd661aeb8ee476 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Sep 2022 12:09:46 +0100 Subject: [PATCH 3/3] fix: add joda to build-time deps --- producer-datagen/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/producer-datagen/pom.xml b/producer-datagen/pom.xml index a4ed69c..5232341 100644 --- a/producer-datagen/pom.xml +++ b/producer-datagen/pom.xml @@ -227,7 +227,7 @@ --initialize-at-run-time=org.apache.kafka.common.security.authenticator.SaslClientAuthenticator - --initialize-at-build-time=${graalvm.at-build-time.slf4j},${graalvm.at-build-time.kafka},${graalvm.at-build-time.json} + --initialize-at-build-time=${graalvm.at-build-time.slf4j},${graalvm.at-build-time.kafka},${graalvm.at-build-time.json},org.joda.time --enable-url-protocols=http --enable-url-protocols=https