├── project ├── build.properties ├── plugins.sbt └── Dependencies.scala ├── .git-blame-ignore-revs ├── src ├── main │ ├── scala │ │ └── org │ │ │ └── galaxio │ │ │ └── gatling │ │ │ └── kafka │ │ │ ├── Predef.scala │ │ │ ├── request │ │ │ ├── builder │ │ │ │ ├── RequestBuilder.scala │ │ │ │ ├── KafkaRequestBuilder.scala │ │ │ │ ├── KafkaAttributes.scala │ │ │ │ └── KafkaRequestBuilderBase.scala │ │ │ ├── KafkaProtocolMessage.scala │ │ │ └── KafkaSerdesImplicits.scala │ │ │ ├── package.scala │ │ │ ├── protocol │ │ │ ├── KafkaComponents.scala │ │ │ ├── KafkaProtocolBuilderBackwardCompatible.scala │ │ │ ├── KafkaProtocolBuilder.scala │ │ │ └── KafkaProtocol.scala │ │ │ ├── actions │ │ │ ├── KafkaRequestActionBuilder.scala │ │ │ ├── KafkaRequestReplyActionBuilder.scala │ │ │ ├── KafkaRequestReplyAction.scala │ │ │ ├── KafkaRequestAction.scala │ │ │ └── KafkaAction.scala │ │ │ ├── checks │ │ │ ├── KafkaMessageCheck.scala │ │ │ ├── AvroBodyCheckBuilder.scala │ │ │ ├── KafkaMessagePreparer.scala │ │ │ ├── KafkaCheckMaterializer.scala │ │ │ └── KafkaCheckSupport.scala │ │ │ ├── client │ │ │ ├── KafkaSender.scala │ │ │ ├── KafkaMessageTrackerPool.scala │ │ │ ├── DynamicKafkaConsumer.scala │ │ │ └── KafkaMessageTracker.scala │ │ │ ├── KafkaDsl.scala │ │ │ └── avro4s.scala │ └── java │ │ └── org │ │ └── galaxio │ │ └── gatling │ │ └── kafka │ │ └── javaapi │ │ ├── request │ │ ├── expressions │ │ │ ├── JExpression.java │ │ │ ├── ExpressionBuilder.java │ │ │ └── Builders.java │ │ └── builder │ │ │ ├── RequestBuilder.java │ │ │ ├── ReqRepBase.java │ │ │ ├── RRInTopicStep.java │ │ │ ├── RequestReplyBuilder.java │ │ │ ├── OnlyPublishStep.java │ │ │ ├── KafkaRequestBuilderBase.java │ │ │ └── RROutTopicStep.java │ │ ├── checks │ │ ├── KafkaCheckType.java │ │ └── KafkaChecks.scala │ │ ├── protocol │ │ ├── KafkaProtocolBuilderNewBase.java │ │ ├── KPProducerSettingsStep.java │ │ ├── KafkaProtocolBuilderBackwardCompatible.java │ │ ├── KafkaProtocolBuilderPropertiesStep.java │ │ ├── KafkaProtocolBuilder.java │ │ ├── KPConsumeSettingsStep.java │ │ └── KafkaProtocolBuilderBase.java │ │ └── KafkaDsl.java └── test │ ├── resources │ └── logback.xml │ ├── scala │ └── org │ │ └── galaxio │ │ └── gatling │ │ └── kafka │ │ └── examples │ │ ├── GatlingRunner.scala │ │ ├── ProducerSimulation.scala │ │ ├── Avro4sSimulation.scala │ │ ├── MatchSimulation.scala │ │ ├── KafkaJavaapiMethodsGatlingTest.scala │ │ ├── AvroClassWithRequestReplySimulation.scala │ │ ├── BasicSimulation.scala │ │ └── KafkaGatlingTest.scala │ ├── java │ └── org │ │ └── galaxio │ │ └── gatling │ │ └── kafka │ │ └── javaapi │ │ └── examples │ │ ├── ProducerSimulation.java │ │ ├── BasicSimulation.java │ │ ├── AvroClassWithRequestReplySimulation.java │ │ └── MatchSimulation.java │ └── kotlin │ └── org │ └── galaxio │ └── gatling │ └── kafka │ └── javaapi │ └── examples │ ├── ProducerSimulation.kt │ ├── BasicSimulation.kt │ ├── AvroClassWithRequestReplySimulation.kt │ └── MatchSimulation.kt ├── .scalafmt.conf ├── publish.sbt ├── .gitignore ├── README.md ├── .github └── workflows │ └── ci.yml └── LICENSE /project/build.properties: -------------------------------------------------------------------------------- 1 | # suppress inspection "UnusedProperty" 2 | sbt.version=1.11.4 -------------------------------------------------------------------------------- /.git-blame-ignore-revs: -------------------------------------------------------------------------------- 1 | # Scala Steward: Reformat with scalafmt 3.7.1 2 | 3be9944a4bdcf03615c2357d5c32c62a713d2ce7 3 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/Predef.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka 2 | 3 | object Predef extends KafkaDsl 4 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | runner.dialect = "scala213" 2 | version = 3.9.9 3 | binPack.parentConstructors = true 4 | maxColumn = 128 5 | includeCurlyBraceInSelectChains = false 6 | align.preset = most 7 | trailingCommas = always -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/builder/RequestBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.action.builder.ActionBuilder 4 | 5 | trait RequestBuilder[+K, +V] { 6 | 7 | def build: ActionBuilder 8 | } 9 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/expressions/JExpression.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.expressions; 2 | 3 | @FunctionalInterface 4 | public interface JExpression extends java.util.function.Function { 5 | } -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/checks/KafkaCheckType.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.checks; 2 | 3 | import io.gatling.javaapi.core.CheckBuilder.CheckType; 4 | 5 | public enum KafkaCheckType implements CheckType { 6 | ResponseCode, 7 | Simple 8 | } 9 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderNewBase.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import java.util.Map; 4 | 5 | public class KafkaProtocolBuilderNewBase { 6 | 7 | public KPProducerSettingsStep producerSettings(Map ps) { 8 | return new KPProducerSettingsStep(ps); 9 | } 10 | 11 | } 12 | 13 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KPProducerSettingsStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import java.util.Map; 4 | 5 | public class KPProducerSettingsStep { 6 | 7 | private final Map ps; 8 | 9 | public KPProducerSettingsStep(Map ps) { 10 | this.ps = ps; 11 | } 12 | 13 | public KPConsumeSettingsStep consumeSettings(Map cs) { 14 | return new KPConsumeSettingsStep(ps, cs); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/builder/KafkaRequestBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.action.builder.ActionBuilder 4 | import org.galaxio.gatling.kafka.actions.KafkaRequestActionBuilder 5 | 6 | import scala.reflect.ClassTag 7 | 8 | case class KafkaRequestBuilder[+K: ClassTag, +V: ClassTag](attributes: KafkaAttributes[K, V]) extends RequestBuilder[K, V] { 9 | 10 | def build: ActionBuilder = new KafkaRequestActionBuilder(attributes) 11 | 12 | } 13 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/package.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling 2 | 3 | import com.typesafe.scalalogging.StrictLogging 4 | import io.gatling.core.check.Check 5 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 6 | 7 | package object kafka { 8 | type KafkaCheck = Check[KafkaProtocolMessage] 9 | 10 | trait KafkaLogging extends StrictLogging { 11 | def logMessage(text: => String, msg: KafkaProtocolMessage): Unit = { 12 | logger.debug(text) 13 | logger.trace(msg.toString) 14 | } 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/RequestBuilder.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.javaapi.core.ActionBuilder; 4 | 5 | public class RequestBuilder implements ActionBuilder { 6 | 7 | private final org.galaxio.gatling.kafka.request.builder.RequestBuilder wrapped; 8 | 9 | public RequestBuilder(org.galaxio.gatling.kafka.request.builder.RequestBuilder wrapped) { 10 | this.wrapped = wrapped; 11 | } 12 | 13 | @Override 14 | public io.gatling.core.action.builder.ActionBuilder asScala() { 15 | return wrapped.build(); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers ++= Seq( 2 | // need for load sbt-schema-registry-plugin dependencies 3 | "Confluent" at "https://packages.confluent.io/maven/", 4 | ) 5 | libraryDependencies += "org.apache.avro" % "avro-compiler" % "1.11.4" 6 | 7 | addSbtPlugin("com.github.sbt" % "sbt-ci-release" % "1.11.1") 8 | addSbtPlugin("io.gatling" % "gatling-sbt" % "4.13.3") 9 | addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.5.5") 10 | addSbtPlugin("com.github.sbt" % "sbt-avro" % "4.0.1") 11 | addSbtPlugin("org.galaxio" % "sbt-schema-registry-plugin" % "0.5.6") 12 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "2.3.1") 13 | -------------------------------------------------------------------------------- /src/test/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | %d{HH:mm:ss.SSS} [%-5level] %logger{15} - %msg%n%rEx 7 | 8 | false 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/protocol/KafkaComponents.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.protocol 2 | 3 | import io.gatling.core.CoreComponents 4 | import io.gatling.core.protocol.ProtocolComponents 5 | import io.gatling.core.session.Session 6 | import org.galaxio.gatling.kafka.client.{KafkaSender, KafkaMessageTrackerPool} 7 | 8 | case class KafkaComponents( 9 | coreComponents: CoreComponents, 10 | kafkaProtocol: KafkaProtocol, 11 | trackersPool: Option[KafkaMessageTrackerPool], 12 | sender: KafkaSender, 13 | ) extends ProtocolComponents { 14 | 15 | override def onStart: Session => Session = Session.Identity 16 | 17 | override def onExit: Session => Unit = ProtocolComponents.NoopOnExit 18 | } 19 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderBackwardCompatible.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import io.gatling.core.protocol.Protocol; 4 | import io.gatling.javaapi.core.ProtocolBuilder; 5 | 6 | public class KafkaProtocolBuilderBackwardCompatible implements ProtocolBuilder { 7 | 8 | private final org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilderBackwardCompatible wrapped; 9 | 10 | public KafkaProtocolBuilderBackwardCompatible(org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilderBackwardCompatible wrapped) { 11 | this.wrapped = wrapped; 12 | } 13 | 14 | @Override 15 | public Protocol protocol() { 16 | return wrapped.build(); 17 | } 18 | 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/builder/KafkaAttributes.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.session.Expression 4 | import org.apache.kafka.common.header.Headers 5 | import org.apache.kafka.common.serialization.Serde 6 | import org.galaxio.gatling.kafka.KafkaCheck 7 | 8 | final case class KafkaAttributes[+K, +V]( 9 | requestName: Expression[String], 10 | producerTopic: Option[Expression[String]], 11 | consumerTopic: Option[Expression[String]], 12 | key: Option[Expression[? <: K]], 13 | value: Expression[? <: V], 14 | headers: Option[Expression[Headers]], 15 | keySerde: Option[Serde[? <: K]], 16 | valueSerde: Serde[? <: V], 17 | checks: List[KafkaCheck], 18 | ) 19 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/GatlingRunner.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.app.Gatling 4 | import io.gatling.shared.cli.GatlingCliOptions 5 | 6 | object GatlingRunner { 7 | 8 | def main(args: Array[String]): Unit = { 9 | 10 | // this is where you specify the class you want to run 11 | val simulationClass = classOf[BasicSimulation].getName 12 | 13 | Gatling.main( 14 | args ++ 15 | Array( 16 | GatlingCliOptions.Simulation.shortOption, 17 | simulationClass, 18 | GatlingCliOptions.ResultsFolder.shortOption, 19 | "results", 20 | GatlingCliOptions.Launcher.shortOption, 21 | "sbt", 22 | ), 23 | ) 24 | } 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/ReqRepBase.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.javaapi.core.internal.Expressions; 4 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression; 5 | 6 | public class ReqRepBase { 7 | 8 | private final String requestName; 9 | 10 | public ReqRepBase(String requestName) { 11 | this.requestName = requestName; 12 | } 13 | 14 | public RRInTopicStep requestTopic(String inputTopic) { 15 | return new RRInTopicStep(Expressions.toStringExpression(inputTopic), this.requestName); 16 | } 17 | 18 | public RRInTopicStep requestTopic(JExpression inputTopic) { 19 | return new RRInTopicStep(Expressions.javaFunctionToExpression(inputTopic), this.requestName); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/protocol/KafkaProtocolBuilderBackwardCompatible.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.protocol 2 | 3 | import org.apache.kafka.clients.producer.ProducerConfig 4 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher 5 | 6 | import scala.concurrent.duration.FiniteDuration 7 | 8 | final case class KafkaProtocolBuilderBackwardCompatible(topic: String, props: Map[String, AnyRef], timeout: FiniteDuration) { 9 | 10 | def build: KafkaProtocol = { 11 | val serializers = Map( 12 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 13 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 14 | ) 15 | KafkaProtocol(topic, props ++ serializers, Map.empty, timeout, KafkaKeyMatcher) 16 | } 17 | 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/actions/KafkaRequestActionBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.actions 2 | 3 | import io.gatling.core.action.Action 4 | import io.gatling.core.action.builder.ActionBuilder 5 | import io.gatling.core.structure.ScenarioContext 6 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 7 | import org.galaxio.gatling.kafka.request.builder.KafkaAttributes 8 | 9 | import scala.reflect.ClassTag 10 | 11 | final class KafkaRequestActionBuilder[+K: ClassTag, +V: ClassTag](attributes: KafkaAttributes[K, V]) extends ActionBuilder { 12 | 13 | override def build(ctx: ScenarioContext, next: Action): Action = { 14 | 15 | val kafkaComponents = 16 | ctx.protocolComponentsRegistry.components(KafkaProtocol.kafkaProtocolKey) 17 | 18 | new KafkaRequestAction( 19 | kafkaComponents, 20 | attributes, 21 | ctx.coreComponents, 22 | next, 23 | ctx.coreComponents.throttler.filter(_ => ctx.throttled), 24 | ) 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/checks/KafkaMessageCheck.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.checks 2 | 3 | import io.gatling.commons.validation.Validation 4 | import io.gatling.core.check.Check.PreparedCache 5 | import io.gatling.core.check.{Check, CheckResult} 6 | import io.gatling.core.session.{Expression, Session} 7 | import org.galaxio.gatling.kafka.KafkaCheck 8 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 9 | 10 | case class KafkaMessageCheck(wrapped: KafkaCheck) extends KafkaCheck { 11 | override def check(response: KafkaProtocolMessage, session: Session, preparedCache: PreparedCache): Validation[CheckResult] = 12 | wrapped.check(response, session, preparedCache) 13 | 14 | override def checkIf(condition: Expression[Boolean]): Check[KafkaProtocolMessage] = copy(wrapped.checkIf(condition)) 15 | 16 | override def checkIf(condition: (KafkaProtocolMessage, Session) => Validation[Boolean]): Check[KafkaProtocolMessage] = 17 | copy(wrapped.checkIf(condition)) 18 | } 19 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderPropertiesStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import java.util.Map; 4 | 5 | import static scala.jdk.javaapi.CollectionConverters.asScala; 6 | 7 | public class KafkaProtocolBuilderPropertiesStep { 8 | 9 | private final String topic; 10 | private Map props; 11 | 12 | public KafkaProtocolBuilderPropertiesStep(String topic, Map props) { 13 | this.topic = topic; 14 | this.props = props; 15 | } 16 | 17 | public KafkaProtocolBuilderBackwardCompatible properties(Map props) { 18 | this.props = props; 19 | scala.collection.immutable.Map scalaMap = scala.collection.immutable.Map.from(asScala(this.props)); 20 | return new KafkaProtocolBuilderBackwardCompatible( 21 | new org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder.KafkaProtocolBuilderPropertiesStep(this.topic) 22 | .properties(scalaMap) 23 | ); 24 | } 25 | 26 | 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KafkaProtocolBuilder.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import io.gatling.core.protocol.Protocol; 4 | import io.gatling.javaapi.core.ProtocolBuilder; 5 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage; 6 | import scala.Function1; 7 | 8 | public class KafkaProtocolBuilder implements ProtocolBuilder { 9 | 10 | private org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder wrapped; 11 | 12 | public KafkaProtocolBuilder(org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder wrapped) { 13 | this.wrapped = wrapped; 14 | } 15 | 16 | public KafkaProtocolBuilder matchByValue() { 17 | this.wrapped = wrapped.matchByValue(); 18 | return this; 19 | } 20 | 21 | public KafkaProtocolBuilder matchByMessage(Function1 keyExtractor) { 22 | this.wrapped = wrapped.matchByMessage(keyExtractor); 23 | return this; 24 | } 25 | 26 | @Override 27 | public Protocol protocol() { 28 | return wrapped.build(); 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /publish.sbt: -------------------------------------------------------------------------------- 1 | ThisBuild / versionScheme := Some("semver-spec") 2 | ThisBuild / organization := "org.galaxio" 3 | ThisBuild / organizationName := "Galaxio Team" 4 | ThisBuild / organizationHomepage := Some(url("https://github.com/galax-io")) 5 | ThisBuild / description := "Plugin to support kafka performance testing in Gatling." 6 | ThisBuild / homepage := Some(url("https://github.com/galax-io/gatling-kafka-plugin")) 7 | ThisBuild / scmInfo := Some( 8 | ScmInfo( 9 | url("https://github.com/galax-io/gatling-kafka-plugin"), 10 | "scm:git:git@github.com:galax-io/gatling-kafka-plugin.git", 11 | ), 12 | ) 13 | 14 | ThisBuild / scalaVersion := "2.13.16" 15 | 16 | ThisBuild / developers := List( 17 | Developer( 18 | id = "jigarkhwar", 19 | name = "Ioann Akhaltsev", 20 | email = "jigarkhwar88@gmail.com", 21 | url = url("https://github.com/jigarkhwar"), 22 | ), 23 | ) 24 | 25 | // Remove all additional repository other than Maven Central from POM 26 | ThisBuild / pomIncludeRepository := { _ => false } 27 | ThisBuild / licenses += ("Apache-2.0", url("https://www.apache.org/licenses/LICENSE-2.0")) 28 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/RRInTopicStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | 4 | import io.gatling.commons.validation.Validation; 5 | import io.gatling.core.session.Session; 6 | import io.gatling.javaapi.core.internal.Expressions; 7 | 8 | import java.util.function.Function; 9 | 10 | public class RRInTopicStep { 11 | 12 | private final scala.Function1> inputTopic; 13 | private final String requestName; 14 | 15 | public RRInTopicStep(scala.Function1> inputTopic, String requestName) { 16 | this.inputTopic = inputTopic; 17 | this.requestName = requestName; 18 | } 19 | 20 | public RROutTopicStep replyTopic(String outputTopic) { 21 | return new RROutTopicStep(this.inputTopic, Expressions.toStringExpression(outputTopic), this.requestName); 22 | } 23 | 24 | public RROutTopicStep replyTopic(Function outputTopic) { 25 | return new RROutTopicStep(this.inputTopic, Expressions.javaFunctionToExpression(outputTopic), this.requestName); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/RequestReplyBuilder.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.javaapi.core.ActionBuilder; 4 | import io.gatling.javaapi.core.CheckBuilder; 5 | import org.galaxio.gatling.kafka.javaapi.checks.KafkaChecks; 6 | 7 | import java.util.Arrays; 8 | import java.util.List; 9 | 10 | public class RequestReplyBuilder implements ActionBuilder { 11 | 12 | private org.galaxio.gatling.kafka.actions.KafkaRequestReplyActionBuilder wrapped; 13 | 14 | public RequestReplyBuilder(org.galaxio.gatling.kafka.actions.KafkaRequestReplyActionBuilder wrapped) { 15 | this.wrapped = wrapped; 16 | } 17 | 18 | public RequestReplyBuilder check(CheckBuilder... checks) { 19 | return check(Arrays.asList(checks)); 20 | } 21 | 22 | public RequestReplyBuilder check(List checks) { 23 | this.wrapped = wrapped.check(KafkaChecks.toScalaChecks(checks)); 24 | return this; 25 | } 26 | 27 | @Override 28 | public io.gatling.core.action.builder.ActionBuilder asScala() { 29 | return wrapped; 30 | } 31 | } -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/actions/KafkaRequestReplyActionBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.actions 2 | 3 | import io.gatling.core.action.Action 4 | import io.gatling.core.action.builder.ActionBuilder 5 | import io.gatling.core.structure.ScenarioContext 6 | import io.gatling.internal.quicklens._ 7 | import org.galaxio.gatling.kafka.KafkaCheck 8 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 9 | import org.galaxio.gatling.kafka.request.builder.KafkaAttributes 10 | 11 | import scala.reflect.ClassTag 12 | 13 | case class KafkaRequestReplyActionBuilder[+K: ClassTag, +V: ClassTag](attributes: KafkaAttributes[K, V]) extends ActionBuilder { 14 | def check(checks: KafkaCheck*): KafkaRequestReplyActionBuilder[K, V] = 15 | this.modify(_.attributes.checks).using(_ ::: checks.toList) 16 | 17 | override def build(ctx: ScenarioContext, next: Action): Action = { 18 | val kafkaComponents = ctx.protocolComponentsRegistry.components(KafkaProtocol.kafkaProtocolKey) 19 | new KafkaRequestReplyAction[K, V]( 20 | kafkaComponents, 21 | attributes, 22 | ctx.coreComponents, 23 | next, 24 | ctx.coreComponents.throttler.filter(_ => ctx.throttled), 25 | ) 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Created by .ignore support plugin (hsz.mobi) 2 | ### JetBrains template 3 | # Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and WebStorm 4 | # Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839 5 | 6 | # User-specific stuff 7 | .idea/**/workspace.xml 8 | .idea/**/tasks.xml 9 | .idea/**/usage.statistics.xml 10 | .idea/**/dictionaries 11 | .idea/**/shelf 12 | 13 | # Generated files 14 | .idea/**/contentModel.xml 15 | 16 | # Sensitive or high-churn files 17 | .idea/**/dataSources/ 18 | .idea/**/dataSources.ids 19 | .idea/**/dataSources.local.xml 20 | .idea/**/sqlDataSources.xml 21 | .idea/**/dynamic.xml 22 | .idea/**/uiDesigner.xml 23 | .idea/**/dbnavigator.xml 24 | 25 | # File-based project format 26 | *.iws 27 | 28 | # IntelliJ 29 | out/ 30 | 31 | # mpeltonen/sbt-idea plugin 32 | .idea_modules/ 33 | 34 | ### SBT template 35 | # Simple Build Tool 36 | # http://www.scala-sbt.org/release/docs/Getting-Started/Directories.html#configuring-version-control 37 | 38 | dist/* 39 | target/ 40 | lib_managed/ 41 | src_managed/ 42 | project/boot/ 43 | project/plugins/project/ 44 | .cache 45 | .lib/ 46 | 47 | ### Scala template 48 | *.log 49 | 50 | /.idea/ 51 | /.bsp/ 52 | .DS_Store 53 | /results 54 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/ProducerSimulation.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.core.Predef._ 4 | import io.gatling.core.structure.ScenarioBuilder 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.galaxio.gatling.kafka.Predef._ 7 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 8 | 9 | class ProducerSimulation extends Simulation { 10 | 11 | val kafkaConsumerConf: KafkaProtocol = 12 | kafka 13 | .properties( 14 | Map( 15 | ProducerConfig.ACKS_CONFIG -> "1", 16 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 17 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.DoubleSerializer", 18 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 19 | ), 20 | ) 21 | 22 | val scn: ScenarioBuilder = scenario("Basic") 23 | .exec( 24 | kafka("BasicRequest") 25 | .topic("test.topic") 26 | .send[Double](1.16423), 27 | ) 28 | .exec(kafka("BasicRequestWithKey").topic("test.topic").send[String, Double]("true", 12.0)) 29 | 30 | setUp(scn.inject(atOnceUsers(5))).protocols(kafkaConsumerConf) 31 | 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/expressions/ExpressionBuilder.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.expressions; 2 | 3 | import io.gatling.commons.validation.Validation; 4 | import io.gatling.javaapi.core.internal.Expressions; 5 | import org.apache.kafka.common.serialization.Serde; 6 | 7 | public abstract class ExpressionBuilder { 8 | private final JExpression javaExpression; 9 | private final Class type; 10 | private final Serde serde; 11 | 12 | protected ExpressionBuilder(JExpression javaExpression, Class type, Serde serde) { 13 | this.javaExpression = javaExpression; 14 | this.type = type; 15 | this.serde = serde; 16 | } 17 | 18 | scala.Function1> bytes(String topic) { 19 | return Expressions.javaFunctionToExpression(javaExpression.andThen(v -> serde.serializer().serialize(topic, v))); 20 | } 21 | 22 | public scala.Function1> gatlingExpression() { 23 | return Expressions.javaFunctionToExpression(javaExpression); 24 | } 25 | 26 | public Class getType() { 27 | return type; 28 | } 29 | 30 | public Serde getSerde() { 31 | return serde; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/client/KafkaSender.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.client 2 | 3 | import org.apache.kafka.clients.producer.{KafkaProducer, Producer, RecordMetadata} 4 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 5 | 6 | import scala.jdk.CollectionConverters._ 7 | 8 | trait KafkaSender { 9 | def send(protocolMessage: KafkaProtocolMessage)( 10 | onSuccess: RecordMetadata => Unit, 11 | onFailure: Throwable => Unit, 12 | ): Unit 13 | def close(): Unit 14 | } 15 | 16 | object KafkaSender { 17 | private final class Impl(producer: Producer[Array[Byte], Array[Byte]]) extends KafkaSender { 18 | override def send( 19 | protocolMessage: KafkaProtocolMessage, 20 | )(onSuccess: RecordMetadata => Unit, onFailure: Throwable => Unit): Unit = { 21 | producer.send( 22 | protocolMessage.toProducerRecord, 23 | (metadata: RecordMetadata, exception: Throwable) => 24 | if (exception == null) 25 | onSuccess(metadata) 26 | else 27 | onFailure(exception), 28 | ) 29 | 30 | } 31 | 32 | override def close(): Unit = 33 | producer.close() 34 | 35 | } 36 | 37 | def apply(producerSettings: Map[String, AnyRef]): KafkaSender = { 38 | val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerSettings.asJava) 39 | new Impl(producer) 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/checks/AvroBodyCheckBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.checks 2 | 3 | import io.gatling.commons.validation.{TryWrapper, Validation} 4 | import io.gatling.core.check.CheckBuilder.Find 5 | import io.gatling.core.check.{CheckBuilder, CheckMaterializer, Extractor} 6 | import io.gatling.core.session.ExpressionSuccessWrapper 7 | import org.apache.avro.generic.GenericRecord 8 | import org.apache.kafka.common.serialization.Serde 9 | import org.galaxio.gatling.kafka.KafkaCheck 10 | import org.galaxio.gatling.kafka.checks.KafkaCheckMaterializer.KafkaMessageCheckType 11 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 12 | 13 | import scala.util.Try 14 | 15 | object AvroBodyCheckBuilder { 16 | private type KafkaCheckMaterializer[T, P] = CheckMaterializer[T, KafkaCheck, KafkaProtocolMessage, P] 17 | 18 | def _avroBody[T: Serde]: CheckBuilder.Find[KafkaMessageCheckType, KafkaProtocolMessage, T] = { 19 | val tExtractor = new Extractor[KafkaProtocolMessage, T] { 20 | val name = "avroBody" 21 | val arity = "find" 22 | def apply(prepared: KafkaProtocolMessage): Validation[Option[T]] = { 23 | Try(Option(implicitly[Serde[T]].deserializer().deserialize(prepared.consumerTopic, prepared.value))).toValidation 24 | } 25 | }.expressionSuccess 26 | 27 | new Find.Default[KafkaMessageCheckType, KafkaProtocolMessage, T](tExtractor, displayActualValue = true) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/java/org/galaxio/gatling/kafka/javaapi/examples/ProducerSimulation.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples; 2 | 3 | import io.gatling.javaapi.core.ScenarioBuilder; 4 | import io.gatling.javaapi.core.Session; 5 | import io.gatling.javaapi.core.Simulation; 6 | import org.apache.kafka.clients.producer.ProducerConfig; 7 | import org.apache.kafka.common.header.Headers; 8 | import org.apache.kafka.common.header.internals.RecordHeaders; 9 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl; 10 | import org.galaxio.gatling.kafka.javaapi.protocol.KafkaProtocolBuilder; 11 | 12 | import java.nio.charset.Charset; 13 | import java.util.Map; 14 | import java.util.Optional; 15 | 16 | import static io.gatling.javaapi.core.CoreDsl.scenario; 17 | import static org.galaxio.gatling.kafka.javaapi.KafkaDsl.kafka; 18 | 19 | public class ProducerSimulation extends Simulation { 20 | 21 | private final KafkaProtocolBuilder kafkaConsumerConf = 22 | KafkaDsl.kafka() 23 | .properties(Map.of(ProducerConfig.ACKS_CONFIG, "1")); 24 | 25 | private Headers header(Session session) { 26 | var uuid = Optional.ofNullable(session.getString("UUID")).orElse(""); 27 | return new RecordHeaders().add("uuid-header", uuid.getBytes(Charset.defaultCharset())); 28 | } 29 | 30 | private final ScenarioBuilder scn = scenario("Basic") 31 | .exec(kafka("BasicRequest").topic("test.topic").send("foo")) 32 | .exec(kafka("dld").topic("test.topic").send("true", 12.0)) 33 | .exec(kafka("Msg1").topic("test.topic").send("key", "val", this::header)); 34 | 35 | } 36 | -------------------------------------------------------------------------------- /project/Dependencies.scala: -------------------------------------------------------------------------------- 1 | import sbt.* 2 | 3 | object Dependencies { 4 | private object Versions { 5 | val kafka = "7.9.2-ccs" 6 | val gatling = "3.13.5" 7 | val avro4s = "4.1.2" 8 | val avro = "1.12.0" 9 | val kafkaAvroSerde = "7.9.2" 10 | } 11 | 12 | lazy val gatling: Seq[ModuleID] = Seq( 13 | "io.gatling" % "gatling-core" % Versions.gatling % "provided", 14 | "io.gatling" % "gatling-core-java" % Versions.gatling % "provided", 15 | ) 16 | 17 | lazy val gatlingTest: Seq[ModuleID] = Seq( 18 | "io.gatling.highcharts" % "gatling-charts-highcharts" % Versions.gatling % "it,test", 19 | "io.gatling" % "gatling-test-framework" % Versions.gatling % "it,test", 20 | ) 21 | 22 | lazy val kafka: Seq[ModuleID] = Seq( 23 | ("org.apache.kafka" % "kafka-clients" % Versions.kafka) 24 | .exclude("org.slf4j", "slf4j-api"), 25 | ("org.apache.kafka" %% "kafka-streams-scala" % Versions.kafka) 26 | .exclude("org.slf4j", "slf4j-api"), 27 | ) 28 | 29 | lazy val avro4s: ModuleID = "com.sksamuel.avro4s" %% "avro4s-core" % Versions.avro4s % "provided" 30 | 31 | lazy val avroCompiler: ModuleID = "org.apache.avro" % "avro-compiler" % Versions.avro 32 | lazy val avroCore: ModuleID = "org.apache.avro" % "avro" % Versions.avro 33 | lazy val avroSerdes: ModuleID = 34 | ("io.confluent" % "kafka-streams-avro-serde" % Versions.kafkaAvroSerde).exclude("org.apache.kafka", "kafka-streams-scala") 35 | lazy val avroSerializers: ModuleID = "io.confluent" % "kafka-avro-serializer" % Versions.kafkaAvroSerde 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/KafkaDsl.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka 2 | 3 | import io.gatling.core.action.builder.ActionBuilder 4 | import io.gatling.core.session._ 5 | import org.apache.kafka.common.header.internals.RecordHeaders 6 | import org.apache.kafka.common.header.{Header, Headers} 7 | import org.galaxio.gatling.kafka.checks.KafkaCheckSupport 8 | import org.galaxio.gatling.kafka.protocol.{KafkaProtocol, KafkaProtocolBuilder, KafkaProtocolBuilderBackwardCompatible} 9 | import org.galaxio.gatling.kafka.request.KafkaSerdesImplicits 10 | import org.galaxio.gatling.kafka.request.builder.{KafkaRequestBuilderBase, RequestBuilder} 11 | 12 | import scala.jdk.CollectionConverters._ 13 | 14 | trait KafkaDsl extends KafkaCheckSupport with KafkaSerdesImplicits { 15 | 16 | val kafka: KafkaProtocolBuilder.type = KafkaProtocolBuilder 17 | 18 | def kafka(requestName: Expression[String]): KafkaRequestBuilderBase = 19 | KafkaRequestBuilderBase(requestName) 20 | 21 | implicit def kafkaProtocolBuilder2kafkaProtocol(builder: KafkaProtocolBuilder): KafkaProtocol = builder.build 22 | 23 | implicit def kafkaProtocolBuilderBackwardCompatible2kafkaProtocol( 24 | builder: KafkaProtocolBuilderBackwardCompatible, 25 | ): KafkaProtocol = builder.build 26 | 27 | implicit def kafkaRequestBuilder2ActionBuilder[K, V](builder: RequestBuilder[K, V]): ActionBuilder = builder.build 28 | 29 | implicit def listHeaderToHeaders(lh: Expression[List[Header]]): Expression[Headers] = lh.map(l => new RecordHeaders(l.asJava)) 30 | 31 | implicit def listHeaderToExpression(lh: List[Header]): Expression[Headers] = listHeaderToHeaders(lh.expressionSuccess) 32 | 33 | } 34 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/avro4s.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka 2 | 3 | import com.sksamuel.avro4s.{AvroInputStream, AvroOutputStream, Decoder, Encoder, SchemaFor} 4 | import org.apache.avro.Schema 5 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer, Serdes} 6 | 7 | import java.io.ByteArrayOutputStream 8 | 9 | object avro4s { 10 | private final class Avro4sGenericSerializer[T: Encoder] extends Serializer[T] { 11 | override def serialize(topic: String, data: T): Array[Byte] = 12 | if (data == null) 13 | null 14 | else { 15 | val baos = new ByteArrayOutputStream() 16 | 17 | val avroOutputStream = AvroOutputStream.binary[T] 18 | val output = avroOutputStream.to(baos).build() 19 | output.write(data) 20 | output.close() 21 | baos.toByteArray 22 | } 23 | } 24 | 25 | private final class Avro4sGenericDeserializer[T: Decoder](schema: Schema) extends Deserializer[T] { 26 | override def deserialize(topic: String, data: Array[Byte]): T = 27 | if (data == null) 28 | null.asInstanceOf[T] 29 | else { 30 | val avroInputStream = AvroInputStream.binary[T] 31 | val input = avroInputStream.from(data).build(schema) 32 | val result = input.iterator.next() 33 | input.close() 34 | result 35 | } 36 | } 37 | 38 | implicit def serdeAvro4s[T: SchemaFor: Encoder: Decoder]: Serde[T] = { 39 | val schemaFor = implicitly[SchemaFor[T]] 40 | Serdes.serdeFrom( 41 | new Avro4sGenericSerializer[T], 42 | new Avro4sGenericDeserializer[T](schemaFor.schema), 43 | ) 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/Avro4sSimulation.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.core.Predef._ 4 | import io.gatling.core.structure.ScenarioBuilder 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.galaxio.gatling.kafka.Predef._ 7 | 8 | import org.galaxio.gatling.kafka.avro4s._ 9 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 10 | 11 | class Avro4sSimulation extends Simulation { 12 | 13 | val kafkaAclConf: KafkaProtocol = kafka 14 | .properties( 15 | ProducerConfig.ACKS_CONFIG -> "1", 16 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 17 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 18 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 19 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 20 | "schema.registry.url" -> "http://localhost:9094", 21 | ) 22 | 23 | case class Ingredient(name: String, sugar: Double, fat: Double) 24 | 25 | val scn: ScenarioBuilder = scenario("Kafka Test") 26 | .exec( 27 | kafka("Simple Avro4s Request") 28 | // message to send 29 | .topic("my.acl.topic") 30 | .send[Ingredient](Ingredient("Cheese", 0d, 70d)), 31 | ) 32 | .exec( 33 | kafka("Simple Avro4s Request with Key") 34 | // message to send 35 | .topic("my.acl.topic") 36 | .send[String, Ingredient]("Key", Ingredient("Cheese", 0d, 70d)), 37 | ) 38 | 39 | setUp(scn.inject(atOnceUsers(1))).protocols(kafkaAclConf) 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/KafkaProtocolMessage.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request 2 | 3 | import org.apache.kafka.clients.consumer.ConsumerRecord 4 | import org.apache.kafka.clients.producer.ProducerRecord 5 | import org.apache.kafka.common.header.Headers 6 | 7 | /** Topic may either be an 'input' or an 'output' topic. If both are defined here, the serdes may need to pick one without any 8 | * real prior knowledge (see KafkaProtocolBuilderNew.matchByMessage) 9 | * 10 | * @param key 11 | * the event Key 12 | * @param value 13 | * the event 'data' 14 | * @param producerTopic 15 | * The name of the Kafka topic to which the message will be sent. 16 | * @param consumerTopic 17 | * the topic this event is for 18 | * @param headers 19 | * any supplementary headers e.g. serde related headers 20 | * @param responseCode 21 | * a response code 22 | */ 23 | 24 | final case class KafkaProtocolMessage( 25 | key: Array[Byte], 26 | value: Array[Byte], 27 | producerTopic: String, 28 | consumerTopic: String, 29 | headers: Option[Headers] = None, 30 | responseCode: Option[String] = None, 31 | ) { 32 | def toProducerRecord: ProducerRecord[Array[Byte], Array[Byte]] = { 33 | headers.fold(new ProducerRecord(producerTopic, key, value))(hs => new ProducerRecord(producerTopic, null, key, value, hs)) 34 | } 35 | } 36 | 37 | object KafkaProtocolMessage { 38 | def from(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], inputTopic: Option[String]): KafkaProtocolMessage = 39 | KafkaProtocolMessage( 40 | consumerRecord.key(), 41 | consumerRecord.value(), 42 | inputTopic.getOrElse(""), 43 | consumerRecord.topic(), 44 | Option(consumerRecord.headers()), 45 | ) 46 | } 47 | -------------------------------------------------------------------------------- /src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples/ProducerSimulation.kt: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples 2 | 3 | import io.gatling.javaapi.core.CoreDsl.* 4 | import io.gatling.javaapi.core.Simulation 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl.* 7 | import java.time.Duration 8 | 9 | class ProducerSimulation : Simulation() { 10 | 11 | // example of using custom serde 12 | private val ser = KafkaAvroSerializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Serializer 13 | private val de = KafkaAvroDeserializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Deserializer 14 | 15 | private val kafkaConsumerConf = kafka().topic("test.topic") 16 | .properties(mapOf(ProducerConfig.ACKS_CONFIG to "1", 17 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092", 18 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG to "org.apache.kafka.common.serialization.StringSerializer", 19 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG to "org.apache.kafka.common.serialization.StringSerializer" 20 | )) 21 | 22 | private val scn = scenario("Basic") 23 | .exec(kafka("BasicRequest").send("foo")) 24 | .exec(kafka("dld").send("true", "12.0")) 25 | .exec(kafka("avro_serde").send("#{sessionIdKey}", avro( 26 | { session: Session -> session.get("event") }, 27 | ser, 28 | de 29 | ) 30 | ) { session: Session -> session.get("headers") } 31 | 32 | init { 33 | setUp( 34 | scn.injectOpen(atOnceUsers(1)) 35 | ) 36 | .protocols(kafkaConsumerConf) 37 | .maxDuration(Duration.ofSeconds(120)) 38 | } 39 | 40 | } -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KPConsumeSettingsStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | import scala.jdk.javaapi.DurationConverters; 4 | 5 | import java.time.Duration; 6 | import java.util.Map; 7 | 8 | import static scala.jdk.javaapi.CollectionConverters.asScala; 9 | 10 | public class KPConsumeSettingsStep { 11 | 12 | private final Map producerSettings; 13 | private final Map consumerSettings; 14 | 15 | public KPConsumeSettingsStep(Map producerSettings, Map consumerSettings) { 16 | this.producerSettings = producerSettings; 17 | this.consumerSettings = consumerSettings; 18 | } 19 | 20 | public KafkaProtocolBuilder timeout(Duration timeout) { 21 | scala.collection.immutable.Map ps = scala.collection.immutable.Map.from(asScala(this.producerSettings)); 22 | scala.collection.immutable.Map cs = scala.collection.immutable.Map.from(asScala(this.consumerSettings)); 23 | return new KafkaProtocolBuilder(org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder.apply(ps, cs, DurationConverters.toScala(timeout), org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher$.MODULE$)); 24 | } 25 | 26 | public KafkaProtocolBuilder withDefaultTimeout() { 27 | scala.collection.immutable.Map ps = scala.collection.immutable.Map.from(asScala(this.producerSettings)); 28 | scala.collection.immutable.Map cs = scala.collection.immutable.Map.from(asScala(this.consumerSettings)); 29 | return new KafkaProtocolBuilder(org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder.apply(ps, cs, DurationConverters.toScala(Duration.ofSeconds(60)), org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher$.MODULE$)); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples/BasicSimulation.kt: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples 2 | 3 | import io.gatling.javaapi.core.CoreDsl.* 4 | import io.gatling.javaapi.core.Simulation 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.apache.kafka.common.header.internals.RecordHeaders 7 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl.* 8 | import java.time.Duration 9 | import java.util.concurrent.atomic.AtomicInteger 10 | 11 | class BasicSimulation : Simulation() { 12 | 13 | private val kafkaConf = kafka() 14 | .topic("test.topic") 15 | .properties(mapOf(ProducerConfig.ACKS_CONFIG to "1")) 16 | 17 | private val kafkaProtocolC = kafka().requestReply() 18 | .producerSettings( 19 | mapOf( 20 | ProducerConfig.ACKS_CONFIG to "1", 21 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092", 22 | ) 23 | ) 24 | .consumeSettings( 25 | mapOf("bootstrap.servers" to "localhost:9092") 26 | ).timeout(Duration.ofSeconds(5)) 27 | 28 | private val c = AtomicInteger(0) 29 | 30 | private val feeder = generateSequence { 31 | mapOf("kekey" to c.incrementAndGet()) 32 | }.iterator() 33 | 34 | private val headers = RecordHeaders().add("test-header", "test_value".toByteArray()) 35 | 36 | private val scn = scenario("Basic") 37 | .feed(feeder) 38 | .exec( 39 | kafka("ReqRep").requestReply() 40 | .requestTopic("test.t") 41 | .replyTopic("test.t") 42 | .send( 43 | "#{kekey}", 44 | """{ "m": "dkf" }""", 45 | headers, 46 | String::class.java, 47 | String::class.java 48 | ) 49 | .check(jsonPath("$.m").`is`("dkf")) 50 | ) 51 | 52 | init { 53 | setUp(scn.injectOpen(atOnceUsers(5))).protocols(kafkaProtocolC).maxDuration(Duration.ofSeconds(120)) 54 | } 55 | } -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderBase.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.protocol; 2 | 3 | 4 | import scala.jdk.javaapi.DurationConverters; 5 | 6 | import java.time.Duration; 7 | import java.util.Collections; 8 | import java.util.Map; 9 | 10 | import static scala.jdk.javaapi.CollectionConverters.asScala; 11 | 12 | public class KafkaProtocolBuilderBase { 13 | 14 | /** 15 | * Start definition of the protocol for send only pattern 16 | * 17 | * @param name - topic for sending messages 18 | * @return KafkaProtocolBuilderPropertiesStep 19 | * @deprecated use topic definition in kafka request builders 20 | */ 21 | @Deprecated(since = "1.0.0", forRemoval = true) 22 | public KafkaProtocolBuilderPropertiesStep topic(String name) { 23 | return new KafkaProtocolBuilderPropertiesStep(name, Collections.emptyMap()); 24 | } 25 | 26 | /** 27 | * Start definition of the protocol for requestReply pattern 28 | * 29 | * @deprecated separate definition of the protocol for the requestReply scheme is no longer required; use producerSettings right away" 30 | */ 31 | @Deprecated(since = "1.0.0", forRemoval = true) 32 | public KafkaProtocolBuilderNewBase requestReply() { 33 | return new KafkaProtocolBuilderNewBase(); 34 | } 35 | 36 | public KPProducerSettingsStep producerSettings(Map ps) { 37 | return new KafkaProtocolBuilderNewBase().producerSettings(ps); 38 | } 39 | 40 | public KafkaProtocolBuilder properties(Map producerSettings) { 41 | scala.collection.immutable.Map ps = scala.collection.immutable.Map.from(asScala(producerSettings)); 42 | scala.collection.immutable.Map cs = scala.collection.immutable.Map.from(asScala(Map.of())); 43 | return new KafkaProtocolBuilder( 44 | org.galaxio.gatling.kafka.protocol.KafkaProtocolBuilder.apply(ps, cs, DurationConverters.toScala(Duration.ofSeconds(60)), org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher$.MODULE$) 45 | ); 46 | } 47 | 48 | } 49 | -------------------------------------------------------------------------------- /src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.kt: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples 2 | 3 | import io.gatling.javaapi.core.CoreDsl.* 4 | import io.gatling.javaapi.core.Simulation 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.galaxio.gatling.kafka.javaapi.* 7 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl.* 8 | import org.galaxio.gatling.kafka.request.* 9 | import java.time.Duration 10 | 11 | class AvroClassWithRequestReplySimulation : Simulation() { 12 | 13 | // example of using custom serde 14 | val ser = KafkaAvroSerializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Serializer 15 | val de = KafkaAvroDeserializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Deserializer 16 | 17 | // protocol 18 | val kafkaProtocolRRAvro = kafka() 19 | .requestReply() 20 | .producerSettings( 21 | mapOf( 22 | ProducerConfig.ACKS_CONFIG to "1", 23 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092", 24 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG to "org.apache.kafka.common.serialization.StringSerializer", 25 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG to "io.confluent.kafka.serializers.KafkaAvroSerializer", 26 | // schema registry url is required for KafkaAvroSerializer and KafkaAvroDeserializer 27 | "schema.registry.url" to "url" 28 | ) 29 | ) 30 | .consumeSettings( 31 | mapOf( 32 | "bootstrap.servers" to "localhost:9092" 33 | ) 34 | ) 35 | .timeout(Duration.ofSeconds(5)) 36 | 37 | // message 38 | val kafkaMessage = kafka("RequestReply").requestReply() 39 | .requestTopic("request.t") 40 | .replyTopic("reply.t") 41 | .send("key", MyAvroClass(), String::class.java, MyAvroClass::class.java, ser, de) 42 | 43 | // simulation 44 | init { 45 | setUp(scenario("Kafka scenario").exec(kafkaMessage).injectOpen(atOnceUsers(1))).protocols(kafkaProtocolRRAvro) 46 | } 47 | 48 | } -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/MatchSimulation.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.core.Predef._ 4 | import io.gatling.core.feeder.Feeder 5 | import io.gatling.core.structure.ScenarioBuilder 6 | import org.apache.kafka.clients.producer.ProducerConfig 7 | import org.galaxio.gatling.kafka.Predef._ 8 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 9 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 10 | 11 | import java.util.concurrent.atomic.AtomicInteger 12 | import scala.concurrent.duration.DurationInt 13 | 14 | class MatchSimulation extends Simulation { 15 | 16 | val kafkaProtocolMatchByValue: KafkaProtocol = kafka 17 | .producerSettings( 18 | Map( 19 | ProducerConfig.ACKS_CONFIG -> "1", 20 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 21 | ), 22 | ) 23 | .consumeSettings( 24 | Map( 25 | "bootstrap.servers" -> "localhost:9092", 26 | ), 27 | ) 28 | .timeout(5.seconds) 29 | // for match by message value 30 | .matchByValue 31 | 32 | def matchByOwnVal(message: KafkaProtocolMessage): Array[Byte] = { 33 | // do something with the message and extract the values your are interested in 34 | // method is called: 35 | // - for each message which will be sent out 36 | // - for each message which has been received 37 | "Custom Message".getBytes // just returning something 38 | } 39 | 40 | val kafkaProtocolMatchByMessage: KafkaProtocol = kafka 41 | .producerSettings( 42 | Map( 43 | ProducerConfig.ACKS_CONFIG -> "1", 44 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 45 | ), 46 | ) 47 | .consumeSettings( 48 | Map( 49 | "bootstrap.servers" -> "localhost:9092", 50 | ), 51 | ) 52 | .timeout(5.seconds) 53 | .matchByMessage(matchByOwnVal) 54 | 55 | val c = new AtomicInteger(0) 56 | val feeder: Feeder[Int] = Iterator.continually(Map("kekey" -> c.incrementAndGet())) 57 | 58 | val scn: ScenarioBuilder = scenario("Basic") 59 | .feed(feeder) 60 | .exec( 61 | kafka("ReqRep").requestReply 62 | .requestTopic("test.t") 63 | .replyTopic("test.t") 64 | .send[String, String]("#{kekey}", """{ "m": "dkf" }"""), 65 | ) 66 | 67 | setUp(scn.inject(atOnceUsers(1))).protocols(kafkaProtocolMatchByMessage).maxDuration(120.seconds) 68 | } 69 | -------------------------------------------------------------------------------- /src/test/java/org/galaxio/gatling/kafka/javaapi/examples/BasicSimulation.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples; 2 | 3 | import io.gatling.javaapi.core.ScenarioBuilder; 4 | import io.gatling.javaapi.core.Simulation; 5 | import org.apache.kafka.clients.producer.ProducerConfig; 6 | import org.apache.kafka.common.header.Headers; 7 | import org.apache.kafka.common.header.internals.RecordHeaders; 8 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl; 9 | import org.galaxio.gatling.kafka.javaapi.protocol.KafkaProtocolBuilder; 10 | 11 | import java.time.Duration; 12 | import java.util.Collections; 13 | import java.util.Iterator; 14 | import java.util.Map; 15 | import java.util.concurrent.atomic.AtomicInteger; 16 | import java.util.function.Supplier; 17 | import java.util.stream.Stream; 18 | 19 | import static io.gatling.javaapi.core.CoreDsl.*; 20 | 21 | public class BasicSimulation extends Simulation { 22 | 23 | private final KafkaProtocolBuilder kafkaConf = KafkaDsl.kafka() 24 | .properties(Map.of(ProducerConfig.ACKS_CONFIG, "1")); 25 | 26 | private final KafkaProtocolBuilder kafkaProtocolC = KafkaDsl.kafka() 27 | .producerSettings( 28 | Map.of( 29 | ProducerConfig.ACKS_CONFIG, "1", 30 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 31 | ) 32 | ) 33 | .consumeSettings( 34 | Map.of("bootstrap.servers", "localhost:9092") 35 | ).timeout(Duration.ofSeconds(5)); 36 | 37 | private final AtomicInteger c = new AtomicInteger(0); 38 | 39 | private final Iterator> feeder = 40 | Stream.generate((Supplier>) () -> Collections.singletonMap("kekey", c.incrementAndGet()) 41 | ).iterator(); 42 | 43 | private final Headers headers = new RecordHeaders().add("test-header", "test_value".getBytes()); 44 | 45 | private final ScenarioBuilder scn = scenario("Basic") 46 | .feed(feeder) 47 | .exec( 48 | KafkaDsl.kafka("ReqRep").requestReply() 49 | .requestTopic("test.t") 50 | .replyTopic("test.t") 51 | .send("#{kekey}", """ 52 | { "m": "dkf" } 53 | """, headers, String.class, String.class) 54 | .check(jsonPath("$.m").is("dkf")) 55 | ); 56 | 57 | { 58 | setUp(scn.injectOpen(atOnceUsers(5))).protocols(kafkaProtocolC).maxDuration(Duration.ofSeconds(120)); 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/checks/KafkaMessagePreparer.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.checks 2 | 3 | import com.fasterxml.jackson.databind.JsonNode 4 | import io.gatling.commons.validation._ 5 | import io.gatling.core.check.Preparer 6 | import io.gatling.core.check.xpath.XmlParsers 7 | import io.gatling.core.config.GatlingConfiguration 8 | import io.gatling.core.json.JsonParsers 9 | import net.sf.saxon.s9api.XdmNode 10 | import org.apache.avro.generic.GenericRecord 11 | import org.apache.kafka.common.serialization.Serde 12 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 13 | 14 | import java.io.ByteArrayInputStream 15 | import java.nio.charset.Charset 16 | import scala.util.Try 17 | 18 | trait KafkaMessagePreparer[P] extends Preparer[KafkaProtocolMessage, P] 19 | 20 | object KafkaMessagePreparer { 21 | 22 | private def messageCharset(cfg: GatlingConfiguration, msg: KafkaProtocolMessage): Validation[Charset] = 23 | Try(Charset.forName(msg.headers.map(_.lastHeader("content_encoding").value()).map(new String(_)).get)) 24 | .orElse(Try(cfg.core.charset)) 25 | .toValidation 26 | 27 | def stringBodyPreparer(configuration: GatlingConfiguration): KafkaMessagePreparer[String] = 28 | msg => 29 | messageCharset(configuration, msg) 30 | .map(cs => if (msg.value.length > 0) new String(msg.value, cs) else "") 31 | 32 | val bytesBodyPreparer: KafkaMessagePreparer[Array[Byte]] = msg => 33 | (if (msg.value.length > 0) msg.value else Array.emptyByteArray).success 34 | 35 | private val CharsParsingThreshold = 200 * 1000 36 | 37 | def jsonPathPreparer( 38 | jsonParsers: JsonParsers, 39 | configuration: GatlingConfiguration, 40 | ): Preparer[KafkaProtocolMessage, JsonNode] = 41 | msg => 42 | messageCharset(configuration, msg) 43 | .flatMap(bodyCharset => 44 | if (msg.value.length > CharsParsingThreshold) 45 | jsonParsers.safeParse(new ByteArrayInputStream(msg.value)) 46 | else 47 | jsonParsers.safeParse(new String(msg.value, bodyCharset)), 48 | ) 49 | 50 | private val ErrorMapper = "Could not parse response into a DOM Document: " + _ 51 | 52 | def xmlPreparer(configuration: GatlingConfiguration): KafkaMessagePreparer[XdmNode] = 53 | msg => 54 | safely(ErrorMapper) { 55 | messageCharset(configuration, msg).map(cs => XmlParsers.parse(new ByteArrayInputStream(msg.value), cs)) 56 | } 57 | 58 | def avroPreparer[T <: GenericRecord: Serde](config: GatlingConfiguration, topic: String): KafkaMessagePreparer[T] = msg => 59 | safely(ErrorMapper) { 60 | messageCharset(config, msg).map(_ => implicitly[Serde[T]].deserializer().deserialize(topic, msg.value)) 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/KafkaJavaapiMethodsGatlingTest.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.core.Predef._ 4 | import io.gatling.core.feeder.Feeder 5 | import io.gatling.core.protocol.Protocol 6 | import io.gatling.core.structure.ScenarioBuilder 7 | import org.apache.kafka.clients.producer.ProducerConfig 8 | import org.apache.kafka.common.header.Headers 9 | import org.apache.kafka.common.header.internals.RecordHeaders 10 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl._ 11 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression 12 | 13 | import java.util.concurrent.atomic.AtomicInteger 14 | 15 | class KafkaJavaapiMethodsGatlingTest extends Simulation { 16 | 17 | val c = new AtomicInteger(0) 18 | val feeder: Feeder[Int] = Iterator.continually(Map("key" -> c.incrementAndGet())) 19 | val hFeeder: Feeder[Array[Byte]] = Iterator.continually(Map("headerId" -> java.util.UUID.randomUUID().toString.getBytes)) 20 | 21 | val headers: JExpression[Headers] = s => { 22 | val bytes = java.util.Optional.ofNullable(s.get[Array[Byte]]("headerId")).orElse("".getBytes) 23 | new RecordHeaders().add("test-header", bytes) 24 | } 25 | 26 | val kafkaConfwoKey: Protocol = kafka 27 | .properties( 28 | java.util.Map.of( 29 | ProducerConfig.ACKS_CONFIG, 30 | "1", 31 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 32 | "localhost:9093", 33 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 34 | "org.apache.kafka.common.serialization.StringSerializer", 35 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 36 | "org.apache.kafka.common.serialization.StringSerializer", 37 | ), 38 | ) 39 | .protocol() 40 | 41 | setUp( 42 | scenario("Request String without key") 43 | .feed(hFeeder) 44 | .feed(feeder) 45 | .exec( 46 | kafka("Request String without headers and key") 47 | .topic("myTopic3") 48 | .send("testJavaWithoutKeyAndHeaders") 49 | .asScala(), 50 | ) 51 | .exec( 52 | kafka("Request Long without headers and key") 53 | .topic("myTopic3") 54 | .send(12L) 55 | .asScala(), 56 | ) 57 | .exec( 58 | kafka("Request Int Long without headers") 59 | .topic("myTopic3") 60 | .send(0, 12L) 61 | .asScala(), 62 | ) 63 | .exec( 64 | kafka("Request String with headers without key") 65 | .topic("myTopic3") 66 | .send("testJavaWithHeadersWithoutKey", new RecordHeaders().add("test-header", "test_value".getBytes())) 67 | .asScala(), 68 | ) 69 | .exec(kafka("MsgBuilders").topic("myTopic3").send("key#{key}", "val", headers).asScala()) 70 | .inject(nothingFor(1), atOnceUsers(1)) 71 | .protocols(kafkaConfwoKey), 72 | ) 73 | 74 | } 75 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/AvroClassWithRequestReplySimulation.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient 4 | import io.confluent.kafka.serializers.{KafkaAvroDeserializer, KafkaAvroSerializer} 5 | import io.gatling.core.Predef._ 6 | import org.apache.kafka.clients.producer.ProducerConfig 7 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer} 8 | import org.galaxio.gatling.kafka.Predef._ 9 | import org.galaxio.gatling.kafka.actions.KafkaRequestReplyActionBuilder 10 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 11 | 12 | import scala.concurrent.duration.DurationInt 13 | import scala.jdk.CollectionConverters._ 14 | 15 | class AvroClassWithRequestReplySimulation extends Simulation { 16 | 17 | // default serde for unknown classes is avro serde 18 | // schemaRegUrl must be specified if custom avro scheme is used, when the send method requires implicit 19 | implicit val schemaRegUrl: String = "http://localhost:9094" 20 | 21 | // example if you want to use your own or custom serde 22 | val ser = 23 | new KafkaAvroSerializer( 24 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 25 | ) 26 | 27 | val de = 28 | new KafkaAvroDeserializer( 29 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 30 | ) 31 | 32 | implicit val serdeClass: Serde[MyAvroClass] = new Serde[MyAvroClass] { 33 | override def serializer(): Serializer[MyAvroClass] = ser.asInstanceOf[Serializer[MyAvroClass]] 34 | 35 | override def deserializer(): Deserializer[MyAvroClass] = de.asInstanceOf[Deserializer[MyAvroClass]] 36 | } 37 | 38 | // protocol 39 | val kafkaProtocolRRAvro: KafkaProtocol = kafka 40 | .producerSettings( 41 | Map( 42 | ProducerConfig.ACKS_CONFIG -> "1", 43 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 44 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 45 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 46 | // schema registry url is required for KafkaAvroSerializer and KafkaAvroDeserializer 47 | "schema.registry.url" -> "http://localhost:9094", 48 | ), 49 | ) 50 | .consumeSettings( 51 | Map( 52 | "bootstrap.servers" -> "localhost:9093", 53 | ), 54 | ) 55 | .timeout(5.seconds) 56 | 57 | // message 58 | val kafkaMessage: KafkaRequestReplyActionBuilder[String, MyAvroClass] = kafka("RequestReply").requestReply 59 | .requestTopic("request.t") 60 | .replyTopic("reply.t") 61 | .send[String, MyAvroClass]("key", MyAvroClass()) 62 | 63 | // simulation 64 | setUp(scenario("Kafka RequestReply Avro").exec(kafkaMessage).inject(atOnceUsers(1))).protocols(kafkaProtocolRRAvro) 65 | 66 | case class MyAvroClass() 67 | } 68 | -------------------------------------------------------------------------------- /src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples/MatchSimulation.kt: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples 2 | 3 | import io.gatling.javaapi.core.CoreDsl.* 4 | import io.gatling.javaapi.core.Simulation 5 | import org.apache.kafka.clients.producer.ProducerConfig 6 | import org.galaxio.gatling.kafka.javaapi.* 7 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl.* 8 | import org.galaxio.gatling.kafka.request.* 9 | import java.time.Duration 10 | import java.util.concurrent.atomic.AtomicInteger 11 | 12 | class MatchSimulation : Simulation() { 13 | 14 | private val kafkaProtocolMatchByValue = kafka().requestReply() 15 | .producerSettings( 16 | mapOf( 17 | ProducerConfig.ACKS_CONFIG to "1", 18 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092" 19 | ) 20 | ) 21 | .consumeSettings( 22 | mapOf("bootstrap.servers" to "localhost:9092") 23 | ) 24 | .timeout(Duration.ofSeconds(5)) // for match by message value 25 | .matchByValue() 26 | 27 | private fun matchByOwnVal(message: KafkaProtocolMessage): ByteArray { 28 | // do something with the message and extract the values you are interested in 29 | // method is called: 30 | // - for each message which will be sent out 31 | // - for each message which has been received 32 | return "Custom Message".toByteArray() // just returning something 33 | } 34 | 35 | private val kafkaProtocolMatchByMessage = kafka().requestReply() 36 | .producerSettings( 37 | mapOf( 38 | ProducerConfig.ACKS_CONFIG to "1", 39 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092" 40 | ) 41 | ) 42 | .consumeSettings( 43 | mapOf( 44 | "bootstrap.servers" to "localhost:9092" 45 | ) 46 | ) 47 | .timeout(Duration.ofSeconds(5)) 48 | .matchByMessage { message: KafkaProtocolMessage -> matchByOwnVal(message) } 49 | 50 | private val c = AtomicInteger(0) 51 | 52 | private val feeder = generateSequence { 53 | mapOf("kekey" to c.incrementAndGet()) 54 | }.iterator() 55 | 56 | private val scn = scenario("Basic") 57 | .feed(feeder) 58 | .exec( 59 | KafkaDsl.kafka("ReqRep").requestReply() 60 | .requestTopic("test.t") 61 | .replyTopic("test.t") 62 | .send( 63 | "#{kekey}", 64 | """{ "m": "dkf" }""", 65 | String::class.java, 66 | String::class.java 67 | ) 68 | ) 69 | 70 | init 71 | { 72 | setUp( 73 | scn.injectOpen(atOnceUsers(1)) 74 | ) 75 | .protocols(kafkaProtocolMatchByMessage) 76 | .maxDuration(Duration.ofSeconds(120)) 77 | } 78 | 79 | } -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/checks/KafkaCheckMaterializer.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.checks 2 | 3 | import com.fasterxml.jackson.databind.JsonNode 4 | import io.gatling.commons.validation.SuccessWrapper 5 | import io.gatling.core.check.bytes.BodyBytesCheckType 6 | import io.gatling.core.check.jmespath.JmesPathCheckType 7 | import io.gatling.core.check.jsonpath.JsonPathCheckType 8 | import io.gatling.core.check.string.BodyStringCheckType 9 | import io.gatling.core.check.substring.SubstringCheckType 10 | import io.gatling.core.check.xpath.XPathCheckType 11 | import io.gatling.core.check.{CheckMaterializer, Preparer} 12 | import io.gatling.core.config.GatlingConfiguration 13 | import io.gatling.core.json.JsonParsers 14 | import net.sf.saxon.s9api.XdmNode 15 | import org.apache.avro.generic.GenericRecord 16 | import org.apache.kafka.common.serialization.Serde 17 | import org.galaxio.gatling.kafka.KafkaCheck 18 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 19 | 20 | final class KafkaCheckMaterializer[T, P] private[KafkaCheckMaterializer] ( 21 | override val preparer: Preparer[KafkaProtocolMessage, P], 22 | ) extends CheckMaterializer[T, KafkaCheck, KafkaProtocolMessage, P](identity) 23 | 24 | object KafkaCheckMaterializer { 25 | def xpath(configuration: GatlingConfiguration): KafkaCheckMaterializer[XPathCheckType, XdmNode] = 26 | new KafkaCheckMaterializer(KafkaMessagePreparer.xmlPreparer(configuration)) 27 | 28 | val bodyBytes: KafkaCheckMaterializer[BodyBytesCheckType, Array[Byte]] = 29 | new KafkaCheckMaterializer(KafkaMessagePreparer.bytesBodyPreparer) 30 | 31 | def bodyString(configuration: GatlingConfiguration): KafkaCheckMaterializer[BodyStringCheckType, String] = 32 | new KafkaCheckMaterializer(KafkaMessagePreparer.stringBodyPreparer(configuration)) 33 | 34 | def substring(configuration: GatlingConfiguration): KafkaCheckMaterializer[SubstringCheckType, String] = 35 | new KafkaCheckMaterializer(KafkaMessagePreparer.stringBodyPreparer(configuration)) 36 | 37 | def jsonPath( 38 | jsonParsers: JsonParsers, 39 | configuration: GatlingConfiguration, 40 | ): KafkaCheckMaterializer[JsonPathCheckType, JsonNode] = 41 | new KafkaCheckMaterializer(KafkaMessagePreparer.jsonPathPreparer(jsonParsers, configuration)) 42 | 43 | def jmesPath( 44 | jsonParsers: JsonParsers, 45 | configuration: GatlingConfiguration, 46 | ): KafkaCheckMaterializer[JmesPathCheckType, JsonNode] = 47 | new KafkaCheckMaterializer(KafkaMessagePreparer.jsonPathPreparer(jsonParsers, configuration)) 48 | 49 | type KafkaMessageCheckType 50 | 51 | val kafkaStatusCheck: KafkaCheckMaterializer[KafkaMessageCheckType, KafkaProtocolMessage] = 52 | new KafkaCheckMaterializer(_.success) 53 | 54 | def avroBody[T <: GenericRecord: Serde]( 55 | configuration: GatlingConfiguration, 56 | topic: String, 57 | ): KafkaCheckMaterializer[KafkaMessageCheckType, T] = 58 | new KafkaCheckMaterializer(KafkaMessagePreparer.avroPreparer[T](configuration, topic)) 59 | } 60 | -------------------------------------------------------------------------------- /src/test/java/org/galaxio/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples; 2 | 3 | import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; 4 | import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; 5 | import io.confluent.kafka.serializers.KafkaAvroDeserializer; 6 | import io.confluent.kafka.serializers.KafkaAvroSerializer; 7 | import io.gatling.javaapi.core.Simulation; 8 | import org.apache.kafka.clients.producer.ProducerConfig; 9 | import org.apache.kafka.common.serialization.Deserializer; 10 | import org.apache.kafka.common.serialization.Serializer; 11 | import org.galaxio.gatling.kafka.javaapi.protocol.KafkaProtocolBuilder; 12 | import org.galaxio.gatling.kafka.javaapi.request.builder.RequestReplyBuilder; 13 | 14 | import java.time.Duration; 15 | import java.util.Arrays; 16 | import java.util.Map; 17 | 18 | import static io.gatling.javaapi.core.CoreDsl.atOnceUsers; 19 | import static io.gatling.javaapi.core.CoreDsl.scenario; 20 | import static org.galaxio.gatling.kafka.javaapi.KafkaDsl.kafka; 21 | 22 | public class AvroClassWithRequestReplySimulation extends Simulation { 23 | private static final SchemaRegistryClient client = new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16); 24 | 25 | // example of using custom serde 26 | public static Serializer ser = 27 | (Serializer) new KafkaAvroSerializer(client); 28 | public static Deserializer de = 29 | (Deserializer) new KafkaAvroDeserializer(client); 30 | 31 | // protocol 32 | private final KafkaProtocolBuilder kafkaProtocolRRAvro = kafka() 33 | .producerSettings( 34 | Map.of( 35 | ProducerConfig.ACKS_CONFIG, "1", 36 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9093", 37 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer", 38 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer", 39 | // schema registry url is required for KafkaAvroSerializer and KafkaAvroDeserializer 40 | "schema.registry.url", "http://localhost:9094" 41 | ) 42 | ) 43 | .consumeSettings( 44 | Map.of("bootstrap.servers", "localhost:9093") 45 | ) 46 | .timeout(Duration.ofSeconds(5)); 47 | 48 | // message 49 | public static RequestReplyBuilder kafkaMessage = kafka("RequestReply").requestReply() 50 | .requestTopic("request.t") 51 | .replyTopic("reply.t") 52 | .send("key", new MyAvroClass(), String.class, MyAvroClass.class, ser, de); 53 | 54 | // simulation 55 | { 56 | setUp(scenario("Kafka RequestReply Avro").exec(kafkaMessage).injectOpen(atOnceUsers(1))).protocols(kafkaProtocolRRAvro); 57 | } 58 | 59 | private static class MyAvroClass { 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/actions/KafkaRequestReplyAction.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.actions 2 | 3 | import io.gatling.commons.stats.KO 4 | import io.gatling.commons.util.Clock 5 | import io.gatling.core.CoreComponents 6 | import io.gatling.core.action.Action 7 | import io.gatling.core.actor.ActorRef 8 | import io.gatling.core.controller.throttle.Throttler 9 | import io.gatling.core.session.Session 10 | import io.gatling.core.stats.StatsEngine 11 | import org.galaxio.gatling.kafka.client.KafkaMessageTracker 12 | import org.galaxio.gatling.kafka.protocol.KafkaComponents 13 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 14 | import org.galaxio.gatling.kafka.request.builder.KafkaAttributes 15 | 16 | import scala.reflect.ClassTag 17 | 18 | class KafkaRequestReplyAction[K: ClassTag, V: ClassTag]( 19 | components: KafkaComponents, 20 | attributes: KafkaAttributes[K, V], 21 | coreComponents: CoreComponents, 22 | val next: Action, 23 | throttler: Option[ActorRef[Throttler.Command]], 24 | ) extends KafkaAction[K, V](components, attributes, throttler) { 25 | 26 | override def name: String = genName("kafkaRequestReply") 27 | val statsEngine: StatsEngine = coreComponents.statsEngine 28 | val clock: Clock = coreComponents.clock 29 | 30 | override def sendKafkaMessage(requestNameString: String, protocolMessage: KafkaProtocolMessage, session: Session): Unit = { 31 | val requestStartDate = clock.nowMillis 32 | components.sender.send(protocolMessage)( 33 | rm => { 34 | if (logger.underlying.isDebugEnabled) { 35 | logMessage( 36 | s"Record sent user=${session.userId} key=${new String(protocolMessage.key)} topic=${rm.topic()}", 37 | protocolMessage, 38 | ) 39 | } 40 | val id = components.kafkaProtocol.messageMatcher.requestMatch(protocolMessage) 41 | 42 | components.trackersPool.map { trackers => 43 | val tracker = trackers.tracker( 44 | protocolMessage.producerTopic, 45 | protocolMessage.consumerTopic, 46 | components.kafkaProtocol.messageMatcher, 47 | None, 48 | components.kafkaProtocol.timeout, 49 | ) 50 | tracker ! KafkaMessageTracker 51 | .MessagePublished( 52 | id, 53 | clock.nowMillis, 54 | components.kafkaProtocol.timeout.toMillis, 55 | attributes.checks, 56 | session, 57 | next, 58 | requestNameString, 59 | ) 60 | } 61 | }, 62 | e => { 63 | val requestEndDate = clock.nowMillis 64 | logger.error(e.getMessage, e) 65 | statsEngine.logResponse( 66 | session.scenario, 67 | session.groups, 68 | requestNameString, 69 | requestStartDate, 70 | requestEndDate, 71 | KO, 72 | Some("500"), 73 | Some(e.getMessage), 74 | ) 75 | next ! session.logGroupRequestTimings(requestStartDate, requestEndDate).markAsFailed 76 | }, 77 | ) 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/KafkaSerdesImplicits.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request 2 | 3 | import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient 4 | import io.confluent.kafka.serializers.{KafkaAvroDeserializer, KafkaAvroSerializer} 5 | import io.confluent.kafka.streams.serdes.avro.GenericAvroSerde 6 | import org.apache.avro.generic.GenericRecord 7 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer, Serdes => JSerdes} 8 | import org.apache.kafka.streams.kstream.WindowedSerdes 9 | import org.apache.kafka.streams.scala.kstream.Consumed 10 | 11 | import java.nio.ByteBuffer 12 | import java.util.UUID 13 | import scala.jdk.CollectionConverters._ 14 | 15 | trait KafkaSerdesImplicits { 16 | implicit def stringSerde: Serde[String] = JSerdes.String() 17 | implicit def longSerde: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]] 18 | implicit def javaLongSerde: Serde[java.lang.Long] = JSerdes.Long() 19 | implicit def byteArraySerde: Serde[Array[Byte]] = JSerdes.ByteArray() 20 | implicit def bytesSerde: Serde[org.apache.kafka.common.utils.Bytes] = JSerdes.Bytes() 21 | implicit def byteBufferSerde: Serde[ByteBuffer] = JSerdes.ByteBuffer() 22 | implicit def shortSerde: Serde[Short] = JSerdes.Short().asInstanceOf[Serde[Short]] 23 | implicit def javaShortSerde: Serde[java.lang.Short] = JSerdes.Short() 24 | implicit def floatSerde: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]] 25 | implicit def javaFloatSerde: Serde[java.lang.Float] = JSerdes.Float() 26 | implicit def doubleSerde: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]] 27 | implicit def javaDoubleSerde: Serde[java.lang.Double] = JSerdes.Double() 28 | implicit def intSerde: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]] 29 | implicit def javaIntegerSerde: Serde[java.lang.Integer] = JSerdes.Integer() 30 | implicit def uuidSerde: Serde[UUID] = JSerdes.UUID() 31 | 32 | implicit def sessionWindowedSerde[T](implicit tSerde: Serde[T]): WindowedSerdes.SessionWindowedSerde[T] = 33 | new WindowedSerdes.SessionWindowedSerde[T](tSerde) 34 | 35 | implicit def serdeClass[T](implicit schemaRegUrl: String): Serde[T] = new Serde[T] { 36 | override def serializer(): Serializer[T] = new KafkaAvroSerializer( 37 | new CachedSchemaRegistryClient(schemaRegUrl.split(',').toList.asJava, 16), 38 | ).asInstanceOf[Serializer[T]] 39 | 40 | override def deserializer(): Deserializer[T] = new KafkaAvroDeserializer( 41 | new CachedSchemaRegistryClient(schemaRegUrl.split(',').toList.asJava, 16), 42 | ).asInstanceOf[Deserializer[T]] 43 | } 44 | 45 | implicit val avroSerde: Serde[GenericRecord] = new GenericAvroSerde() 46 | 47 | implicit def consumedFromSerde[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): Consumed[K, V] = 48 | Consumed.`with`[K, V] 49 | 50 | } 51 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/BasicSimulation.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import io.gatling.core.Predef._ 4 | import io.gatling.core.feeder.Feeder 5 | import io.gatling.core.session.Expression 6 | import io.gatling.core.structure.ScenarioBuilder 7 | import org.apache.kafka.clients.consumer.ConsumerConfig 8 | import org.apache.kafka.clients.producer.ProducerConfig 9 | import org.apache.kafka.common.header.Headers 10 | import org.apache.kafka.common.header.internals.RecordHeaders 11 | import org.apache.kafka.streams.{KafkaStreams, KeyValue, StreamsBuilder, StreamsConfig} 12 | import org.apache.kafka.streams.kstream.KeyValueMapper 13 | import org.galaxio.gatling.kafka.Predef._ 14 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 15 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 16 | 17 | import java.util.Properties 18 | import java.util.concurrent.atomic.AtomicInteger 19 | import scala.concurrent.duration.DurationInt 20 | 21 | class BasicSimulation extends Simulation { 22 | 23 | // val kafkaConf: KafkaProtocol = kafka 24 | // .topic("test.topic") 25 | // .properties(Map(ProducerConfig.ACKS_CONFIG -> "1")) 26 | 27 | def getHeader(headerKey: String): KafkaProtocolMessage => Array[Byte] = 28 | _.headers 29 | .flatMap(hs => Option(hs.lastHeader(headerKey)).map(_.value())) 30 | .getOrElse(Array.emptyByteArray) 31 | 32 | def kafkaProtocolC: KafkaProtocol = kafka 33 | .producerSettings( 34 | ProducerConfig.ACKS_CONFIG -> "1", 35 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 36 | ) 37 | .consumeSettings( 38 | ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 39 | ) 40 | .timeout(10.seconds) 41 | // .matchByMessage(getHeader("test-header")) 42 | 43 | val c = new AtomicInteger(1) 44 | val feeder: Feeder[java.util.UUID] = Iterator.continually(Map("kekey" -> java.util.UUID.randomUUID())) 45 | val hFeeder: Feeder[Array[Byte]] = Iterator.continually(Map("headerId" -> java.util.UUID.randomUUID().toString.getBytes)) 46 | 47 | val headers: Expression[Headers] = 48 | _("headerId").validate[Array[Byte]].map(bytes => new RecordHeaders().add("test-header", bytes)) 49 | 50 | def scn(id: String): ScenarioBuilder = scenario(s"Basic$id") 51 | .feed(feeder) 52 | .feed(hFeeder) 53 | .exec( 54 | kafka("ReqRep").requestReply 55 | .requestTopic("test.t") 56 | .replyTopic("test.t") 57 | .send[String, String]("#{kekey}", """{ "m": "dkf" }""", headers) 58 | .check(jsonPath("$.m").is("dkf")), 59 | ) 60 | .exec( 61 | kafka("ReqRep2").requestReply 62 | .requestTopic("myTopic2") 63 | .replyTopic("test.t1") 64 | .send[String, String]("#{kekey}", """{ "m": "dkf" }""", headers) 65 | .check(jsonPath("$.M").is("DKF")), 66 | ) 67 | 68 | setUp( 69 | scn("A").inject(atOnceUsers(50)), 70 | scn("B").inject(atOnceUsers(1)), 71 | scn("C").inject(atOnceUsers(1)), 72 | scn("D").inject(atOnceUsers(1)), 73 | scn("E").inject(atOnceUsers(1)), 74 | ).protocols(kafkaProtocolC).maxDuration(120.seconds) 75 | 76 | } 77 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/actions/KafkaRequestAction.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.actions 2 | 3 | import io.gatling.commons.stats.{KO, OK} 4 | import io.gatling.commons.util.Clock 5 | import io.gatling.commons.validation._ 6 | import io.gatling.core.CoreComponents 7 | import io.gatling.core.action.Action 8 | import io.gatling.core.actor.ActorRef 9 | import io.gatling.core.controller.throttle.Throttler 10 | import io.gatling.core.session._ 11 | import io.gatling.core.stats.StatsEngine 12 | import org.galaxio.gatling.kafka.protocol.KafkaComponents 13 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 14 | import org.galaxio.gatling.kafka.request.builder.KafkaAttributes 15 | 16 | import scala.reflect.ClassTag 17 | 18 | final class KafkaRequestAction[K: ClassTag, V: ClassTag]( 19 | val components: KafkaComponents, 20 | val attributes: KafkaAttributes[K, V], 21 | val coreComponents: CoreComponents, 22 | val next: Action, 23 | val throttler: Option[ActorRef[Throttler.Command]], 24 | ) extends KafkaAction[K, V](components, attributes, throttler) { 25 | 26 | override def name: String = genName("kafkaRequest") 27 | val statsEngine: StatsEngine = coreComponents.statsEngine 28 | val clock: Clock = coreComponents.clock 29 | 30 | private def reportUnbuildableRequest(session: Session, error: String): Unit = { 31 | val loggedName = attributes.requestName(session) match { 32 | case Success(requestNameValue) => 33 | statsEngine.logRequestCrash(session.scenario, session.groups, requestNameValue, s"Failed to build request: $error") 34 | requestNameValue 35 | case _ => name 36 | } 37 | logger.error(s"'$loggedName' failed to execute: $error") 38 | } 39 | 40 | override def sendKafkaMessage(requestNameString: String, protocolMessage: KafkaProtocolMessage, session: Session): Unit = { 41 | val requestStartDate = clock.nowMillis 42 | components.sender.send(protocolMessage)( 43 | metadata => { 44 | val requestEndDate = clock.nowMillis 45 | if (logger.underlying.isDebugEnabled) { 46 | logger.debug(s"Record sent user=${session.userId} key=${new String(protocolMessage.key)} topic=${metadata.topic()}") 47 | logger.trace(s"ProducerRecord=${protocolMessage.toProducerRecord}") 48 | } 49 | 50 | statsEngine.logResponse( 51 | session.scenario, 52 | session.groups, 53 | requestNameString, 54 | startTimestamp = requestStartDate, 55 | endTimestamp = requestEndDate, 56 | OK, 57 | None, 58 | None, 59 | ) 60 | next ! session.logGroupRequestTimings(requestStartDate, requestEndDate) 61 | }, 62 | exception => { 63 | val requestEndDate = clock.nowMillis 64 | 65 | logger.error(exception.getMessage, exception) 66 | reportUnbuildableRequest(session, exception.getMessage) 67 | 68 | statsEngine.logResponse( 69 | session.scenario, 70 | session.groups, 71 | requestNameString, 72 | startTimestamp = requestStartDate, 73 | endTimestamp = requestEndDate, 74 | KO, 75 | None, 76 | Some(exception.getMessage), 77 | ) 78 | next ! session.logGroupRequestTimings(requestStartDate, requestEndDate).markAsFailed 79 | }, 80 | ) 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/test/java/org/galaxio/gatling/kafka/javaapi/examples/MatchSimulation.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.examples; 2 | 3 | import io.gatling.javaapi.core.ScenarioBuilder; 4 | import io.gatling.javaapi.core.Simulation; 5 | import org.apache.kafka.clients.producer.ProducerConfig; 6 | import org.galaxio.gatling.kafka.javaapi.KafkaDsl; 7 | import org.galaxio.gatling.kafka.javaapi.protocol.KafkaProtocolBuilder; 8 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage; 9 | 10 | import java.time.Duration; 11 | import java.util.Collections; 12 | import java.util.Iterator; 13 | import java.util.Map; 14 | import java.util.concurrent.atomic.AtomicInteger; 15 | import java.util.function.Supplier; 16 | import java.util.stream.Stream; 17 | 18 | import static io.gatling.javaapi.core.CoreDsl.atOnceUsers; 19 | import static io.gatling.javaapi.core.CoreDsl.scenario; 20 | 21 | public class MatchSimulation extends Simulation { 22 | 23 | private final KafkaProtocolBuilder kafkaProtocolMatchByValue = KafkaDsl.kafka() 24 | .producerSettings( 25 | Map.of( 26 | ProducerConfig.ACKS_CONFIG, "1", 27 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 28 | ) 29 | ) 30 | .consumeSettings( 31 | Map.of("bootstrap.servers", "localhost:9092") 32 | ) 33 | .timeout(Duration.ofSeconds(5)) 34 | // for match by message value 35 | .matchByValue(); 36 | 37 | private byte[] matchByOwnVal(KafkaProtocolMessage message) { 38 | // do something with the message and extract the values you are interested in 39 | // method is called: 40 | // - for each message which will be sent out 41 | // - for each message which has been received 42 | return "Custom Message".getBytes(); // just returning something 43 | } 44 | 45 | private final KafkaProtocolBuilder kafkaProtocolMatchByMessage = KafkaDsl.kafka() 46 | .producerSettings( 47 | Map.of( 48 | ProducerConfig.ACKS_CONFIG, "1", 49 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 50 | ) 51 | ) 52 | .consumeSettings( 53 | Map.of( 54 | "bootstrap.servers", "localhost:9092" 55 | ) 56 | ) 57 | .timeout(Duration.ofSeconds(5)) 58 | .matchByMessage(this::matchByOwnVal); 59 | 60 | private final AtomicInteger c = new AtomicInteger(0); 61 | private final Iterator> feeder = 62 | Stream.generate((Supplier>) () -> Collections.singletonMap("kekey", c.incrementAndGet()) 63 | ).iterator(); 64 | 65 | private final ScenarioBuilder scn = scenario("Basic") 66 | .feed(feeder) 67 | .exec( 68 | KafkaDsl.kafka("ReqRep").requestReply() 69 | .requestTopic("test.t") 70 | .replyTopic("test.t") 71 | .send("#{kekey}", """ 72 | { "m": "dkf" } 73 | """)); 74 | 75 | { 76 | setUp( 77 | scn.injectOpen(atOnceUsers(1))) 78 | .protocols(kafkaProtocolMatchByMessage) 79 | .maxDuration(Duration.ofSeconds(120)); 80 | } 81 | 82 | } 83 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/expressions/Builders.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.expressions; 2 | 3 | import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; 4 | import io.confluent.kafka.serializers.KafkaAvroDeserializer; 5 | import io.confluent.kafka.serializers.KafkaAvroSerializer; 6 | import org.apache.kafka.common.serialization.Deserializer; 7 | import org.apache.kafka.common.serialization.Serdes; 8 | import org.apache.kafka.common.serialization.Serializer; 9 | import org.apache.kafka.common.utils.Bytes; 10 | 11 | import java.nio.ByteBuffer; 12 | 13 | public class Builders { 14 | public Builders() { 15 | } 16 | 17 | public static final class FloatExpressionBuilder extends ExpressionBuilder { 18 | public FloatExpressionBuilder(JExpression javaExpression) { 19 | super(javaExpression, Float.class, Serdes.Float()); 20 | } 21 | } 22 | 23 | public static final class DoubleExpressionBuilder extends ExpressionBuilder { 24 | public DoubleExpressionBuilder(JExpression javaExpression) { 25 | super(javaExpression, Double.class, Serdes.Double()); 26 | } 27 | } 28 | 29 | public static final class ShortExpressionBuilder extends ExpressionBuilder { 30 | public ShortExpressionBuilder(JExpression javaExpression) { 31 | super(javaExpression, Short.class, Serdes.Short()); 32 | } 33 | } 34 | 35 | public static final class IntegerExpressionBuilder extends ExpressionBuilder { 36 | public IntegerExpressionBuilder(JExpression javaExpression) { 37 | super(javaExpression, Integer.class, Serdes.Integer()); 38 | } 39 | } 40 | 41 | public static final class LongExpressionBuilder extends ExpressionBuilder { 42 | public LongExpressionBuilder(JExpression javaExpression) { 43 | super(javaExpression, Long.class, Serdes.Long()); 44 | } 45 | } 46 | 47 | public static final class ByteArrayExpressionBuilder extends ExpressionBuilder { 48 | public ByteArrayExpressionBuilder(JExpression javaExpression) { 49 | super(javaExpression, byte[].class, Serdes.ByteArray()); 50 | } 51 | } 52 | 53 | public static final class ByteBufferExpressionBuilder extends ExpressionBuilder { 54 | public ByteBufferExpressionBuilder(JExpression javaExpression) { 55 | super(javaExpression, ByteBuffer.class, Serdes.ByteBuffer()); 56 | } 57 | } 58 | 59 | public static final class BytesExpressionBuilder extends ExpressionBuilder { 60 | public BytesExpressionBuilder(JExpression javaExpression) { 61 | super(javaExpression, Bytes.class, Serdes.Bytes()); 62 | } 63 | } 64 | 65 | public static final class StringExpressionBuilder extends ExpressionBuilder { 66 | public StringExpressionBuilder(JExpression javaExpression) { 67 | super(javaExpression, String.class, Serdes.String()); 68 | } 69 | } 70 | 71 | public static final class AvroExpressionBuilder extends ExpressionBuilder { 72 | public AvroExpressionBuilder(JExpression valueF, SchemaRegistryClient client) { 73 | super(valueF, Object.class, Serdes.serdeFrom(new KafkaAvroSerializer(client), new KafkaAvroDeserializer(client))); 74 | } 75 | 76 | public AvroExpressionBuilder(JExpression valueF, Serializer ser, Deserializer deser) { 77 | super(valueF, Object.class, Serdes.serdeFrom(ser, deser)); 78 | } 79 | } 80 | 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/checks/KafkaChecks.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.checks 2 | 3 | import com.fasterxml.jackson.databind.JsonNode 4 | import io.confluent.kafka.streams.serdes.avro.GenericAvroSerde 5 | import io.gatling.core.check._ 6 | import io.gatling.core.check.bytes.BodyBytesCheckType 7 | import io.gatling.core.check.jmespath.JmesPathCheckType 8 | import io.gatling.core.check.jsonpath.JsonPathCheckType 9 | import io.gatling.core.check.string.BodyStringCheckType 10 | import io.gatling.core.check.substring.SubstringCheckType 11 | import io.gatling.core.check.xpath.XPathCheckType 12 | import io.gatling.javaapi.core.internal.CoreCheckType 13 | import net.sf.saxon.s9api.XdmNode 14 | import org.apache.avro.generic.GenericRecord 15 | import org.apache.kafka.common.serialization.Serde 16 | import org.galaxio.gatling.kafka.checks.{KafkaCheckMaterializer, KafkaCheckSupport} 17 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 18 | import org.galaxio.gatling.kafka.{KafkaCheck, checks} 19 | 20 | import java.{util => ju} 21 | import scala.jdk.CollectionConverters._ 22 | 23 | object KafkaChecks { 24 | class SimpleChecksScala extends KafkaCheckSupport {} 25 | 26 | val avroSerde: Serde[GenericRecord] = new GenericAvroSerde() 27 | 28 | private def toScalaCheck(javaCheckBuilder: io.gatling.javaapi.core.CheckBuilder): KafkaCheck = { 29 | val scalaCheck = javaCheckBuilder.asScala 30 | javaCheckBuilder.`type` match { 31 | case CoreCheckType.BodyBytes => 32 | scalaCheck 33 | .asInstanceOf[CheckBuilder[BodyBytesCheckType, Array[Byte]]] 34 | .build(KafkaCheckMaterializer.bodyBytes) 35 | case CoreCheckType.BodyString => 36 | scalaCheck 37 | .asInstanceOf[CheckBuilder[BodyStringCheckType, String]] 38 | .build(KafkaCheckMaterializer.bodyString(io.gatling.core.Predef.configuration)) 39 | case CoreCheckType.Substring => 40 | scalaCheck 41 | .asInstanceOf[CheckBuilder[SubstringCheckType, String]] 42 | .build(KafkaCheckMaterializer.substring(io.gatling.core.Predef.configuration)) 43 | case CoreCheckType.XPath => 44 | scalaCheck 45 | .asInstanceOf[CheckBuilder[XPathCheckType, XdmNode]] 46 | .build(KafkaCheckMaterializer.xpath(io.gatling.core.Predef.configuration)) 47 | case CoreCheckType.JsonPath => 48 | scalaCheck 49 | .asInstanceOf[CheckBuilder[JsonPathCheckType, JsonNode]] 50 | .build( 51 | KafkaCheckMaterializer.jsonPath(io.gatling.core.Predef.defaultJsonParsers, io.gatling.core.Predef.configuration), 52 | ) 53 | case CoreCheckType.JmesPath => 54 | scalaCheck 55 | .asInstanceOf[CheckBuilder[JmesPathCheckType, JsonNode]] 56 | .build( 57 | KafkaCheckMaterializer.jmesPath(io.gatling.core.Predef.defaultJsonParsers, io.gatling.core.Predef.configuration), 58 | ) 59 | case KafkaCheckType.ResponseCode => 60 | scalaCheck 61 | .asInstanceOf[CheckBuilder[checks.KafkaCheckMaterializer.KafkaMessageCheckType, KafkaProtocolMessage]] 62 | .build( 63 | KafkaCheckMaterializer.kafkaStatusCheck, 64 | ) 65 | case KafkaCheckType.Simple => 66 | scalaCheck 67 | .asInstanceOf[CheckBuilder[checks.KafkaCheckMaterializer.KafkaMessageCheckType, KafkaProtocolMessage]] 68 | .build(KafkaCheckMaterializer.kafkaStatusCheck) 69 | case unknown => throw new IllegalArgumentException(s"Kafka DSL doesn't support $unknown") 70 | } 71 | 72 | } 73 | 74 | def toScalaChecks(javaChecks: ju.List[io.gatling.javaapi.core.CheckBuilder]): Seq[KafkaCheck] = 75 | javaChecks.asScala.map(toScalaCheck).toSeq 76 | 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/client/KafkaMessageTrackerPool.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.client 2 | 3 | import io.gatling.commons.util.Clock 4 | import io.gatling.core.actor.{ActorRef, ActorSystem} 5 | import io.gatling.core.stats.StatsEngine 6 | import io.gatling.core.util.NameGen 7 | import org.apache.kafka.clients.consumer.ConsumerConfig 8 | import org.galaxio.gatling.kafka.KafkaLogging 9 | import org.galaxio.gatling.kafka.client.KafkaMessageTracker.MessageConsumed 10 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaMatcher 11 | import org.galaxio.gatling.kafka.request.{KafkaProtocolMessage, KafkaSerdesImplicits} 12 | 13 | import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Executors} 14 | import scala.concurrent.duration.FiniteDuration 15 | 16 | object KafkaMessageTrackerPool { 17 | 18 | def apply( 19 | consumerSettings: Map[String, AnyRef], 20 | actorSystem: ActorSystem, 21 | statsEngine: StatsEngine, 22 | clock: Clock, 23 | ): Option[KafkaMessageTrackerPool] = 24 | Option.when(consumerSettings.contains(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG))( 25 | new KafkaMessageTrackerPool(consumerSettings, actorSystem, statsEngine, clock), 26 | ) 27 | 28 | private val consumerExecutor: ExecutorService = Executors.newSingleThreadExecutor() 29 | } 30 | 31 | final class KafkaMessageTrackerPool( 32 | consumerSettings: Map[String, AnyRef], 33 | actorSystem: ActorSystem, 34 | statsEngine: StatsEngine, 35 | clock: Clock, 36 | ) extends KafkaLogging with NameGen with KafkaSerdesImplicits { 37 | 38 | // Trackers map Output Topic (String) to Tracker/Actor 39 | private val trackers = new ConcurrentHashMap[String, ActorRef[KafkaMessageTracker.TrackerMessage]] 40 | private val trackerName = "kafkaTracker" 41 | 42 | private val consumer: DynamicKafkaConsumer[Array[Byte], Array[Byte]] = 43 | DynamicKafkaConsumer( 44 | if (consumerSettings.contains(ConsumerConfig.GROUP_ID_CONFIG)) 45 | consumerSettings 46 | else 47 | consumerSettings + (ConsumerConfig.GROUP_ID_CONFIG -> s"gatling-kafka-test-${java.util.UUID.randomUUID()}"), 48 | Set.empty, 49 | record => { 50 | val kafkaProtocolMessage = KafkaProtocolMessage.from(record, None) 51 | val receivedTimestamp = clock.nowMillis 52 | val tracker = Option(trackers.get(record.topic())) 53 | 54 | tracker.map( 55 | _ ! MessageConsumed( 56 | receivedTimestamp, 57 | kafkaProtocolMessage, 58 | ), 59 | ) 60 | }, 61 | exception => logger.error(exception.getMessage, exception), 62 | ) 63 | 64 | private val consumerFuture = KafkaMessageTrackerPool.consumerExecutor.submit(consumer) 65 | actorSystem.registerOnTermination { 66 | logger.debug("Closing consumer {}", consumer) 67 | consumer.close() 68 | try { 69 | consumerFuture.get() 70 | } catch { 71 | case e: Throwable => 72 | logger.error(e.getMessage, e) 73 | } 74 | KafkaMessageTrackerPool.consumerExecutor.shutdown() 75 | } 76 | 77 | private def withProducerTopic(producerTopic: String): KafkaProtocolMessage => KafkaProtocolMessage = 78 | _.copy(producerTopic = producerTopic) 79 | 80 | def tracker( 81 | producerTopic: String, 82 | consumerTopic: String, 83 | messageMatcher: KafkaMatcher, 84 | responseTransformer: Option[KafkaProtocolMessage => KafkaProtocolMessage], 85 | timeout: FiniteDuration, 86 | ): ActorRef[KafkaMessageTracker.TrackerMessage] = { 87 | 88 | trackers.computeIfAbsent( 89 | consumerTopic, 90 | _ => { 91 | logger.debug( 92 | "Computing new tracker for topic {}, there are currently {} other trackers", 93 | consumerTopic, 94 | trackers.size(), 95 | ) 96 | val name = genName(trackerName) 97 | val transformations = 98 | responseTransformer.fold(withProducerTopic(producerTopic))(_.compose(withProducerTopic(producerTopic))) 99 | val tracker = 100 | actorSystem.actorOf( 101 | KafkaMessageTracker.actor( 102 | name, 103 | statsEngine, 104 | clock, 105 | messageMatcher, 106 | Option(transformations), 107 | ), 108 | ) 109 | consumer.addTopicForSubscription(consumerTopic, timeout) 110 | tracker 111 | }, 112 | ) 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/checks/KafkaCheckSupport.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.checks 2 | 3 | import com.fasterxml.jackson.databind.JsonNode 4 | import io.gatling.commons.validation._ 5 | import io.gatling.core.check.Check.PreparedCache 6 | import io.gatling.core.check.bytes.BodyBytesCheckType 7 | import io.gatling.core.check.jmespath.JmesPathCheckType 8 | import io.gatling.core.check.jsonpath.JsonPathCheckType 9 | import io.gatling.core.check.string.BodyStringCheckType 10 | import io.gatling.core.check.substring.SubstringCheckType 11 | import io.gatling.core.check.xpath.XPathCheckType 12 | import io.gatling.core.check.{Check, CheckBuilder, CheckMaterializer, CheckResult, TypedCheckIfMaker, UntypedCheckIfMaker} 13 | import io.gatling.core.config.GatlingConfiguration 14 | import io.gatling.core.json.JsonParsers 15 | import io.gatling.core.session.Session 16 | import net.sf.saxon.s9api.XdmNode 17 | import org.apache.avro.generic.GenericRecord 18 | import org.apache.kafka.common.serialization.Serde 19 | import org.galaxio.gatling.kafka.KafkaCheck 20 | import org.galaxio.gatling.kafka.checks.KafkaCheckMaterializer.KafkaMessageCheckType 21 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 22 | 23 | import scala.annotation.implicitNotFound 24 | 25 | trait KafkaCheckSupport { 26 | def messageCheck: KafkaMessageCheck.type = KafkaMessageCheck 27 | 28 | def avroBody[T: Serde]: CheckBuilder.Find[KafkaMessageCheckType, KafkaProtocolMessage, T] = 29 | AvroBodyCheckBuilder._avroBody 30 | 31 | def simpleCheck(f: KafkaProtocolMessage => Boolean): KafkaCheck = 32 | Check.Simple( 33 | (response: KafkaProtocolMessage, _: Session, _: PreparedCache) => 34 | if (f(response)) { 35 | CheckResult.NoopCheckResultSuccess 36 | } else { 37 | "Kafka record check failed".failure 38 | }, 39 | None, 40 | ) 41 | 42 | @implicitNotFound("Could not find a CheckMaterializer. This check might not be valid for Kafka.") 43 | implicit def checkBuilder2KafkaCheck[T, P]( 44 | checkBuilder: CheckBuilder[T, P], 45 | )(implicit materializer: CheckMaterializer[T, KafkaCheck, KafkaProtocolMessage, P]): KafkaCheck = 46 | checkBuilder.build(materializer) 47 | 48 | @implicitNotFound("Could not find a CheckMaterializer. This check might not be valid for Kafka.") 49 | implicit def validatorCheckBuilder2KafkaCheck[T, P, X]( 50 | validate: CheckBuilder.Validate[T, P, X], 51 | )(implicit materializer: CheckMaterializer[T, KafkaCheck, KafkaProtocolMessage, P]): KafkaCheck = 52 | validate.exists 53 | 54 | @implicitNotFound("Could not find a CheckMaterializer. This check might not be valid for Kafka.") 55 | implicit def findCheckBuilder2KafkaCheck[T, P, X]( 56 | find: CheckBuilder.Find[T, P, X], 57 | )(implicit materializer: CheckMaterializer[T, KafkaCheck, KafkaProtocolMessage, P]): KafkaCheck = 58 | find.find.exists 59 | 60 | implicit def kafkaXPathMaterializer(implicit 61 | configuration: GatlingConfiguration, 62 | ): KafkaCheckMaterializer[XPathCheckType, XdmNode] = 63 | KafkaCheckMaterializer.xpath(configuration) 64 | 65 | implicit def kafkaJsonPathMaterializer(implicit 66 | jsonParsers: JsonParsers, 67 | configuration: GatlingConfiguration, 68 | ): KafkaCheckMaterializer[JsonPathCheckType, JsonNode] = 69 | KafkaCheckMaterializer.jsonPath(jsonParsers, configuration) 70 | 71 | implicit def kafkaJmesPathMaterializer(implicit 72 | jsonParsers: JsonParsers, 73 | configuration: GatlingConfiguration, 74 | ): KafkaCheckMaterializer[JmesPathCheckType, JsonNode] = 75 | KafkaCheckMaterializer.jmesPath(jsonParsers, configuration) 76 | 77 | implicit def kafkaBodyStringMaterializer(implicit 78 | configuration: GatlingConfiguration, 79 | ): KafkaCheckMaterializer[BodyStringCheckType, String] = 80 | KafkaCheckMaterializer.bodyString(configuration) 81 | 82 | implicit def kafkaSubstringMaterializer(implicit 83 | configuration: GatlingConfiguration, 84 | ): KafkaCheckMaterializer[SubstringCheckType, String] = 85 | KafkaCheckMaterializer.substring(configuration) 86 | 87 | implicit def kafkaBodyByteMaterializer: KafkaCheckMaterializer[BodyBytesCheckType, Array[Byte]] = 88 | KafkaCheckMaterializer.bodyBytes 89 | 90 | implicit val kafkaStatusCheckMaterializer: KafkaCheckMaterializer[KafkaMessageCheckType, KafkaProtocolMessage] = 91 | KafkaCheckMaterializer.kafkaStatusCheck 92 | 93 | implicit val kafkaUntypedConditionalCheckWrapper: UntypedCheckIfMaker[KafkaCheck] = _.checkIf(_) 94 | 95 | implicit val kafkaTypedConditionalCheckWrapper: TypedCheckIfMaker[KafkaProtocolMessage, KafkaCheck] = _.checkIf(_) 96 | } 97 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/protocol/KafkaProtocolBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.protocol 2 | 3 | import org.apache.kafka.clients.consumer.ConsumerConfig 4 | import org.apache.kafka.clients.producer.ProducerConfig 5 | import org.apache.kafka.common.serialization.Serdes 6 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol._ 7 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 8 | 9 | import scala.concurrent.duration.{DurationInt, FiniteDuration} 10 | 11 | object KafkaProtocolBuilder { 12 | 13 | case class KafkaProtocolBuilderPropertiesStep(topic: String) { 14 | 15 | def properties(producerSettings: Map[String, Object]): KafkaProtocolBuilderBackwardCompatible = 16 | KafkaProtocolBuilderBackwardCompatible(topic, producerSettings, 60.seconds) 17 | } 18 | 19 | @deprecated("use topic definition in kafka request builders", "1.0.0") 20 | def topic(name: String): KafkaProtocolBuilderPropertiesStep = 21 | KafkaProtocolBuilderPropertiesStep(name) 22 | 23 | @deprecated( 24 | "separate definition of the protocol for the requestReply scheme is no longer required; use producerSettings right away", 25 | "1.0.0", 26 | ) 27 | def requestReply: KafkaProtocolBuilder.type = KafkaProtocolBuilder 28 | 29 | def producerSettings(ps: Map[String, AnyRef]): KPProducerSettingsStep = KPProducerSettingsStep(ps) 30 | 31 | def producerSettings(pp: (String, AnyRef), pps: (String, AnyRef)*): KPProducerSettingsStep = producerSettings( 32 | (pp +: pps).toMap, 33 | ) 34 | 35 | def properties(producerSettings: Map[String, Object]): KafkaProtocolBuilder = 36 | KafkaProtocolBuilder(producerSettings, Map.empty, 60.seconds) 37 | 38 | def properties(p: (String, AnyRef), ps: (String, AnyRef)*): KafkaProtocolBuilder = 39 | properties((p +: ps).toMap) 40 | 41 | case class KPProducerSettingsStep(producerSettings: Map[String, AnyRef]) { 42 | def consumeSettings(cs: Map[String, AnyRef]): KPConsumeSettingsStep = KPConsumeSettingsStep(producerSettings, cs) 43 | 44 | def consumeSettings(cp: (String, AnyRef), cps: (String, AnyRef)*): KPConsumeSettingsStep = consumeSettings( 45 | (cp +: cps).toMap, 46 | ) 47 | 48 | def timeout(t: FiniteDuration): KafkaProtocolBuilder = KafkaProtocolBuilder(producerSettings, Map.empty, t) 49 | def withDefaultTimeout: KafkaProtocolBuilder = KafkaProtocolBuilder(producerSettings, Map.empty, 60.seconds) 50 | 51 | } 52 | 53 | case class KPConsumeSettingsStep(producerSettings: Map[String, AnyRef], consumeSettings: Map[String, AnyRef]) { 54 | def timeout(t: FiniteDuration): KafkaProtocolBuilder = KafkaProtocolBuilder(producerSettings, consumeSettings, t) 55 | def withDefaultTimeout: KafkaProtocolBuilder = KafkaProtocolBuilder(producerSettings, consumeSettings, 60.seconds) 56 | } 57 | } 58 | 59 | final case class KafkaProtocolBuilder( 60 | producerSettings: Map[String, AnyRef], 61 | consumeSettings: Map[String, AnyRef], 62 | timeout: FiniteDuration, 63 | messageMatcher: KafkaMatcher = KafkaKeyMatcher, 64 | ) { 65 | 66 | def matchByValue: KafkaProtocolBuilder = 67 | messageMatcher(KafkaValueMatcher) 68 | 69 | def matchByMessage(keyExtractor: KafkaProtocolMessage => Array[Byte]): KafkaProtocolBuilder = 70 | messageMatcher(KafkaMessageMatcher(keyExtractor)) 71 | 72 | private def messageMatcher(matcher: KafkaMatcher): KafkaProtocolBuilder = 73 | copy(messageMatcher = matcher) 74 | 75 | private def withDefaultAutoReset(settings: Map[String, AnyRef]): Map[String, AnyRef] = 76 | if (settings.contains(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)) settings 77 | else settings + (ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "latest") 78 | 79 | private def withDefaultAutoCommit(settings: Map[String, AnyRef]): Map[String, AnyRef] = 80 | if (settings.contains(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) settings 81 | else settings + (ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "true") 82 | 83 | def build: KafkaProtocol = { 84 | 85 | val serializers = Map( 86 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 87 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 88 | ) 89 | 90 | val consumerSettingsWithDefaults = 91 | withDefaultAutoReset(consumeSettings) ++ withDefaultAutoCommit(consumeSettings) ++ 92 | Map( 93 | ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> Serdes.ByteArray().deserializer().getClass.getName, 94 | ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> Serdes.ByteArray().deserializer().getClass.getName, 95 | ) 96 | 97 | KafkaProtocol("kafka-test", producerSettings ++ serializers, consumerSettingsWithDefaults, timeout, messageMatcher) 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/protocol/KafkaProtocol.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.protocol 2 | 3 | import io.gatling.core.CoreComponents 4 | import io.gatling.core.config.GatlingConfiguration 5 | import io.gatling.core.protocol.{Protocol, ProtocolKey} 6 | import org.apache.kafka.clients.consumer.ConsumerConfig 7 | import org.apache.kafka.clients.producer.ProducerConfig 8 | import org.galaxio.gatling.kafka.client.{KafkaMessageTrackerPool, KafkaSender} 9 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaMatcher 10 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 11 | 12 | import java.util.concurrent.ConcurrentHashMap 13 | import java.util.concurrent.atomic.AtomicReference 14 | import scala.concurrent.duration.FiniteDuration 15 | 16 | object KafkaProtocol { 17 | 18 | trait KafkaMatcher { 19 | def requestMatch(msg: KafkaProtocolMessage): Array[Byte] 20 | def responseMatch(msg: KafkaProtocolMessage): Array[Byte] 21 | } 22 | 23 | object KafkaKeyMatcher extends KafkaMatcher { 24 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.key 25 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.key 26 | } 27 | 28 | object KafkaValueMatcher extends KafkaMatcher { 29 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.value 30 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.value 31 | } 32 | 33 | case class KafkaMessageMatcher(keyExtractor: KafkaProtocolMessage => Array[Byte]) extends KafkaMatcher { 34 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = keyExtractor(msg) 35 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = keyExtractor(msg) 36 | } 37 | 38 | val kafkaProtocolKey: ProtocolKey[KafkaProtocol, KafkaComponents] = new ProtocolKey[KafkaProtocol, KafkaComponents] { 39 | private val senders = new ConcurrentHashMap[String, KafkaSender]() 40 | private val trackerPools = new ConcurrentHashMap[String, Option[KafkaMessageTrackerPool]]() 41 | 42 | private def getOrCreateSender(protocol: KafkaProtocol): KafkaSender = 43 | protocol.producerProperties.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) match { 44 | case Some(servers) => this.senders.computeIfAbsent(servers.toString, _ => KafkaSender(protocol.producerProperties)) 45 | case None => 46 | throw new IllegalArgumentException( 47 | s"Producer settings don't set the required '${ProducerConfig.BOOTSTRAP_SERVERS_CONFIG}' parameter", 48 | ) 49 | } 50 | 51 | private def getOrCreateTrackerPool( 52 | coreComponents: CoreComponents, 53 | protocol: KafkaProtocol, 54 | ): Option[KafkaMessageTrackerPool] = 55 | protocol.consumerProperties 56 | .get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG) 57 | .flatMap(servers => 58 | trackerPools.computeIfAbsent( 59 | servers.toString, 60 | _ => 61 | KafkaMessageTrackerPool( 62 | protocol.consumerProperties, 63 | coreComponents.actorSystem, 64 | coreComponents.statsEngine, 65 | coreComponents.clock, 66 | ), 67 | ), 68 | ) 69 | 70 | override def protocolClass: Class[Protocol] = 71 | classOf[KafkaProtocol].asInstanceOf[Class[Protocol]] 72 | 73 | override def defaultProtocolValue(configuration: GatlingConfiguration): KafkaProtocol = 74 | throw new IllegalStateException("Can't provide a default value for KafkaProtocol") 75 | 76 | override def newComponents(coreComponents: CoreComponents): KafkaProtocol => KafkaComponents = 77 | kafkaProtocol => 78 | KafkaComponents( 79 | coreComponents, 80 | kafkaProtocol, 81 | getOrCreateTrackerPool(coreComponents, kafkaProtocol), 82 | getOrCreateSender(kafkaProtocol), 83 | ) 84 | } 85 | } 86 | 87 | final case class KafkaProtocol( 88 | producerTopic: String, // TODO: remove after 1.1.0 (when topic moved from protocol to request builders) 89 | producerProperties: Map[String, AnyRef], 90 | consumerProperties: Map[String, AnyRef], 91 | timeout: FiniteDuration, 92 | messageMatcher: KafkaMatcher, 93 | ) extends Protocol { 94 | 95 | def topic(t: String): KafkaProtocol = copy(producerTopic = t) 96 | 97 | def properties(properties: Map[String, AnyRef]): KafkaProtocol = 98 | copy(producerProperties = properties) 99 | 100 | def producerProperties(properties: Map[String, AnyRef]): KafkaProtocol = copy(producerProperties = properties) 101 | def consumerProperties(properties: Map[String, AnyRef]): KafkaProtocol = copy(consumerProperties = properties) 102 | def timeout(t: FiniteDuration): KafkaProtocol = copy(timeout = t) 103 | } 104 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/request/builder/KafkaRequestBuilderBase.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.session._ 4 | import org.apache.kafka.common.header.{Header, Headers} 5 | import org.apache.kafka.common.serialization.Serde 6 | import org.galaxio.gatling.kafka.actions.KafkaRequestReplyActionBuilder 7 | import org.galaxio.gatling.kafka.Predef._ 8 | 9 | import scala.reflect.ClassTag 10 | 11 | object KafkaRequestBuilderBase { 12 | 13 | final case class OnlyPublishStep(requestName: Expression[String], producerTopic: Expression[String]) { 14 | def send[V: Serde: ClassTag](value: Expression[V]): RequestBuilder[Nothing, V] = 15 | KafkaRequestBuilder[Nothing, V]( 16 | KafkaAttributes( 17 | requestName = requestName, 18 | producerTopic = Option(producerTopic), 19 | consumerTopic = None, 20 | key = None, 21 | value = value, 22 | headers = None, 23 | keySerde = None, 24 | valueSerde = implicitly[Serde[V]], 25 | checks = List.empty, 26 | ), 27 | ) 28 | 29 | def send[K: Serde: ClassTag, V: Serde: ClassTag]( 30 | key: Expression[K], 31 | value: Expression[V], 32 | headers: Expression[Headers] = List.empty[Header], 33 | ): RequestBuilder[K, V] = 34 | KafkaRequestBuilder( 35 | KafkaAttributes( 36 | requestName = requestName, 37 | producerTopic = Option(producerTopic), 38 | consumerTopic = None, 39 | key = Option(key), 40 | value = value, 41 | headers = Option(headers), 42 | keySerde = Option(implicitly[Serde[K]]), 43 | valueSerde = implicitly[Serde[V]], 44 | checks = List.empty, 45 | ), 46 | ) 47 | } 48 | 49 | } 50 | 51 | case class KafkaRequestBuilderBase(requestName: Expression[String]) { 52 | 53 | def send[K: Serde: ClassTag, V: Serde: ClassTag]( 54 | key: Expression[K], 55 | payload: Expression[V], 56 | headers: Expression[Headers] = List.empty[Header], 57 | ): RequestBuilder[K, V] = { 58 | if (key == null) 59 | KafkaRequestBuilder[Nothing, V]( 60 | KafkaAttributes( 61 | requestName = requestName, 62 | producerTopic = None, // TODO: it should be set after topic definition 63 | consumerTopic = None, 64 | key = None, 65 | value = payload, 66 | headers = Option(headers), 67 | keySerde = None, 68 | valueSerde = implicitly[Serde[V]], 69 | checks = List.empty, 70 | ), 71 | ) 72 | else 73 | KafkaRequestBuilder( 74 | KafkaAttributes( 75 | requestName = requestName, 76 | producerTopic = None, // TODO: it should be set after topic definition 77 | consumerTopic = None, 78 | key = Option(key), 79 | value = payload, 80 | headers = Option(headers), 81 | keySerde = Some(implicitly[Serde[K]]), 82 | valueSerde = implicitly[Serde[V]], 83 | checks = List.empty, 84 | ), 85 | ) 86 | } 87 | 88 | def send[V: Serde: ClassTag](payload: Expression[V]): RequestBuilder[Nothing, V] = 89 | KafkaRequestBuilder[Nothing, V]( 90 | KafkaAttributes( 91 | requestName = requestName, 92 | producerTopic = None, // TODO: it should be set after topic definition 93 | consumerTopic = None, 94 | key = None, 95 | value = payload, 96 | headers = None, 97 | keySerde = None, 98 | valueSerde = implicitly[Serde[V]], 99 | checks = List.empty, 100 | ), 101 | ) 102 | 103 | def topic(producerTopic: Expression[String]): KafkaRequestBuilderBase.OnlyPublishStep = 104 | KafkaRequestBuilderBase.OnlyPublishStep(requestName, producerTopic) 105 | 106 | def requestReply: ReqRepBase.type = ReqRepBase 107 | 108 | object ReqRepBase { 109 | case class RROutTopicStep(producerTopic: Expression[String], consumerTopic: Expression[String]) { 110 | def send[K: Serde: ClassTag, V: Serde: ClassTag]( 111 | key: Expression[K], 112 | payload: Expression[V], 113 | headers: Expression[Headers] = List.empty[Header].expressionSuccess, 114 | ): KafkaRequestReplyActionBuilder[K, V] = { 115 | KafkaRequestReplyActionBuilder[K, V]( 116 | KafkaAttributes[K, V]( 117 | requestName, 118 | Option(producerTopic), 119 | Option(consumerTopic), 120 | Option(key), 121 | payload, 122 | Option(headers), 123 | Option(implicitly[Serde[K]]), 124 | implicitly[Serde[V]], 125 | List.empty, 126 | ), 127 | ) 128 | } 129 | } 130 | 131 | case class RRInTopicStep(producerTopic: Expression[String]) { 132 | def replyTopic(consumerTopic: Expression[String]): RROutTopicStep = RROutTopicStep(producerTopic, consumerTopic) 133 | } 134 | def requestTopic(rt: Expression[String]): RRInTopicStep = RRInTopicStep(rt) 135 | 136 | } 137 | 138 | } 139 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/actions/KafkaAction.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.actions 2 | 3 | import io.gatling.commons.validation._ 4 | import io.gatling.core.action.RequestAction 5 | import io.gatling.core.actor.ActorRef 6 | import io.gatling.core.controller.throttle.Throttler 7 | import io.gatling.core.session.{Expression, Session} 8 | import io.gatling.core.session.el._ 9 | import io.gatling.core.util.NameGen 10 | import org.apache.kafka.common.serialization.{Serde, Serializer} 11 | import org.galaxio.gatling.kafka.KafkaLogging 12 | import org.galaxio.gatling.kafka.protocol.KafkaComponents 13 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 14 | import org.galaxio.gatling.kafka.request.builder.KafkaAttributes 15 | 16 | import scala.reflect.{ClassTag, classTag} 17 | 18 | abstract class KafkaAction[K: ClassTag, V: ClassTag]( 19 | components: KafkaComponents, // TODO: remove it after 1.1.0 (when topic removed from protocol) 20 | attributes: KafkaAttributes[K, V], 21 | throttler: Option[ActorRef[Throttler.Command]], 22 | ) extends RequestAction with KafkaLogging with NameGen { 23 | 24 | override def requestName: Expression[String] = attributes.requestName 25 | 26 | override def sendRequest(session: Session): Validation[Unit] = { 27 | for { 28 | requestNameString <- requestName(session) 29 | protocolMessage <- resolveToProtocolMessage(session) 30 | } yield throttler 31 | .fold(sendKafkaMessage(requestNameString, protocolMessage, session))( 32 | _ ! Throttler.Command 33 | .ThrottledRequest(session.scenario, () => sendKafkaMessage(requestNameString, protocolMessage, session)), 34 | ) 35 | 36 | } 37 | 38 | private def traverse[T](ovt: Option[Validation[T]]): Validation[Option[T]] = 39 | ovt.fold(Option.empty[T].success)(_.map(Option[T])) 40 | 41 | private def serializeKey( 42 | serde: Option[Serde[? <: K]], 43 | keyExpression: Option[Expression[? <: K]], 44 | topicExpression: Expression[String], 45 | ): Expression[Option[Array[Byte]]] = session => 46 | // need for work gatling Expression Language 47 | if (classTag[K].runtimeClass.getCanonicalName == "java.lang.String") 48 | for { 49 | topic <- topicExpression(session) 50 | result <- traverse(for { 51 | serializer <- serde.asInstanceOf[Option[Serde[String]]].map(_.serializer()) 52 | key <- keyExpression.asInstanceOf[Option[Expression[String]]].map(_(session)) 53 | keyEl = key.flatMap(_.el[String].apply(session)) 54 | } yield keyEl.map(serializer.serialize(topic, _))) 55 | } yield result 56 | else 57 | for { 58 | topic <- topicExpression(session) 59 | result <- traverse(for { 60 | serializer <- serde.map(_.serializer().asInstanceOf[Serializer[K]]) 61 | key <- keyExpression.map(_(session)) 62 | } yield key.map(serializer.serialize(topic, _))) 63 | } yield result 64 | 65 | private def resolveToProtocolMessage: Expression[KafkaProtocolMessage] = s => 66 | // need for work gatling Expression Language 67 | if (classTag[V].runtimeClass.getCanonicalName == "java.lang.String") 68 | for { 69 | key <- serializeKey( 70 | attributes.keySerde, 71 | attributes.key, 72 | attributes.producerTopic.getOrElse(components.kafkaProtocol.producerTopic.el), 73 | )(s) 74 | producerTopic <- attributes.producerTopic.fold(components.kafkaProtocol.producerTopic.success)(_(s)) 75 | consumerTopic <- traverse(attributes.consumerTopic.map(_(s))) 76 | value <- attributes.value 77 | .asInstanceOf[Expression[String]](s) 78 | .flatMap(_.el[String].apply(s)) 79 | .map(v => attributes.valueSerde.asInstanceOf[Serde[String]].serializer().serialize(producerTopic, v)) 80 | headers <- traverse(attributes.headers.map(_(s))) 81 | } yield KafkaProtocolMessage( 82 | key.getOrElse(Array.emptyByteArray), 83 | value, 84 | producerTopic, 85 | consumerTopic.getOrElse(""), 86 | headers, 87 | ) 88 | else 89 | for { 90 | key <- serializeKey( 91 | attributes.keySerde, 92 | attributes.key, 93 | attributes.producerTopic.getOrElse(components.kafkaProtocol.producerTopic.el), 94 | )(s) 95 | producerTopic <- attributes.producerTopic.fold(components.kafkaProtocol.producerTopic.success)(_(s)) 96 | consumerTopic <- traverse(attributes.consumerTopic.map(_(s))) 97 | value <- attributes 98 | .value(s) 99 | .map(v => attributes.valueSerde.serializer().asInstanceOf[Serializer[V]].serialize(producerTopic, v)) 100 | headers <- traverse(attributes.headers.map(_(s))) 101 | } yield KafkaProtocolMessage( 102 | key.getOrElse(Array.emptyByteArray), 103 | value, 104 | producerTopic, 105 | consumerTopic.getOrElse(""), 106 | headers, 107 | ) 108 | 109 | def sendKafkaMessage(requestNameString: String, protocolMessage: KafkaProtocolMessage, session: Session): Unit 110 | } 111 | -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/client/DynamicKafkaConsumer.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.client 2 | 3 | import com.typesafe.scalalogging.StrictLogging 4 | import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} 5 | import org.apache.kafka.common.TopicPartition 6 | import org.apache.kafka.common.errors.WakeupException 7 | 8 | import java.time.Duration 9 | import java.util 10 | import java.util.Properties 11 | import java.util.concurrent.atomic.AtomicBoolean 12 | import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch} 13 | import scala.collection.mutable 14 | import scala.concurrent.duration.{DurationInt, FiniteDuration} 15 | import scala.jdk.CollectionConverters._ 16 | 17 | object DynamicKafkaConsumer { 18 | 19 | def apply[K, V]( 20 | settingsMap: Map[String, AnyRef], 21 | topics: Set[String], 22 | onRecord: ConsumerRecord[K, V] => Unit, 23 | onFailure: Exception => Unit, 24 | ): DynamicKafkaConsumer[K, V] = { 25 | val settings = new Properties() 26 | settings.putAll(settingsMap.asJava) 27 | new DynamicKafkaConsumer[K, V](settings, topics, onRecord, onFailure) 28 | } 29 | private val initializationTimeout = 90.seconds 30 | private val defaultAssignTimeout = 60.seconds 31 | } 32 | 33 | final class DynamicKafkaConsumer[K, V] private ( 34 | settings: Properties, 35 | topics: Set[String], 36 | onRecord: ConsumerRecord[K, V] => Unit, 37 | onFailure: Exception => Unit, 38 | ) extends Runnable with AutoCloseable with StrictLogging { 39 | 40 | private val topicsQueue: java.util.Queue[(String, CountDownLatch)] = new ConcurrentLinkedQueue[(String, CountDownLatch)]() 41 | topicsQueue.addAll(topics.map((_, new CountDownLatch(0))).asJava) 42 | 43 | private val running: AtomicBoolean = new AtomicBoolean(true) 44 | private val consumer: KafkaConsumer[K, V] = new KafkaConsumer[K, V](settings) 45 | private val initLatch: CountDownLatch = if (this.topicsQueue.isEmpty) new CountDownLatch(1) else new CountDownLatch(0) 46 | 47 | def addTopicForSubscription( 48 | newTopic: String, 49 | assignTimeout: FiniteDuration = DynamicKafkaConsumer.defaultAssignTimeout, 50 | ): Unit = { 51 | val latch = new CountDownLatch(1) 52 | this.topicsQueue.add(newTopic, latch) 53 | if (initLatch.getCount > 0) { // need for staring processing loop 54 | initLatch.countDown() 55 | } 56 | latch.await(assignTimeout.length, assignTimeout.unit) 57 | } 58 | 59 | private def getTopicsForSubscription: Set[(String, CountDownLatch)] = { 60 | if (!this.topicsQueue.isEmpty) { 61 | val currentSubscription = this.consumer.subscription() 62 | val forSubscribe = mutable.Set.empty[(String, CountDownLatch)] 63 | while (!this.topicsQueue.isEmpty) { 64 | val (topic, latch) = this.topicsQueue.poll() 65 | if (currentSubscription.contains(topic)) { 66 | latch.countDown() 67 | } else { 68 | forSubscribe.add(topic, latch) 69 | } 70 | } 71 | if (forSubscribe.isEmpty) 72 | return Set.empty 73 | 74 | forSubscribe.addAll(currentSubscription.asScala.map((_, new CountDownLatch(1)))) 75 | return forSubscribe.toSet 76 | } 77 | Set.empty 78 | } 79 | 80 | private def subscribeTopics(forSubscribe: Set[(String, CountDownLatch)]): Unit = { 81 | if (forSubscribe.nonEmpty) { 82 | val (topics, latches) = 83 | forSubscribe.foldLeft((mutable.Set.empty[String], mutable.Set.empty[CountDownLatch]))((result, tl) => { 84 | val (ts, ls) = result 85 | val (topic, latch) = tl 86 | ts.add(topic) 87 | ls.add(latch) 88 | result 89 | }) 90 | this.consumer.subscribe( 91 | topics.asJava, 92 | new ConsumerRebalanceListener { 93 | override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { 94 | logger.debug(s"revoked partitions $partitions") 95 | } 96 | 97 | override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { 98 | logger.debug(s"assigned partitions $partitions") 99 | latches.foreach(_.countDown()) 100 | } 101 | }, 102 | ) 103 | } 104 | } 105 | 106 | override def run(): Unit = { 107 | try { 108 | val timeout = DynamicKafkaConsumer.initializationTimeout 109 | this.initLatch.await(timeout.length, timeout.unit) 110 | subscribeTopics(getTopicsForSubscription) 111 | while (running.get) { 112 | val records = this.consumer.poll(Duration.ofMillis(1000)) 113 | records.forEach(record => 114 | try this.onRecord(record) 115 | catch { 116 | case e: Exception => 117 | this.onFailure(e) 118 | }, 119 | ) 120 | subscribeTopics(getTopicsForSubscription) 121 | } 122 | } catch { 123 | case e: WakeupException => 124 | // Ignore exception if closing 125 | // rethrow when someone call wakeup while it is working 126 | if (running.get) throw e 127 | case e: Exception => 128 | // unexpected exception 129 | throw new RuntimeException(e) 130 | } finally { 131 | this.topicsQueue.clear() 132 | consumer.close() 133 | } 134 | } 135 | 136 | override def close(): Unit = { 137 | this.running.set(false) 138 | if (this.initLatch.getCount > 0) { 139 | this.initLatch.countDown() 140 | } 141 | this.consumer.wakeup() 142 | } 143 | } 144 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/KafkaDsl.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi; 2 | 3 | import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; 4 | import io.gatling.core.check.Check; 5 | import io.gatling.core.check.CheckBuilder; 6 | import io.gatling.core.check.CheckMaterializer; 7 | import org.apache.avro.generic.GenericRecord; 8 | import org.apache.kafka.common.serialization.Deserializer; 9 | import org.apache.kafka.common.serialization.Serializer; 10 | import org.apache.kafka.common.utils.Bytes; 11 | import org.galaxio.gatling.kafka.javaapi.checks.KafkaCheckType; 12 | import org.galaxio.gatling.kafka.javaapi.checks.KafkaChecks; 13 | import org.galaxio.gatling.kafka.javaapi.protocol.KafkaProtocolBuilderBase; 14 | import org.galaxio.gatling.kafka.javaapi.request.builder.KafkaRequestBuilderBase; 15 | import org.galaxio.gatling.kafka.javaapi.request.expressions.Builders.*; 16 | import org.galaxio.gatling.kafka.javaapi.request.expressions.ExpressionBuilder; 17 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression; 18 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage; 19 | 20 | import java.nio.ByteBuffer; 21 | import java.util.function.Function; 22 | 23 | import static io.gatling.javaapi.core.internal.Expressions.toStringExpression; 24 | 25 | public final class KafkaDsl { 26 | 27 | public static JExpression cf(T t) { 28 | return i -> t; 29 | } 30 | 31 | public static ExpressionBuilder stringExp(JExpression f) { 32 | return new StringExpressionBuilder(f); 33 | } 34 | 35 | public static ExpressionBuilder stringExp(String v) { 36 | return stringExp(cf(v)); 37 | } 38 | 39 | public static ExpressionBuilder floatExp(JExpression f) { 40 | return new FloatExpressionBuilder(f); 41 | } 42 | 43 | public static ExpressionBuilder floatExp(Float v) { 44 | return floatExp(cf(v)); 45 | } 46 | 47 | public static ExpressionBuilder doubleExp(JExpression f) { 48 | return new DoubleExpressionBuilder(f); 49 | } 50 | 51 | public static ExpressionBuilder doubleExp(Double v) { 52 | return doubleExp(cf(v)); 53 | } 54 | 55 | public static ExpressionBuilder shortExp(JExpression f) { 56 | return new ShortExpressionBuilder(f); 57 | } 58 | 59 | public static ExpressionBuilder shortExp(Short v) { 60 | return shortExp(cf(v)); 61 | } 62 | 63 | public static ExpressionBuilder integerExp(JExpression f) { 64 | return new IntegerExpressionBuilder(f); 65 | } 66 | 67 | public static ExpressionBuilder integerExp(Integer v) { 68 | return integerExp(cf(v)); 69 | } 70 | 71 | public static ExpressionBuilder longExp(JExpression f) { 72 | return new LongExpressionBuilder(f); 73 | } 74 | 75 | public static ExpressionBuilder longExp(Long v) { 76 | return longExp(cf(v)); 77 | } 78 | 79 | public static ExpressionBuilder byteBufferExp(JExpression f) { 80 | return new ByteBufferExpressionBuilder(f); 81 | } 82 | 83 | public static ExpressionBuilder byteBufferExp(ByteBuffer v) { 84 | return byteBufferExp(cf(v)); 85 | } 86 | 87 | public static ExpressionBuilder byteArrayExp(byte[] v) { 88 | return byteArrayExp(cf(v)); 89 | } 90 | 91 | public static ExpressionBuilder byteArrayExp(JExpression f) { 92 | return new ByteArrayExpressionBuilder(f); 93 | } 94 | 95 | public static ExpressionBuilder bytesExp(JExpression f) { 96 | return new BytesExpressionBuilder(f); 97 | } 98 | 99 | public static ExpressionBuilder bytesExp(Bytes v) { 100 | return bytesExp(cf(v)); 101 | } 102 | 103 | public static AvroExpressionBuilder avro(Object o, SchemaRegistryClient client) { 104 | return avro(cf(o), client); 105 | } 106 | 107 | public static AvroExpressionBuilder avro(JExpression s, SchemaRegistryClient client) { 108 | return new AvroExpressionBuilder(s, client); 109 | } 110 | 111 | public static AvroExpressionBuilder avro(JExpression s, Serializer ser, Deserializer deser) { 112 | return new AvroExpressionBuilder(s, ser, deser); 113 | } 114 | 115 | public static KafkaProtocolBuilderBase kafka() { 116 | return new KafkaProtocolBuilderBase(); 117 | } 118 | 119 | public static KafkaRequestBuilderBase kafka(String requestName) { 120 | return new KafkaRequestBuilderBase(org.galaxio.gatling.kafka.Predef.kafka(toStringExpression(requestName)), requestName); 121 | } 122 | 123 | 124 | public static io.gatling.javaapi.core.CheckBuilder simpleCheck(Function f) { 125 | return new io.gatling.javaapi.core.CheckBuilder() { 126 | @Override 127 | @SuppressWarnings("rawtypes") 128 | public CheckBuilder asScala() { 129 | return new CheckBuilder() { 130 | @Override 131 | public Check build(CheckMaterializer materializer) { 132 | return new KafkaChecks.SimpleChecksScala().simpleCheck(f::apply); 133 | 134 | } 135 | }; 136 | } 137 | 138 | @Override 139 | public CheckType type() { 140 | return KafkaCheckType.Simple; 141 | } 142 | }; 143 | } 144 | 145 | public static CheckBuilder.Find avroBody() { 146 | return new KafkaChecks.SimpleChecksScala().avroBody(org.galaxio.gatling.kafka.javaapi.checks.KafkaChecks.avroSerde()); 147 | } 148 | 149 | } 150 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Gatling Kafka Plugin 2 | 3 | ![Build](https://github.com/galax-io/gatling-kafka-plugin/workflows/Build/badge.svg) [![Maven Central](https://img.shields.io/maven-central/v/org.galaxio/gatling-kafka-plugin_2.13.svg?color=success)](https://search.maven.org/search?q=org.galaxio.gatling-kafka) [![Scala Steward badge](https://img.shields.io/badge/Scala_Steward-helping-blue.svg?style=flat&logo=data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAA4AAAAQCAMAAAARSr4IAAAAVFBMVEUAAACHjojlOy5NWlrKzcYRKjGFjIbp293YycuLa3pYY2LSqql4f3pCUFTgSjNodYRmcXUsPD/NTTbjRS+2jomhgnzNc223cGvZS0HaSD0XLjbaSjElhIr+AAAAAXRSTlMAQObYZgAAAHlJREFUCNdNyosOwyAIhWHAQS1Vt7a77/3fcxxdmv0xwmckutAR1nkm4ggbyEcg/wWmlGLDAA3oL50xi6fk5ffZ3E2E3QfZDCcCN2YtbEWZt+Drc6u6rlqv7Uk0LdKqqr5rk2UCRXOk0vmQKGfc94nOJyQjouF9H/wCc9gECEYfONoAAAAASUVORK5CYII=)](https://scala-steward.org) 4 | [![codecov.io](https://codecov.io/github/galax-io/gatling-kafka-plugin/coverage.svg?branch=master)](https://codecov.io/github/galax-io/gatling-kafka-plugin?branch=master) 5 | 6 | # Introduction 7 | 8 | Plugin to support Kafka in Gatling (3.9.x)+ 9 | 10 | | Gatling Version | Kafka Plugin Version | Notable Changes | 11 | |-----------------|-----------------------|-------------------------------------------------------------------------------------------------------------| 12 | | 3.9.x - 3.11.x | Releases up to 0.15.1 | May require additional akka dependency in your dependencies - e.g. com.typesafe.akka:akka-actor_2.13:2.6.20 | 13 | | 3.13.x | Releases after 0.15.1 | Akka dependency no longer required | 14 | 15 | # Usage 16 | 17 | ### Getting Started 18 | 19 | This plugin is currently available for Scala 2.13 / Java 17 / Kotlin. 20 | To use it, you should include it as a dependency in your gatling project with your tests. 21 | To include: 22 | 23 | ### Scala 24 | 25 | ```scala 26 | libraryDependencies += "org.galaxio" %% "gatling-kafka-plugin" % % Test 27 | ``` 28 | 29 | ### Java 30 | 31 | Add this to your dependencies block in build.gradle: 32 | 33 | ```java 34 | gatling "org.galaxio:gatling-kafka-plugin_2.13:" 35 | ``` 36 | 37 | ### Kotlin 38 | 39 | Add this to your dependencies block in build.gradle: 40 | 41 | ```kotlin 42 | gatling("org.galaxio:gatling-kafka-plugin_2.13:") 43 | ``` 44 | 45 | ## Example Scenarios 46 | 47 | ### Scala 48 | 49 | Examples [here](src/test/scala/org/galaxio/gatling/kafka/examples) 50 | 51 | ### Java 52 | 53 | Examples [here](src/test/java/org/galaxio/gatling/kafka/javaapi/examples) 54 | 55 | ### Kotlin 56 | 57 | Examples [here](src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples) 58 | 59 | ## Download and create Avro schema 60 | 61 | Avro schema is downloaded using the 62 | plugin [sbt-schema-registry-plugin](https://github.com/galax-io/sbt-schema-registry-plugin) 63 | and for that you need to configure schemas and url in `build.sbt` and run the command: 64 | 65 | ```bash 66 | sbt schemaRegistryDownload 67 | ``` 68 | 69 | To create java classes you should add use capabilities, that provide plugin [sbt-avro](https://github.com/sbt/sbt-avro). 70 | This plugin is included in project and will do all needed for creating java classes in compile stage. 71 | To run you should create scala object in root project directory and type `sbt run`. 72 | 73 | ### Example download avro-schema 74 | 75 | Example [here](https://github.com/galax-io/gatling-kafka-plugin/tree/master/src/test/scala/org/galaxio/gatling/kafka/examples) 76 | 77 | ## Avro support in Request-Reply 78 | 79 | ### Scala 80 | 81 | To use avro messages as payload in key or value, you must: 82 | 83 | - define implicit for schema registry url: 84 | 85 | ```scala 86 | implicit val schemaRegUrl: String = "http://localhost:9094" 87 | ``` 88 | 89 | - or define serde for your class: 90 | 91 | ```scala 92 | val ser = 93 | new KafkaAvroSerializer( 94 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 95 | ) 96 | 97 | val de = 98 | new KafkaAvroDeserializer( 99 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 100 | ) 101 | 102 | implicit val serdeClass: Serde[MyAvroClass] = new Serde[MyAvroClass] { 103 | override def serializer(): Serializer[MyAvroClass] = ser.asInstanceOf[Serializer[MyAvroClass]] 104 | 105 | override def deserializer(): Deserializer[MyAvroClass] = de.asInstanceOf[Deserializer[MyAvroClass]] 106 | } 107 | ``` 108 | 109 | ### Java 110 | 111 | To use avro messages as payload in key or value, you must define serde for your class: 112 | 113 | ```java 114 | public static Serializer ser = (Serializer) new KafkaAvroSerializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 115 | public static Deserializer de = (Deserializer) new KafkaAvroDeserializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 116 | ``` 117 | 118 | ### Kotlin 119 | 120 | To use avro messages as payload in key or value, you must define serde for your class: 121 | 122 | ```kotlin 123 | val ser = KafkaAvroSerializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Serializer 124 | val de = KafkaAvroDeserializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Deserializer 125 | ``` 126 | 127 | ### Example usage Avro in Request-Reply 128 | 129 | Example [scala](src/test/scala/org/galaxio/gatling/kafka/examples/AvroClassWithRequestReplySimulation.scala) 130 | 131 | Example [java](src/test/java/org/galaxio/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.java) 132 | 133 | Example [kotlin](src/test/kotlin/org/galaxio/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.kt) 134 | 135 | 136 | 137 | ### Build this plugin 138 | 139 | When contributing to this project - check your build with: 140 | 141 | ```shell 142 | sbt clean scalafmtCheckAll scalafmtSbtCheck compile coverage "Gatling / testOnly org.galaxio.gatling.kafka.examples.KafkaGatlingTest" test coverageOff 143 | ``` 144 | 145 | Reference to [ci.yml](/.github/workflows/ci.yml) -------------------------------------------------------------------------------- /src/main/scala/org/galaxio/gatling/kafka/client/KafkaMessageTracker.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.client 2 | 3 | import io.gatling.commons.stats.{KO, OK, Status} 4 | import io.gatling.commons.util.Clock 5 | import io.gatling.commons.validation.Failure 6 | import io.gatling.core.action.Action 7 | import io.gatling.core.actor.{Actor, Behavior} 8 | import io.gatling.core.check.Check 9 | import io.gatling.core.session.Session 10 | import io.gatling.core.stats.StatsEngine 11 | import org.galaxio.gatling.kafka.client.KafkaMessageTracker._ 12 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol.KafkaMatcher 13 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 14 | import org.galaxio.gatling.kafka.{KafkaCheck, KafkaLogging} 15 | 16 | import scala.collection.mutable 17 | import scala.concurrent.duration.DurationInt 18 | 19 | object KafkaMessageTracker { 20 | 21 | def actor[K, V]( 22 | actorName: String, 23 | statsEngine: StatsEngine, 24 | clock: Clock, 25 | messageMatcher: KafkaMatcher, 26 | responseTransformer: Option[KafkaProtocolMessage => KafkaProtocolMessage], 27 | ): Actor[TrackerMessage] = 28 | new KafkaMessageTracker[K, V](actorName, statsEngine, clock, messageMatcher, responseTransformer) 29 | 30 | sealed trait TrackerMessage 31 | 32 | final case class MessagePublished( 33 | matchId: Array[Byte], 34 | sentTimestamp: Long, 35 | replyTimeout: Long, 36 | checks: List[KafkaCheck], 37 | session: Session, 38 | next: Action, 39 | requestName: String, 40 | ) extends TrackerMessage 41 | 42 | final case class MessageConsumed( 43 | received: Long, 44 | message: KafkaProtocolMessage, 45 | ) extends TrackerMessage 46 | 47 | private final case object TimeoutScan extends TrackerMessage 48 | 49 | private def makeKeyForSentMessages(m: Array[Byte]): String = 50 | Option(m).map(java.util.Base64.getEncoder.encodeToString(_)).getOrElse("") 51 | } 52 | 53 | /** Actor to record request and response Kafka Events, publishing data to the Gatling core DataWriter 54 | */ 55 | class KafkaMessageTracker[K, V]( 56 | name: String, 57 | statsEngine: StatsEngine, 58 | clock: Clock, 59 | messageMatcher: KafkaMatcher, 60 | responseTransformer: Option[KafkaProtocolMessage => KafkaProtocolMessage], 61 | ) extends Actor[TrackerMessage](name) with KafkaLogging { 62 | 63 | private val sentMessages = mutable.HashMap.empty[String, MessagePublished] 64 | private val timedOutMessages = mutable.ArrayBuffer.empty[MessagePublished] 65 | private var periodicTimeoutScanTriggered = false 66 | 67 | private def triggerPeriodicTimeoutScan(): Unit = 68 | if (!periodicTimeoutScanTriggered) { 69 | periodicTimeoutScanTriggered = true 70 | scheduler.scheduleAtFixedRate(1000.millis) { 71 | self ! TimeoutScan 72 | } 73 | } 74 | 75 | override def init(): Behavior[TrackerMessage] = { 76 | // message was sent; add the timestamps to the map 77 | case messageSent: MessagePublished => 78 | val key = makeKeyForSentMessages(messageSent.matchId) 79 | logger.debug("Published with MatchId: {} Tracking Key: {}", new String(messageSent.matchId), key) 80 | sentMessages += key -> messageSent 81 | if (messageSent.replyTimeout > 0) { 82 | triggerPeriodicTimeoutScan() 83 | } 84 | stay 85 | 86 | // message was received; publish stats and remove from the map 87 | case MessageConsumed(receivedTimestamp, forTransformMessage) => 88 | val message = responseTransformer.map(_(forTransformMessage)).getOrElse(forTransformMessage) 89 | if (messageMatcher.responseMatch(message) == null) { 90 | logger.error("no messageMatcher key for read message {}", message.key) 91 | } else { 92 | if (message.key == null || message.value == null) { 93 | logger.warn(" --- received message with null key or value") 94 | } else { 95 | logger.trace(" --- received {} {}", message.key, message.value) 96 | } 97 | 98 | val replyId = messageMatcher.responseMatch(message) 99 | val messageKey = if (message.key == null) "null" else new String(message.key) 100 | logMessage(s"Record received key=$messageKey", message) 101 | // if key is missing, message was already acked and is a dup, or request timeout 102 | val key = makeKeyForSentMessages(replyId) 103 | logger.debug( 104 | "Received with MatchId: {} Tracking Key: {}, producerTopic: {}, consumerTopic: {}", 105 | new String(replyId), 106 | key, 107 | message.producerTopic, 108 | message.consumerTopic, 109 | ) 110 | sentMessages.remove(key).foreach { case MessagePublished(_, sentTimestamp, _, checks, session, next, requestName) => 111 | processMessage(session, sentTimestamp, receivedTimestamp, checks, message, next, requestName) 112 | } 113 | } 114 | stay 115 | 116 | case TimeoutScan => 117 | val now = clock.nowMillis 118 | sentMessages.valuesIterator.foreach { messagePublished => 119 | val replyTimeout = messagePublished.replyTimeout 120 | if (replyTimeout > 0 && (now - messagePublished.sentTimestamp) > replyTimeout) { 121 | timedOutMessages += messagePublished 122 | } 123 | } 124 | for (MessagePublished(matchId, sentTimestamp, receivedTimeout, _, session, next, requestName) <- timedOutMessages) { 125 | val matchKey = makeKeyForSentMessages(matchId) 126 | logger.warn("Did not receive match for {} - key: {} after {}ms", new String(matchId), matchKey, receivedTimeout) 127 | sentMessages.remove(matchKey) 128 | executeNext( 129 | session.markAsFailed, 130 | sentTimestamp, 131 | now, 132 | KO, 133 | next, 134 | requestName, 135 | None, 136 | Some(s"Reply timeout after $receivedTimeout ms"), 137 | ) 138 | } 139 | timedOutMessages.clear() 140 | stay 141 | } 142 | 143 | private def executeNext( 144 | session: Session, 145 | sentTimestamp: Long, 146 | receivedTimestamp: Long, 147 | status: Status, 148 | next: Action, 149 | requestName: String, 150 | responseCode: Option[String], 151 | message: Option[String], 152 | ): Unit = { 153 | statsEngine.logResponse( 154 | session.scenario, 155 | session.groups, 156 | requestName, 157 | sentTimestamp, 158 | receivedTimestamp, 159 | status, 160 | responseCode, 161 | message, 162 | ) 163 | next ! session.logGroupRequestTimings(sentTimestamp, receivedTimestamp) 164 | } 165 | 166 | /** Processes a matched message 167 | */ 168 | private def processMessage( 169 | session: Session, 170 | sentTimestamp: Long, 171 | receivedTimestamp: Long, 172 | checks: List[KafkaCheck], 173 | message: KafkaProtocolMessage, 174 | next: Action, 175 | requestName: String, 176 | ): Unit = { 177 | val (newSession, error) = Check.check(message, session, checks) 178 | error match { 179 | case Some(Failure(errorMessage)) => 180 | executeNext( 181 | newSession.markAsFailed, 182 | sentTimestamp, 183 | receivedTimestamp, 184 | KO, 185 | next, 186 | requestName, 187 | message.responseCode, 188 | Some(errorMessage), 189 | ) 190 | case _ => 191 | executeNext(newSession, sentTimestamp, receivedTimestamp, OK, next, requestName, None, None) 192 | } 193 | } 194 | } 195 | -------------------------------------------------------------------------------- /src/test/scala/org/galaxio/gatling/kafka/examples/KafkaGatlingTest.scala: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.examples 2 | 3 | import com.sksamuel.avro4s._ 4 | import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient 5 | import io.confluent.kafka.serializers.{KafkaAvroDeserializer, KafkaAvroSerializer} 6 | import io.gatling.core.Predef._ 7 | import io.gatling.core.feeder.Feeder 8 | import io.gatling.core.structure.ScenarioBuilder 9 | import org.apache.kafka.clients.producer.ProducerConfig 10 | import org.galaxio.gatling.kafka.Predef._ 11 | import org.galaxio.gatling.kafka.avro4s._ 12 | import org.apache.kafka.common.header.Headers 13 | import org.apache.kafka.common.header.internals.RecordHeaders 14 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer} 15 | import org.galaxio.gatling.kafka.protocol.KafkaProtocol 16 | import org.galaxio.gatling.kafka.request.KafkaProtocolMessage 17 | 18 | import scala.concurrent.duration.DurationInt 19 | 20 | class KafkaGatlingTest extends Simulation { 21 | 22 | case class Ingredient(name: String, sugar: Double, fat: Double) 23 | 24 | val kafkaConf: KafkaProtocol = kafka 25 | .properties( 26 | Map( 27 | ProducerConfig.ACKS_CONFIG -> "1", 28 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 29 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 30 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 31 | ), 32 | ) 33 | 34 | val kafkaConfwoKey: KafkaProtocol = kafka 35 | .properties( 36 | Map( 37 | ProducerConfig.ACKS_CONFIG -> "1", 38 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 39 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 40 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 41 | ), 42 | ) 43 | 44 | val kafkaConfBytes: KafkaProtocol = kafka 45 | .properties( 46 | Map( 47 | ProducerConfig.ACKS_CONFIG -> "1", 48 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 49 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 50 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 51 | ), 52 | ) 53 | 54 | val kafkaProtocolRRString: KafkaProtocol = kafka 55 | .producerSettings( 56 | ProducerConfig.ACKS_CONFIG -> "1", 57 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 58 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 59 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 60 | ) 61 | .consumeSettings( 62 | "bootstrap.servers" -> "localhost:9093", 63 | ) 64 | .withDefaultTimeout 65 | 66 | val kafkaProtocolRRBytes: KafkaProtocol = kafka 67 | .producerSettings( 68 | ProducerConfig.ACKS_CONFIG -> "1", 69 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 70 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 71 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 72 | ) 73 | .consumeSettings( 74 | "bootstrap.servers" -> "localhost:9093", 75 | ) 76 | .timeout(5.seconds) 77 | .matchByValue 78 | 79 | val kafkaProtocolRRBytes2: KafkaProtocol = kafka 80 | .producerSettings( 81 | ProducerConfig.ACKS_CONFIG -> "1", 82 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 83 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 84 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 85 | ) 86 | .consumeSettings( 87 | "bootstrap.servers" -> "localhost:9093", 88 | ) 89 | .timeout(1.seconds) 90 | .matchByValue 91 | 92 | val kafkaAvro4sConf: KafkaProtocol = kafka 93 | .properties( 94 | Map( 95 | ProducerConfig.ACKS_CONFIG -> "1", 96 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 97 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 98 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 99 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 100 | "schema.registry.url" -> "http://localhost:9094", 101 | ), 102 | ) 103 | 104 | def matchByOwnVal(message: KafkaProtocolMessage): Array[Byte] = { 105 | message.key 106 | } 107 | 108 | val kafkaProtocolRRAvro: KafkaProtocol = kafka 109 | .producerSettings( 110 | ProducerConfig.ACKS_CONFIG -> "1", 111 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 112 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 113 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 114 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 115 | "schema.registry.url" -> "http://localhost:9094", 116 | ) 117 | .consumeSettings( 118 | "bootstrap.servers" -> "localhost:9093", 119 | ) 120 | .timeout(7.seconds) 121 | .matchByMessage(matchByOwnVal) 122 | 123 | val scnRR: ScenarioBuilder = scenario("RequestReply String") 124 | .exec( 125 | kafka("Request Reply String").requestReply 126 | .requestTopic("myTopic1") 127 | .replyTopic("test.t1") 128 | .send[String, String]("testCheckJson", """{ "m": "dkf" }""") 129 | .check(jsonPath("$.m").is("dkf")), 130 | ) 131 | 132 | val scnwokey: ScenarioBuilder = scenario("Request String without key") 133 | .exec( 134 | kafka("Request String") 135 | .topic("myTopic3") 136 | .send[String]("foo"), 137 | ) 138 | .exec( 139 | kafka("Request String With null key") 140 | .topic("myTopic3") 141 | .send[Int, String](null, "nullkey"), 142 | ) 143 | 144 | val scn: ScenarioBuilder = scenario("Request String") 145 | .exec(kafka("Request String 2").topic("test.t1").send[String, String]("testCheckJson", """{ "m": "dkf" }""")) 146 | 147 | val scn2: ScenarioBuilder = scenario("Request Byte") 148 | .exec( 149 | kafka("Request Byte") 150 | .topic("test.t2") 151 | .send[Array[Byte], Array[Byte]]("key".getBytes(), "tstBytes".getBytes()), 152 | ) 153 | 154 | val scnRR2: ScenarioBuilder = scenario("RequestReply Bytes") 155 | .exec( 156 | kafka("Request Reply Bytes").requestReply 157 | .requestTopic("myTopic2") 158 | .replyTopic("test.t2") 159 | .send[Array[Byte], Array[Byte]]("test".getBytes(), "tstBytes".getBytes()) 160 | .check(bodyBytes.is("tstBytes".getBytes()).saveAs("bodyInfo")), 161 | ) 162 | 163 | val scnAvro4s: ScenarioBuilder = scenario("Request Avro4s") 164 | .exec( 165 | kafka("Request Simple Avro4s") 166 | .topic("test.t3") 167 | .send[Ingredient](Ingredient("Cheese", 1d, 50d)), 168 | ) 169 | .exec( 170 | kafka("Request Avro4s") 171 | .topic("test.t3") 172 | .send[String, Ingredient]("key4s", Ingredient("Cheese", 0d, 70d)), 173 | ) 174 | 175 | val scnRRwo: ScenarioBuilder = scenario("RequestReply w/o answer") 176 | .exec( 177 | kafka("Request Reply Bytes wo").requestReply 178 | .requestTopic("myTopic2") 179 | .replyTopic("test.t2") 180 | .send[Array[Byte], Array[Byte]]("testWO".getBytes(), "tstBytesWO".getBytes()), 181 | ) 182 | 183 | setUp( 184 | scnRR.inject(atOnceUsers(1)).protocols(kafkaProtocolRRString), 185 | scn.inject(nothingFor(1), atOnceUsers(1)).protocols(kafkaConf), 186 | scnRR2.inject(atOnceUsers(1)).protocols(kafkaProtocolRRBytes), 187 | scn2.inject(nothingFor(2), atOnceUsers(1)).protocols(kafkaConfBytes), 188 | scnAvro4s.inject(atOnceUsers(1)).protocols(kafkaAvro4sConf), 189 | scnRRwo.inject(atOnceUsers(1)).protocols(kafkaProtocolRRBytes2), 190 | scnwokey.inject(nothingFor(1), atOnceUsers(1)).protocols(kafkaConfwoKey), 191 | ).assertions( 192 | global.failedRequests.percent.lt(15.0), 193 | ).maxDuration(120.seconds) 194 | 195 | } 196 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | name: Continuous Integration 2 | 3 | on: 4 | pull_request: 5 | branches: ['**'] 6 | push: 7 | branches: ['**'] 8 | tags: ['v*'] 9 | 10 | permissions: 11 | contents: write 12 | actions: read 13 | checks: write 14 | pull-requests: write 15 | packages: write 16 | 17 | concurrency: 18 | group: ci-${{ github.workflow }}-${{ github.ref }} 19 | cancel-in-progress: true 20 | 21 | env: 22 | SBT_OPTS: -Dsbt.color=true -Dsbt.log.noformat=false -Dsbt.supershell=false 23 | JAVA_OPTS: -Xms2G -Xmx2G -Xss4M -XX:+UseG1GC 24 | JVM_OPTS: -Xms2G -Xmx2G -Xss4M -XX:+UseG1GC 25 | FORCE_COLOR: "1" 26 | TERM: xterm-256color 27 | 28 | jobs: 29 | test: 30 | name: Lint, Compile & Test 31 | runs-on: ubuntu-24.04 32 | services: 33 | zookeeper: 34 | image: wurstmeister/zookeeper 35 | env: 36 | ZOO_MY_ID: "1" 37 | ZOO_PORT: "2181" 38 | ZOO_SERVERS: server.1=zoo1:2888:3888 39 | ports: 40 | - '2181:2181' 41 | kafka: 42 | image: wurstmeister/kafka:2.13-2.8.1 43 | env: 44 | KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 45 | KAFKA_ADVERTISED_HOST_NAME: kafka 46 | KAFKA_LISTENERS: BROKER://:9092,EXTERNAL://:9093 47 | KAFKA_ADVERTISED_LISTENERS: BROKER://kafka:9092,EXTERNAL://localhost:9093 48 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: BROKER:PLAINTEXT,EXTERNAL:PLAINTEXT 49 | KAFKA_INTER_BROKER_LISTENER_NAME: BROKER 50 | KAFKA_BROKER_ID: "1" 51 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: "1" 52 | KAFKA_CREATE_TOPICS: "myTopic1:1:1, test.t1:1:1, myTopic2:1:1, test.t2:1:1, myTopic3:1:1, test.t3:1:1" 53 | ports: 54 | - '9092:9092' 55 | - '9093:9093' 56 | schema-registry: 57 | image: confluentinc/cp-schema-registry:7.2.1 58 | env: 59 | SCHEMA_REGISTRY_HOST_NAME: schema-registry 60 | SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: 'kafka:9092,localhost:9093' 61 | SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:9094 62 | ports: 63 | - '9094:9094' 64 | 65 | steps: 66 | - name: Checkout 67 | uses: actions/checkout@v5 68 | with: 69 | fetch-depth: 0 70 | 71 | - name: Setup Java (Temurin 17) with sbt cache 72 | uses: actions/setup-java@v4 73 | with: 74 | distribution: temurin 75 | java-version: '17' 76 | cache: sbt 77 | 78 | - uses: sbt/setup-sbt@v1 79 | 80 | - name: Cache Coursier 81 | uses: coursier/cache-action@v6 82 | 83 | - name: Cache Ivy/SBT 84 | uses: actions/cache@v4 85 | with: 86 | path: | 87 | ~/.ivy2/cache 88 | ~/.sbt 89 | ~/.cache/coursier 90 | key: sbt-${{ runner.os }}-${{ hashFiles('**/*.sbt') }}-${{ hashFiles('project/**') }} 91 | restore-keys: sbt-${{ runner.os }}- 92 | 93 | - name: Check Formatting 94 | run: sbt scalafmtCheckAll scalafmtSbtCheck 95 | 96 | - name: Compile 97 | run: sbt clean compile 98 | 99 | - name: Test (Gatling targeted) 100 | run: sbt coverage "Gatling / testOnly org.galaxio.gatling.kafka.examples.KafkaGatlingTest" "Gatling / testOnly org.galaxio.gatling.kafka.examples.KafkaJavaapiMethodsGatlingTest" test coverageOff coverageReport 101 | 102 | - name: Upload coverage reports to Codecov 103 | uses: codecov/codecov-action@v4 104 | 105 | release: 106 | name: Release (tag-driven) 107 | needs: [test] 108 | runs-on: ubuntu-24.04 109 | if: github.event_name == 'push' && (github.ref == 'refs/heads/main' || startsWith(github.ref, 'refs/tags/v')) 110 | permissions: 111 | contents: write 112 | steps: 113 | - name: Checkout 114 | uses: actions/checkout@v5 115 | with: 116 | fetch-depth: 0 117 | - name: Setup Java (Temurin 17) with sbt cache 118 | uses: actions/setup-java@v4 119 | with: 120 | distribution: temurin 121 | java-version: '17' 122 | cache: sbt 123 | - name: Setup sbt 124 | uses: sbt/setup-sbt@v1 125 | - name: Cache Coursier 126 | uses: coursier/cache-action@v6 127 | - name: Cache Ivy/SBT 128 | uses: actions/cache@v4 129 | with: 130 | path: | 131 | ~/.ivy2/cache 132 | ~/.sbt 133 | ~/.cache/coursier 134 | key: sbt-${{ runner.os }}-${{ hashFiles('**/*.sbt') }}-${{ hashFiles('project/**') }} 135 | restore-keys: sbt-${{ runner.os }}- 136 | - name: Compute next version & create tag on main 137 | id: bump 138 | if: github.ref == 'refs/heads/main' 139 | env: 140 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 141 | shell: bash 142 | run: | 143 | set -euo pipefail 144 | 145 | git fetch --tags --force --prune 146 | 147 | LAST_TAG=$(git describe --tags --abbrev=0 --match "v*" --first-parent 2>/dev/null || echo "") 148 | echo "last_tag=${LAST_TAG}" >> "$GITHUB_OUTPUT" 149 | RANGE="${LAST_TAG:+${LAST_TAG}..HEAD}" 150 | 151 | BUMP="patch" 152 | COMMITS=$(git log --pretty=format:%s ${RANGE}) 153 | if echo "$COMMITS" | grep -Eiq 'BREAKING CHANGE|!:'; then 154 | BUMP="major" 155 | elif echo "$COMMITS" | grep -Eiq '^feat(\(.+\))?:'; then 156 | BUMP="minor" 157 | fi 158 | 159 | if [ -z "$LAST_TAG" ]; then 160 | MAJOR=0; MINOR=0; PATCH=0 161 | else 162 | IFS='.' read -r MAJOR MINOR PATCH <<< "${LAST_TAG#v}" 163 | fi 164 | 165 | case "$BUMP" in 166 | major) MAJOR=$((MAJOR+1)); MINOR=0; PATCH=0 ;; 167 | minor) MINOR=$((MINOR+1)); PATCH=0 ;; 168 | patch) PATCH=$((PATCH+1)) ;; 169 | esac 170 | 171 | NEXT_TAG="v${MAJOR}.${MINOR}.${PATCH}" 172 | 173 | if git rev-parse -q --verify "refs/tags/${NEXT_TAG}" >/dev/null; then 174 | TAG_COMMIT=$(git rev-list -n 1 "${NEXT_TAG}") 175 | HEAD_COMMIT=$(git rev-parse HEAD) 176 | if [ "${TAG_COMMIT}" = "${HEAD_COMMIT}" ]; then 177 | SKIP_TAG_CREATE=1 178 | else 179 | while git rev-parse -q --verify "refs/tags/${NEXT_TAG}" >/dev/null; do 180 | IFS='.' read -r MAJOR MINOR PATCH <<< "${NEXT_TAG#v}" 181 | PATCH=$((PATCH+1)) 182 | NEXT_TAG="v${MAJOR}.${MINOR}.${PATCH}" 183 | done 184 | fi 185 | fi 186 | 187 | echo "bump=$BUMP" >> "$GITHUB_OUTPUT" 188 | echo "tag=$NEXT_TAG" >> "$GITHUB_OUTPUT" 189 | 190 | git config user.name "github-actions[bot]" 191 | git config user.email "41898282+github-actions[bot]@users.noreply.github.com" 192 | git remote set-url origin "https://x-access-token:${GITHUB_TOKEN}@github.com/${{ github.repository }}.git" 193 | 194 | if [ "${SKIP_TAG_CREATE:-0}" != "1" ]; then 195 | git tag -a "$NEXT_TAG" -m "Release $NEXT_TAG" 196 | git push origin "$NEXT_TAG" 197 | fi 198 | - name: Publish to Sonatype via sbt-ci-release 199 | if: startsWith(github.ref, 'refs/tags/v') || (github.ref == 'refs/heads/main' && steps.bump.outputs.tag) 200 | env: 201 | PGP_PASSPHRASE: ${{ secrets.PGP_PASSPHRASE }} 202 | PGP_SECRET: ${{ secrets.PGP_SECRET }} 203 | SONATYPE_PASSWORD: ${{ secrets.SONATYPE_PASSWORD }} 204 | SONATYPE_USERNAME: ${{ secrets.SONATYPE_USERNAME }} 205 | shell: bash 206 | run: | 207 | set -euo pipefail 208 | 209 | if [[ "${GITHUB_REF}" == refs/tags/* ]]; then 210 | TAG="${GITHUB_REF##*/}" 211 | else 212 | TAG="${{ steps.bump.outputs.tag }}" 213 | export GITHUB_REF="refs/tags/${TAG}" 214 | export GITHUB_REF_NAME="${TAG}" 215 | fi 216 | 217 | VERSION="${TAG#v}" 218 | echo "Publishing version $VERSION (tag $TAG)" 219 | 220 | if [ -n "${PGP_SECRET:-}" ]; then 221 | echo "$PGP_SECRET" | base64 --decode | gpg --batch --import 222 | fi 223 | 224 | if sbt 'show dynver' >/dev/null 2>&1; then 225 | sbt ci-release 226 | else 227 | sbt "set ThisBuild/version := \"${VERSION}\"" ci-release 228 | fi 229 | - name: Generate Changelog 230 | id: changelog 231 | uses: mikepenz/release-changelog-builder-action@v5 232 | if: startsWith(github.ref, 'refs/tags/v') || (github.ref == 'refs/heads/main' && steps.bump.outputs.tag) 233 | with: 234 | mode: "COMMIT" 235 | fromTag: ${{ github.ref == 'refs/heads/main' && steps.bump.outputs.last_tag || '' }} 236 | toTag: ${{ github.ref == 'refs/heads/main' && steps.bump.outputs.tag || '' }} 237 | configurationJson: | 238 | { 239 | "template": "# Changelog\n\n#{{CHANGELOG}}", 240 | "categories": [ 241 | {"title": "✨ Features", "labels": ["feat", "feature"]}, 242 | {"title": "🐛 Bug Fixes", "labels": ["fix", "bug"]}, 243 | {"title": "📝 Documentation", "labels": ["docs"]}, 244 | {"title": "⚡ Performance Improvements", "labels": ["perf"]}, 245 | {"title": "♻️ Refactoring", "labels": ["refactor"]}, 246 | {"title": "🧪 Tests", "labels": ["test"]}, 247 | {"title": "🧹 Chores", "labels": ["chore"]}, 248 | {"title": "📦 Dependencies", "labels": ["dependency"]}, 249 | {"title": "🚨 Breaking Changes", "labels": ["breaking"]}, 250 | {"title": "🤖 CI / Build", "labels": ["ci"]} 251 | ], 252 | "label_extractor": [ 253 | { 254 | "pattern": "^(build|chore|ci|docs|feat|fix|perf|refactor|revert|style|test){1}(\\([\\w\\-\\.]+\\))?(!)?: ([\\w ])+([\\s\\S]*)", 255 | "on_property": "title", 256 | "target": "$1" 257 | } 258 | ] 259 | } 260 | env: 261 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 262 | - name: Create GitHub Release 263 | if: startsWith(github.ref, 'refs/tags/v') || (github.ref == 'refs/heads/main' && steps.bump.outputs.tag) 264 | uses: softprops/action-gh-release@v2 265 | with: 266 | tag_name: ${{ github.ref == 'refs/heads/main' && steps.bump.outputs.tag || '' }} 267 | body: ${{ steps.changelog.outputs.changelog }} 268 | env: 269 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 270 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/OnlyPublishStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import org.apache.kafka.common.header.Headers; 4 | import org.apache.kafka.common.header.internals.RecordHeaders; 5 | import org.apache.kafka.common.serialization.Serdes; 6 | import org.apache.kafka.common.utils.Bytes; 7 | import org.galaxio.gatling.kafka.javaapi.request.expressions.ExpressionBuilder; 8 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression; 9 | import org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase; 10 | import scala.reflect.ClassTag; 11 | 12 | import java.nio.ByteBuffer; 13 | 14 | import static io.gatling.javaapi.core.internal.Expressions.javaFunctionToExpression; 15 | import static io.gatling.javaapi.core.internal.Expressions.toStaticValueExpression; 16 | import static org.galaxio.gatling.kafka.javaapi.KafkaDsl.*; 17 | 18 | public class OnlyPublishStep { 19 | private final org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase.OnlyPublishStep wrapped; 20 | 21 | public OnlyPublishStep(KafkaRequestBuilderBase.OnlyPublishStep wrapped) { 22 | this.wrapped = wrapped; 23 | } 24 | 25 | public RequestBuilder send(String key, String payload) { 26 | return send(stringExp(key), stringExp(payload)); 27 | } 28 | 29 | public RequestBuilder send(String key, String payload, Headers headers) { 30 | return send(stringExp(key), stringExp(payload), headers); 31 | } 32 | 33 | public RequestBuilder send(String key, String payload, JExpression headers) { 34 | return send(stringExp(key), stringExp(payload), headers); 35 | } 36 | 37 | public RequestBuilder send(String key, ExpressionBuilder payload) { 38 | return send(stringExp(key), payload); 39 | } 40 | 41 | public RequestBuilder send(String key, ExpressionBuilder payload, Headers headers) { 42 | return send(stringExp(key), payload, headers); 43 | } 44 | 45 | public RequestBuilder send(String key, ExpressionBuilder payload, JExpression headers) { 46 | return send(stringExp(key), payload, headers); 47 | } 48 | 49 | public RequestBuilder send(ExpressionBuilder key, String payload) { 50 | return send(key, stringExp(payload)); 51 | } 52 | 53 | public RequestBuilder send(ExpressionBuilder key, String payload, Headers headers) { 54 | return send(key, stringExp(payload), headers); 55 | } 56 | 57 | public RequestBuilder send(ExpressionBuilder key, String payload, JExpression headers) { 58 | return send(key, stringExp(payload), headers); 59 | } 60 | 61 | public RequestBuilder send(Float key, ExpressionBuilder payload) { 62 | return send(floatExp(key), payload); 63 | } 64 | 65 | public RequestBuilder send(Float key, ExpressionBuilder payload, Headers headers) { 66 | return send(floatExp(key), payload, headers); 67 | } 68 | 69 | public RequestBuilder send(Float key, ExpressionBuilder payload, JExpression headers) { 70 | return send(floatExp(key), payload, headers); 71 | } 72 | 73 | public RequestBuilder send(ExpressionBuilder key, Float payload) { 74 | return send(key, floatExp(payload)); 75 | } 76 | 77 | public RequestBuilder send(ExpressionBuilder key, Float payload, Headers headers) { 78 | return send(key, floatExp(payload), headers); 79 | } 80 | 81 | public RequestBuilder send(ExpressionBuilder key, Float payload, JExpression headers) { 82 | return send(key, floatExp(payload), headers); 83 | } 84 | 85 | public RequestBuilder send(Double key, ExpressionBuilder payload) { 86 | return send(doubleExp(key), payload); 87 | } 88 | 89 | public RequestBuilder send(Double key, ExpressionBuilder payload, Headers headers) { 90 | return send(doubleExp(key), payload, headers); 91 | } 92 | 93 | public RequestBuilder send(Double key, ExpressionBuilder payload, JExpression headers) { 94 | return send(doubleExp(key), payload, headers); 95 | } 96 | 97 | public RequestBuilder send(ExpressionBuilder key, Double payload) { 98 | return send(key, doubleExp(payload)); 99 | } 100 | 101 | public RequestBuilder send(ExpressionBuilder key, Double payload, Headers headers) { 102 | return send(key, doubleExp(payload), headers); 103 | } 104 | 105 | public RequestBuilder send(ExpressionBuilder key, Double payload, JExpression headers) { 106 | return send(key, doubleExp(payload), headers); 107 | } 108 | 109 | public RequestBuilder send(Short key, ExpressionBuilder payload) { 110 | return send(shortExp(key), payload); 111 | } 112 | 113 | public RequestBuilder send(Short key, ExpressionBuilder payload, Headers headers) { 114 | return send(shortExp(key), payload, headers); 115 | } 116 | 117 | public RequestBuilder send(Short key, ExpressionBuilder payload, JExpression headers) { 118 | return send(shortExp(key), payload, headers); 119 | } 120 | 121 | public RequestBuilder send(ExpressionBuilder key, Short payload) { 122 | return send(key, shortExp(payload)); 123 | } 124 | 125 | public RequestBuilder send(ExpressionBuilder key, Short payload, Headers headers) { 126 | return send(key, shortExp(payload), headers); 127 | } 128 | 129 | public RequestBuilder send(ExpressionBuilder key, Short payload, JExpression headers) { 130 | return send(key, shortExp(payload), headers); 131 | } 132 | 133 | public RequestBuilder send(Integer key, ExpressionBuilder payload) { 134 | return send(integerExp(key), payload); 135 | } 136 | 137 | public RequestBuilder send(Integer key, ExpressionBuilder payload, Headers headers) { 138 | return send(integerExp(key), payload, headers); 139 | } 140 | 141 | public RequestBuilder send(Integer key, ExpressionBuilder payload, JExpression headers) { 142 | return send(integerExp(key), payload, headers); 143 | } 144 | 145 | public RequestBuilder send(ExpressionBuilder key, Integer payload) { 146 | return send(key, integerExp(payload)); 147 | } 148 | 149 | public RequestBuilder send(ExpressionBuilder key, Integer payload, Headers headers) { 150 | return send(key, integerExp(payload), headers); 151 | } 152 | 153 | public RequestBuilder send(ExpressionBuilder key, Integer payload, JExpression headers) { 154 | return send(key, integerExp(payload), headers); 155 | } 156 | 157 | public RequestBuilder send(Long key, ExpressionBuilder payload) { 158 | return send(longExp(key), payload); 159 | } 160 | 161 | public RequestBuilder send(Integer key, Long payload) { 162 | return send(integerExp(key), longExp(payload)); 163 | } 164 | 165 | public RequestBuilder send(Long key, ExpressionBuilder payload, Headers headers) { 166 | return send(longExp(key), payload, headers); 167 | } 168 | 169 | public RequestBuilder send(Long key, ExpressionBuilder payload, JExpression headers) { 170 | return send(longExp(key), payload, headers); 171 | } 172 | 173 | public RequestBuilder send(ExpressionBuilder key, Long payload) { 174 | return send(key, longExp(payload)); 175 | } 176 | 177 | public RequestBuilder send(ExpressionBuilder key, Long payload, Headers headers) { 178 | return send(key, longExp(payload), headers); 179 | } 180 | 181 | public RequestBuilder send(ExpressionBuilder key, Long payload, JExpression headers) { 182 | return send(key, longExp(payload), headers); 183 | } 184 | 185 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload) { 186 | return send(byteBufferExp(key), payload); 187 | } 188 | 189 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload, Headers headers) { 190 | return send(byteBufferExp(key), payload, headers); 191 | } 192 | 193 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload, JExpression headers) { 194 | return send(byteBufferExp(key), payload, headers); 195 | } 196 | 197 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload) { 198 | return send(key, byteBufferExp(payload)); 199 | } 200 | 201 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload, Headers headers) { 202 | return send(key, byteBufferExp(payload), headers); 203 | } 204 | 205 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload, JExpression headers) { 206 | return send(key, byteBufferExp(payload), headers); 207 | } 208 | 209 | public RequestBuilder send(Bytes key, ExpressionBuilder payload) { 210 | return send(bytesExp(key), payload); 211 | } 212 | 213 | public RequestBuilder send(Bytes key, ExpressionBuilder payload, Headers headers) { 214 | return send(bytesExp(key), payload, headers); 215 | } 216 | 217 | public RequestBuilder send(Bytes key, ExpressionBuilder payload, JExpression headers) { 218 | return send(bytesExp(key), payload, headers); 219 | } 220 | 221 | public RequestBuilder send(ExpressionBuilder key, Bytes payload) { 222 | return send(key, bytesExp(payload)); 223 | } 224 | 225 | public RequestBuilder send(ExpressionBuilder key, Bytes payload, Headers headers) { 226 | return send(key, bytesExp(payload), headers); 227 | } 228 | 229 | public RequestBuilder send(ExpressionBuilder key, Bytes payload, JExpression headers) { 230 | return send(key, bytesExp(payload), headers); 231 | } 232 | 233 | public RequestBuilder send(byte[] key, ExpressionBuilder payload) { 234 | return send(byteArrayExp(key), payload); 235 | } 236 | 237 | public RequestBuilder send(byte[] key, ExpressionBuilder payload, Headers headers) { 238 | return send(byteArrayExp(key), payload, headers); 239 | } 240 | 241 | public RequestBuilder send(byte[] key, ExpressionBuilder payload, JExpression headers) { 242 | return send(byteArrayExp(key), payload, headers); 243 | } 244 | 245 | public RequestBuilder send(ExpressionBuilder key, byte[] payload) { 246 | return send(key, byteArrayExp(payload)); 247 | } 248 | 249 | public RequestBuilder send(ExpressionBuilder key, byte[] payload, Headers headers) { 250 | return send(key, byteArrayExp(payload), headers); 251 | } 252 | 253 | public RequestBuilder send(ExpressionBuilder key, byte[] payload, JExpression headers) { 254 | return send(key, byteArrayExp(payload), headers); 255 | } 256 | 257 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload) { 258 | return send(key, payload, new RecordHeaders()); 259 | } 260 | 261 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload, Headers headers) { 262 | return new RequestBuilder(wrapped.send( 263 | key.gatlingExpression(), 264 | payload.gatlingExpression(), 265 | toStaticValueExpression(headers), 266 | key.getSerde(), 267 | ClassTag.apply(key.getType()), 268 | payload.getSerde(), 269 | ClassTag.apply(payload.getType()) 270 | )); 271 | } 272 | 273 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload, 274 | JExpression headers) { 275 | return new RequestBuilder<>(wrapped.send( 276 | key.gatlingExpression(), 277 | payload.gatlingExpression(), 278 | javaFunctionToExpression(headers), 279 | key.getSerde(), 280 | ClassTag.apply(key.getType()), 281 | payload.getSerde(), 282 | ClassTag.apply(payload.getType()) 283 | )); 284 | } 285 | 286 | @SuppressWarnings("unchecked") 287 | public RequestBuilder sendWithClass(V payload, Class vClass) { 288 | var res = (Object) wrapped.send(toStaticValueExpression(payload), Serdes.serdeFrom(vClass), ClassTag.apply(vClass)); 289 | return new RequestBuilder<>( 290 | (org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilder) 291 | res); 292 | } 293 | 294 | @SuppressWarnings("unchecked") 295 | public RequestBuilder sendWithClass(V payload, Class vClass, Headers headers) { 296 | var res = (Object) wrapped.send( 297 | null, 298 | toStaticValueExpression(payload), 299 | toStaticValueExpression(headers), 300 | Serdes.serdeFrom(String.class), 301 | ClassTag.apply(String.class), 302 | Serdes.serdeFrom(vClass), 303 | ClassTag.apply(vClass)); 304 | return new RequestBuilder<>( 305 | (org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilder) 306 | res); 307 | } 308 | 309 | 310 | public RequestBuilder send(String string) { 311 | return sendWithClass(string, String.class); 312 | } 313 | 314 | public RequestBuilder send(String string, Headers headers) { 315 | return sendWithClass(string, String.class, headers); 316 | } 317 | 318 | public RequestBuilder send(Integer value) { 319 | return sendWithClass(value, Integer.class); 320 | } 321 | 322 | public RequestBuilder send(Long value) { 323 | return sendWithClass(value, Long.class); 324 | } 325 | 326 | public RequestBuilder send(String string, double v) { 327 | return send(stringExp(string), doubleExp(v)); 328 | } 329 | 330 | 331 | } 332 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/KafkaRequestBuilderBase.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import org.apache.kafka.common.header.Headers; 4 | import org.apache.kafka.common.header.internals.RecordHeaders; 5 | import org.apache.kafka.common.serialization.Serdes; 6 | import org.apache.kafka.common.utils.Bytes; 7 | import org.galaxio.gatling.kafka.javaapi.request.expressions.ExpressionBuilder; 8 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression; 9 | import scala.reflect.ClassTag; 10 | 11 | import java.nio.ByteBuffer; 12 | 13 | import static io.gatling.javaapi.core.internal.Expressions.javaFunctionToExpression; 14 | import static io.gatling.javaapi.core.internal.Expressions.toStaticValueExpression; 15 | import static org.galaxio.gatling.kafka.javaapi.KafkaDsl.*; 16 | 17 | public class KafkaRequestBuilderBase { 18 | 19 | private final org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase wrapped; 20 | private final String requestName; 21 | 22 | public KafkaRequestBuilderBase(org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase wrapped, String requestName) { 23 | this.wrapped = wrapped; 24 | this.requestName = requestName; 25 | } 26 | 27 | 28 | public RequestBuilder send(String key, String payload) { 29 | return send(stringExp(key), stringExp(payload)); 30 | } 31 | 32 | public RequestBuilder send(String key, String payload, Headers headers) { 33 | return send(stringExp(key), stringExp(payload), headers); 34 | } 35 | 36 | public RequestBuilder send(String key, String payload, JExpression headers) { 37 | return send(stringExp(key), stringExp(payload), headers); 38 | } 39 | 40 | public RequestBuilder send(String key, ExpressionBuilder payload) { 41 | return send(stringExp(key), payload); 42 | } 43 | 44 | public RequestBuilder send(String key, ExpressionBuilder payload, Headers headers) { 45 | return send(stringExp(key), payload, headers); 46 | } 47 | 48 | public RequestBuilder send(String key, ExpressionBuilder payload, JExpression headers) { 49 | return send(stringExp(key), payload, headers); 50 | } 51 | 52 | public RequestBuilder send(ExpressionBuilder key, String payload) { 53 | return send(key, stringExp(payload)); 54 | } 55 | 56 | public RequestBuilder send(ExpressionBuilder key, String payload, Headers headers) { 57 | return send(key, stringExp(payload), headers); 58 | } 59 | 60 | public RequestBuilder send(ExpressionBuilder key, String payload, JExpression headers) { 61 | return send(key, stringExp(payload), headers); 62 | } 63 | 64 | public RequestBuilder send(Float key, ExpressionBuilder payload) { 65 | return send(floatExp(key), payload); 66 | } 67 | 68 | public RequestBuilder send(Float key, ExpressionBuilder payload, Headers headers) { 69 | return send(floatExp(key), payload, headers); 70 | } 71 | 72 | public RequestBuilder send(Float key, ExpressionBuilder payload, JExpression headers) { 73 | return send(floatExp(key), payload, headers); 74 | } 75 | 76 | public RequestBuilder send(ExpressionBuilder key, Float payload) { 77 | return send(key, floatExp(payload)); 78 | } 79 | 80 | public RequestBuilder send(ExpressionBuilder key, Float payload, Headers headers) { 81 | return send(key, floatExp(payload), headers); 82 | } 83 | 84 | public RequestBuilder send(ExpressionBuilder key, Float payload, JExpression headers) { 85 | return send(key, floatExp(payload), headers); 86 | } 87 | 88 | public RequestBuilder send(Double key, ExpressionBuilder payload) { 89 | return send(doubleExp(key), payload); 90 | } 91 | 92 | public RequestBuilder send(Double key, ExpressionBuilder payload, Headers headers) { 93 | return send(doubleExp(key), payload, headers); 94 | } 95 | 96 | public RequestBuilder send(Double key, ExpressionBuilder payload, JExpression headers) { 97 | return send(doubleExp(key), payload, headers); 98 | } 99 | 100 | public RequestBuilder send(ExpressionBuilder key, Double payload) { 101 | return send(key, doubleExp(payload)); 102 | } 103 | 104 | public RequestBuilder send(ExpressionBuilder key, Double payload, Headers headers) { 105 | return send(key, doubleExp(payload), headers); 106 | } 107 | 108 | public RequestBuilder send(ExpressionBuilder key, Double payload, JExpression headers) { 109 | return send(key, doubleExp(payload), headers); 110 | } 111 | 112 | public RequestBuilder send(Short key, ExpressionBuilder payload) { 113 | return send(shortExp(key), payload); 114 | } 115 | 116 | public RequestBuilder send(Short key, ExpressionBuilder payload, Headers headers) { 117 | return send(shortExp(key), payload, headers); 118 | } 119 | 120 | public RequestBuilder send(Short key, ExpressionBuilder payload, JExpression headers) { 121 | return send(shortExp(key), payload, headers); 122 | } 123 | 124 | public RequestBuilder send(ExpressionBuilder key, Short payload) { 125 | return send(key, shortExp(payload)); 126 | } 127 | 128 | public RequestBuilder send(ExpressionBuilder key, Short payload, Headers headers) { 129 | return send(key, shortExp(payload), headers); 130 | } 131 | 132 | public RequestBuilder send(ExpressionBuilder key, Short payload, JExpression headers) { 133 | return send(key, shortExp(payload), headers); 134 | } 135 | 136 | public RequestBuilder send(Integer key, ExpressionBuilder payload) { 137 | return send(integerExp(key), payload); 138 | } 139 | 140 | public RequestBuilder send(Integer key, ExpressionBuilder payload, Headers headers) { 141 | return send(integerExp(key), payload, headers); 142 | } 143 | 144 | public RequestBuilder send(Integer key, ExpressionBuilder payload, JExpression headers) { 145 | return send(integerExp(key), payload, headers); 146 | } 147 | 148 | public RequestBuilder send(ExpressionBuilder key, Integer payload) { 149 | return send(key, integerExp(payload)); 150 | } 151 | 152 | public RequestBuilder send(ExpressionBuilder key, Integer payload, Headers headers) { 153 | return send(key, integerExp(payload), headers); 154 | } 155 | 156 | public RequestBuilder send(ExpressionBuilder key, Integer payload, JExpression headers) { 157 | return send(key, integerExp(payload), headers); 158 | } 159 | 160 | public RequestBuilder send(Long key, ExpressionBuilder payload) { 161 | return send(longExp(key), payload); 162 | } 163 | 164 | public RequestBuilder send(Integer key, Long payload) { 165 | return send(integerExp(key), longExp(payload)); 166 | } 167 | 168 | public RequestBuilder send(Long key, ExpressionBuilder payload, Headers headers) { 169 | return send(longExp(key), payload, headers); 170 | } 171 | 172 | public RequestBuilder send(Long key, ExpressionBuilder payload, JExpression headers) { 173 | return send(longExp(key), payload, headers); 174 | } 175 | 176 | public RequestBuilder send(ExpressionBuilder key, Long payload) { 177 | return send(key, longExp(payload)); 178 | } 179 | 180 | public RequestBuilder send(ExpressionBuilder key, Long payload, Headers headers) { 181 | return send(key, longExp(payload), headers); 182 | } 183 | 184 | public RequestBuilder send(ExpressionBuilder key, Long payload, JExpression headers) { 185 | return send(key, longExp(payload), headers); 186 | } 187 | 188 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload) { 189 | return send(byteBufferExp(key), payload); 190 | } 191 | 192 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload, Headers headers) { 193 | return send(byteBufferExp(key), payload, headers); 194 | } 195 | 196 | public RequestBuilder send(ByteBuffer key, ExpressionBuilder payload, JExpression headers) { 197 | return send(byteBufferExp(key), payload, headers); 198 | } 199 | 200 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload) { 201 | return send(key, byteBufferExp(payload)); 202 | } 203 | 204 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload, Headers headers) { 205 | return send(key, byteBufferExp(payload), headers); 206 | } 207 | 208 | public RequestBuilder send(ExpressionBuilder key, ByteBuffer payload, JExpression headers) { 209 | return send(key, byteBufferExp(payload), headers); 210 | } 211 | 212 | public RequestBuilder send(Bytes key, ExpressionBuilder payload) { 213 | return send(bytesExp(key), payload); 214 | } 215 | 216 | public RequestBuilder send(Bytes key, ExpressionBuilder payload, Headers headers) { 217 | return send(bytesExp(key), payload, headers); 218 | } 219 | 220 | public RequestBuilder send(Bytes key, ExpressionBuilder payload, JExpression headers) { 221 | return send(bytesExp(key), payload, headers); 222 | } 223 | 224 | public RequestBuilder send(ExpressionBuilder key, Bytes payload) { 225 | return send(key, bytesExp(payload)); 226 | } 227 | 228 | public RequestBuilder send(ExpressionBuilder key, Bytes payload, Headers headers) { 229 | return send(key, bytesExp(payload), headers); 230 | } 231 | 232 | public RequestBuilder send(ExpressionBuilder key, Bytes payload, JExpression headers) { 233 | return send(key, bytesExp(payload), headers); 234 | } 235 | 236 | public RequestBuilder send(byte[] key, ExpressionBuilder payload) { 237 | return send(byteArrayExp(key), payload); 238 | } 239 | 240 | public RequestBuilder send(byte[] key, ExpressionBuilder payload, Headers headers) { 241 | return send(byteArrayExp(key), payload, headers); 242 | } 243 | 244 | public RequestBuilder send(byte[] key, ExpressionBuilder payload, JExpression headers) { 245 | return send(byteArrayExp(key), payload, headers); 246 | } 247 | 248 | public RequestBuilder send(ExpressionBuilder key, byte[] payload) { 249 | return send(key, byteArrayExp(payload)); 250 | } 251 | 252 | public RequestBuilder send(ExpressionBuilder key, byte[] payload, Headers headers) { 253 | return send(key, byteArrayExp(payload), headers); 254 | } 255 | 256 | public RequestBuilder send(ExpressionBuilder key, byte[] payload, JExpression headers) { 257 | return send(key, byteArrayExp(payload), headers); 258 | } 259 | 260 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload) { 261 | return send(key, payload, new RecordHeaders()); 262 | } 263 | 264 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload, Headers headers) { 265 | return new RequestBuilder(wrapped.send( 266 | key.gatlingExpression(), 267 | payload.gatlingExpression(), 268 | toStaticValueExpression(headers), 269 | key.getSerde(), 270 | ClassTag.apply(key.getType()), 271 | payload.getSerde(), 272 | ClassTag.apply(payload.getType()) 273 | )); 274 | } 275 | 276 | public RequestBuilder send(ExpressionBuilder key, ExpressionBuilder payload, 277 | JExpression headers) { 278 | return new RequestBuilder<>(wrapped.send( 279 | key.gatlingExpression(), 280 | payload.gatlingExpression(), 281 | javaFunctionToExpression(headers), 282 | key.getSerde(), 283 | ClassTag.apply(key.getType()), 284 | payload.getSerde(), 285 | ClassTag.apply(payload.getType()) 286 | )); 287 | } 288 | 289 | @SuppressWarnings("unchecked") 290 | public RequestBuilder sendWithClass(V payload, Class vClass) { 291 | var res = (Object) wrapped.send(toStaticValueExpression(payload), Serdes.serdeFrom(vClass), ClassTag.apply(vClass)); 292 | return new RequestBuilder<>( 293 | (org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilder) 294 | res); 295 | } 296 | 297 | @SuppressWarnings("unchecked") 298 | public RequestBuilder sendWithClass(V payload, Class vClass, Headers headers) { 299 | var res = (Object) wrapped.send( 300 | null, 301 | toStaticValueExpression(payload), 302 | toStaticValueExpression(headers), 303 | Serdes.serdeFrom(Object.class), 304 | ClassTag.apply(Object.class), 305 | Serdes.serdeFrom(vClass), 306 | ClassTag.apply(vClass)); 307 | return new RequestBuilder<>( 308 | (org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilder) 309 | res); 310 | } 311 | 312 | public ReqRepBase requestReply() { 313 | return new ReqRepBase(requestName); 314 | } 315 | 316 | public RequestBuilder send(String string) { 317 | return sendWithClass(string, String.class); 318 | } 319 | 320 | public RequestBuilder send(String string, Headers headers) { 321 | return sendWithClass(string, String.class, headers); 322 | } 323 | 324 | public RequestBuilder send(Integer value) { 325 | return sendWithClass(value, Integer.class); 326 | } 327 | 328 | public RequestBuilder send(Long value) { 329 | return sendWithClass(value, Long.class); 330 | } 331 | 332 | public RequestBuilder send(String string, double v) { 333 | return send(stringExp(string), doubleExp(v)); 334 | } 335 | 336 | public OnlyPublishStep topic(String producerTopic) { 337 | return new OnlyPublishStep( 338 | new org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase.OnlyPublishStep(toStaticValueExpression(this.requestName), toStaticValueExpression(producerTopic)) 339 | ); 340 | } 341 | } 342 | -------------------------------------------------------------------------------- /src/main/java/org/galaxio/gatling/kafka/javaapi/request/builder/RROutTopicStep.java: -------------------------------------------------------------------------------- 1 | package org.galaxio.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.commons.validation.Validation; 4 | import io.gatling.core.session.Session; 5 | import org.apache.kafka.common.header.Headers; 6 | import org.apache.kafka.common.header.internals.RecordHeaders; 7 | import org.apache.kafka.common.serialization.Deserializer; 8 | import org.apache.kafka.common.serialization.Serdes; 9 | import org.apache.kafka.common.serialization.Serializer; 10 | import org.apache.kafka.common.utils.Bytes; 11 | import org.galaxio.gatling.kafka.javaapi.request.expressions.ExpressionBuilder; 12 | import org.galaxio.gatling.kafka.javaapi.request.expressions.JExpression; 13 | import org.galaxio.gatling.kafka.request.builder.KafkaRequestBuilderBase; 14 | import scala.reflect.ClassTag; 15 | 16 | import java.nio.ByteBuffer; 17 | 18 | import static io.gatling.javaapi.core.internal.Expressions.*; 19 | import static org.galaxio.gatling.kafka.javaapi.KafkaDsl.*; 20 | 21 | public class RROutTopicStep { 22 | 23 | private final scala.Function1> inputTopic; 24 | private final scala.Function1> outputTopic; 25 | private final String requestName; 26 | 27 | public RROutTopicStep(scala.Function1> inputTopic, scala.Function1> outputTopic, String requestName) { 28 | this.inputTopic = inputTopic; 29 | this.outputTopic = outputTopic; 30 | this.requestName = requestName; 31 | } 32 | 33 | public RequestReplyBuilder send(String key, ExpressionBuilder payload) { 34 | return send(stringExp(cf(key)), payload); 35 | } 36 | 37 | public RequestReplyBuilder send(String key, ExpressionBuilder payload, Headers headers) { 38 | return send(stringExp(cf(key)), payload, headers); 39 | } 40 | 41 | public RequestReplyBuilder send(String key, ExpressionBuilder payload, JExpression headers) { 42 | return send(stringExp(cf(key)), payload, headers); 43 | } 44 | 45 | public RequestReplyBuilder send(String key, String payload) { 46 | return send(key, payload, new RecordHeaders()); 47 | } 48 | 49 | public RequestReplyBuilder send(String key, String payload, Headers headers) { 50 | return send(stringExp(cf(key)), stringExp(cf(payload)), headers); 51 | } 52 | 53 | public RequestReplyBuilder send(String key, String payload, JExpression headers) { 54 | return send(stringExp(cf(key)), stringExp(cf(payload)), headers); 55 | } 56 | 57 | public RequestReplyBuilder send(ExpressionBuilder key, String payload) { 58 | return send(key, stringExp(cf(payload))); 59 | } 60 | 61 | public RequestReplyBuilder send(ExpressionBuilder key, String payload, Headers headers) { 62 | return send(key, stringExp(cf(payload)), headers); 63 | } 64 | 65 | public RequestReplyBuilder send(ExpressionBuilder key, String payload, JExpression headers) { 66 | return send(key, stringExp(cf(payload)), headers); 67 | } 68 | 69 | public RequestReplyBuilder send(Float key, ExpressionBuilder payload) { 70 | return send(floatExp(cf(key)), payload); 71 | } 72 | 73 | public RequestReplyBuilder send(Float key, ExpressionBuilder payload, Headers headers) { 74 | return send(floatExp(cf(key)), payload, headers); 75 | } 76 | 77 | public RequestReplyBuilder send(Float key, ExpressionBuilder payload, JExpression headers) { 78 | return send(floatExp(cf(key)), payload, headers); 79 | } 80 | 81 | public RequestReplyBuilder send(ExpressionBuilder key, Float payload) { 82 | return send(key, floatExp(cf(payload))); 83 | } 84 | 85 | public RequestReplyBuilder send(ExpressionBuilder key, Float payload, Headers headers) { 86 | return send(key, floatExp(cf(payload)), headers); 87 | } 88 | 89 | public RequestReplyBuilder send(ExpressionBuilder key, Float payload, JExpression headers) { 90 | return send(key, floatExp(cf(payload)), headers); 91 | } 92 | 93 | public RequestReplyBuilder send(Double key, ExpressionBuilder payload) { 94 | return send(doubleExp(cf(key)), payload); 95 | } 96 | 97 | public RequestReplyBuilder send(Double key, ExpressionBuilder payload, Headers headers) { 98 | return send(doubleExp(cf(key)), payload, headers); 99 | } 100 | 101 | public RequestReplyBuilder send(Double key, ExpressionBuilder payload, JExpression headers) { 102 | return send(doubleExp(cf(key)), payload, headers); 103 | } 104 | 105 | public RequestReplyBuilder send(ExpressionBuilder key, Double payload) { 106 | return send(key, doubleExp(cf(payload))); 107 | } 108 | 109 | public RequestReplyBuilder send(ExpressionBuilder key, Double payload, Headers headers) { 110 | return send(key, doubleExp(cf(payload)), headers); 111 | } 112 | 113 | public RequestReplyBuilder send(ExpressionBuilder key, Double payload, JExpression headers) { 114 | return send(key, doubleExp(cf(payload)), headers); 115 | } 116 | 117 | public RequestReplyBuilder send(Short key, ExpressionBuilder payload) { 118 | return send(shortExp(cf(key)), payload); 119 | } 120 | 121 | public RequestReplyBuilder send(Short key, ExpressionBuilder payload, Headers headers) { 122 | return send(shortExp(cf(key)), payload, headers); 123 | } 124 | 125 | public RequestReplyBuilder send(Short key, ExpressionBuilder payload, JExpression headers) { 126 | return send(shortExp(cf(key)), payload, headers); 127 | } 128 | 129 | public RequestReplyBuilder send(ExpressionBuilder key, Short payload) { 130 | return send(key, shortExp(cf(payload))); 131 | } 132 | 133 | public RequestReplyBuilder send(ExpressionBuilder key, Short payload, Headers headers) { 134 | return send(key, shortExp(cf(payload)), headers); 135 | } 136 | 137 | public RequestReplyBuilder send(ExpressionBuilder key, Short payload, JExpression headers) { 138 | return send(key, shortExp(cf(payload)), headers); 139 | } 140 | 141 | public RequestReplyBuilder send(Integer key, ExpressionBuilder payload) { 142 | return send(integerExp(cf(key)), payload); 143 | } 144 | 145 | public RequestReplyBuilder send(Integer key, ExpressionBuilder payload, Headers headers) { 146 | return send(integerExp(cf(key)), payload, headers); 147 | } 148 | 149 | public RequestReplyBuilder send(Integer key, ExpressionBuilder payload, JExpression headers) { 150 | return send(integerExp(cf(key)), payload, headers); 151 | } 152 | 153 | public RequestReplyBuilder send(ExpressionBuilder key, Integer payload) { 154 | return send(key, integerExp(cf(payload))); 155 | } 156 | 157 | public RequestReplyBuilder send(ExpressionBuilder key, Integer payload, Headers headers) { 158 | return send(key, integerExp(cf(payload)), headers); 159 | } 160 | 161 | public RequestReplyBuilder send(ExpressionBuilder key, Integer payload, JExpression headers) { 162 | return send(key, integerExp(cf(payload)), headers); 163 | } 164 | 165 | public RequestReplyBuilder send(Long key, ExpressionBuilder payload) { 166 | return send(longExp(cf(key)), payload); 167 | } 168 | 169 | public RequestReplyBuilder send(Long key, ExpressionBuilder payload, Headers headers) { 170 | return send(longExp(cf(key)), payload, headers); 171 | } 172 | 173 | public RequestReplyBuilder send(Long key, ExpressionBuilder payload, JExpression headers) { 174 | return send(longExp(cf(key)), payload, headers); 175 | } 176 | 177 | public RequestReplyBuilder send(ExpressionBuilder key, Long payload) { 178 | return send(key, longExp(cf(payload))); 179 | } 180 | 181 | public RequestReplyBuilder send(ExpressionBuilder key, Long payload, Headers headers) { 182 | return send(key, longExp(cf(payload)), headers); 183 | } 184 | 185 | public RequestReplyBuilder send(ExpressionBuilder key, Long payload, JExpression headers) { 186 | return send(key, longExp(cf(payload)), headers); 187 | } 188 | 189 | public RequestReplyBuilder send(ByteBuffer key, ExpressionBuilder payload) { 190 | return send(byteBufferExp(cf(key)), payload); 191 | } 192 | 193 | public RequestReplyBuilder send(ByteBuffer key, ExpressionBuilder payload, Headers headers) { 194 | return send(byteBufferExp(cf(key)), payload, headers); 195 | } 196 | 197 | public RequestReplyBuilder send(ByteBuffer key, ExpressionBuilder payload, JExpression headers) { 198 | return send(byteBufferExp(cf(key)), payload, headers); 199 | } 200 | 201 | public RequestReplyBuilder send(ExpressionBuilder key, ByteBuffer payload) { 202 | return send(key, byteBufferExp(cf(payload))); 203 | } 204 | 205 | public RequestReplyBuilder send(ExpressionBuilder key, ByteBuffer payload, Headers headers) { 206 | return send(key, byteBufferExp(cf(payload)), headers); 207 | } 208 | 209 | public RequestReplyBuilder send(ExpressionBuilder key, ByteBuffer payload, JExpression headers) { 210 | return send(key, byteBufferExp(cf(payload)), headers); 211 | } 212 | 213 | public RequestReplyBuilder send(Bytes key, ExpressionBuilder payload) { 214 | return send(bytesExp(cf(key)), payload); 215 | } 216 | 217 | public RequestReplyBuilder send(Bytes key, ExpressionBuilder payload, Headers headers) { 218 | return send(bytesExp(cf(key)), payload, headers); 219 | } 220 | 221 | public RequestReplyBuilder send(Bytes key, ExpressionBuilder payload, JExpression headers) { 222 | return send(bytesExp(cf(key)), payload, headers); 223 | } 224 | 225 | public RequestReplyBuilder send(ExpressionBuilder key, Bytes payload) { 226 | return send(key, bytesExp(cf(payload))); 227 | } 228 | 229 | public RequestReplyBuilder send(ExpressionBuilder key, Bytes payload, Headers headers) { 230 | return send(key, bytesExp(cf(payload)), headers); 231 | } 232 | 233 | public RequestReplyBuilder send(ExpressionBuilder key, Bytes payload, JExpression headers) { 234 | return send(key, bytesExp(cf(payload)), headers); 235 | } 236 | 237 | public RequestReplyBuilder send(ExpressionBuilder key, ExpressionBuilder payload) { 238 | return send(key, payload, new RecordHeaders()); 239 | } 240 | 241 | public RequestReplyBuilder send(ExpressionBuilder key, ExpressionBuilder payload, Headers headers) { 242 | return new RequestReplyBuilder<>(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 243 | .requestTopic(this.inputTopic) 244 | .replyTopic(this.outputTopic) 245 | .send( 246 | key.gatlingExpression(), 247 | payload.gatlingExpression(), 248 | toStaticValueExpression(headers), 249 | key.getSerde(), 250 | ClassTag.apply(key.getType()), 251 | payload.getSerde(), 252 | ClassTag.apply(payload.getType()) 253 | )); 254 | } 255 | 256 | public RequestReplyBuilder send(ExpressionBuilder key, ExpressionBuilder payload, JExpression headers) { 257 | return new RequestReplyBuilder<>(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 258 | .requestTopic(this.inputTopic) 259 | .replyTopic(this.outputTopic) 260 | .send( 261 | key.gatlingExpression(), 262 | payload.gatlingExpression(), 263 | javaFunctionToExpression(headers), 264 | key.getSerde(), 265 | ClassTag.apply(key.getType()), 266 | payload.getSerde(), 267 | ClassTag.apply(payload.getType()) 268 | )); 269 | } 270 | 271 | public RequestReplyBuilder send(K key, V payload, Class keyClass, Class payloadClass) { 272 | return new RequestReplyBuilder<>(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 273 | .requestTopic(this.inputTopic) 274 | .replyTopic(this.outputTopic) 275 | .send( 276 | toStaticValueExpression(key), 277 | toStaticValueExpression(payload), 278 | toStaticValueExpression(new RecordHeaders()), 279 | Serdes.serdeFrom(keyClass), 280 | ClassTag.apply(keyClass), 281 | Serdes.serdeFrom(payloadClass), 282 | ClassTag.apply(payloadClass) 283 | )); 284 | } 285 | 286 | public RequestReplyBuilder send(K key, V payload, Headers headers, Class keyClass, Class payloadClass) { 287 | return new RequestReplyBuilder<>(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 288 | .requestTopic(this.inputTopic) 289 | .replyTopic(this.outputTopic) 290 | .send( 291 | toStaticValueExpression(key), 292 | toStaticValueExpression(payload), 293 | toStaticValueExpression(headers), 294 | Serdes.serdeFrom(keyClass), 295 | ClassTag.apply(keyClass), 296 | Serdes.serdeFrom(payloadClass), 297 | ClassTag.apply(payloadClass) 298 | )); 299 | } 300 | 301 | public RequestReplyBuilder send(K key, V payload, JExpression headers, Class keyClass, Class payloadClass) { 302 | return new RequestReplyBuilder<>(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 303 | .requestTopic(this.inputTopic) 304 | .replyTopic(this.outputTopic) 305 | .send( 306 | toStaticValueExpression(key), 307 | toStaticValueExpression(payload), 308 | javaFunctionToExpression(headers), 309 | Serdes.serdeFrom(keyClass), 310 | ClassTag.apply(keyClass), 311 | Serdes.serdeFrom(payloadClass), 312 | ClassTag.apply(payloadClass) 313 | )); 314 | } 315 | 316 | public RequestReplyBuilder send(K key, V payload, Class keyClass, Class payloadClass, Serializer ser, Deserializer de) { 317 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 318 | .requestTopic(this.inputTopic) 319 | .replyTopic(this.outputTopic) 320 | .send( 321 | toStaticValueExpression(key), 322 | toStaticValueExpression(payload), 323 | toStaticValueExpression(new RecordHeaders()), 324 | Serdes.serdeFrom(keyClass), 325 | ClassTag.apply(keyClass), 326 | Serdes.serdeFrom(ser, de), 327 | ClassTag.apply(payloadClass) 328 | )); 329 | } 330 | 331 | public RequestReplyBuilder send(K key, V payload, Headers headers, Class keyClass, Class payloadClass, Serializer ser, Deserializer de) { 332 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 333 | .requestTopic(this.inputTopic) 334 | .replyTopic(this.outputTopic) 335 | .send( 336 | toStaticValueExpression(key), 337 | toStaticValueExpression(payload), 338 | toStaticValueExpression(headers), 339 | Serdes.serdeFrom(keyClass), 340 | ClassTag.apply(keyClass), 341 | Serdes.serdeFrom(ser, de), 342 | ClassTag.apply(payloadClass) 343 | )); 344 | } 345 | 346 | public RequestReplyBuilder send(K key, V payload, JExpression headers, Class keyClass, Class payloadClass, Serializer ser, Deserializer de) { 347 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 348 | .requestTopic(this.inputTopic) 349 | .replyTopic(this.outputTopic) 350 | .send( 351 | toStaticValueExpression(key), 352 | toStaticValueExpression(payload), 353 | javaFunctionToExpression(headers), 354 | Serdes.serdeFrom(keyClass), 355 | ClassTag.apply(keyClass), 356 | Serdes.serdeFrom(ser, de), 357 | ClassTag.apply(payloadClass) 358 | )); 359 | } 360 | 361 | } 362 | --------------------------------------------------------------------------------