├── project ├── build.properties ├── plugins.sbt └── Dependencies.scala ├── .git-blame-ignore-revs ├── src ├── main │ ├── scala │ │ └── ru │ │ │ └── tinkoff │ │ │ └── gatling │ │ │ └── kafka │ │ │ ├── Predef.scala │ │ │ ├── request │ │ │ ├── builder │ │ │ │ ├── RequestBuilder.scala │ │ │ │ ├── KafkaAttributes.scala │ │ │ │ ├── KafkaRequestBuilder.scala │ │ │ │ ├── KafkaAvro4sRequestBuilder.scala │ │ │ │ ├── Avro4sAttributes.scala │ │ │ │ ├── KafkaRequestReplyAttributes.scala │ │ │ │ ├── LowPriorSender.scala │ │ │ │ ├── Sender.scala │ │ │ │ └── KafkaRequestBuilderBase.scala │ │ │ ├── KafkaProtocolMessage.scala │ │ │ └── KafkaSerdesImplicits.scala │ │ │ ├── protocol │ │ │ ├── KafkaProtocolBuilderPropertiesStep.scala │ │ │ ├── KafkaComponents.scala │ │ │ ├── KafkaProtocolBuilder.scala │ │ │ ├── KafkaProtocolBuilderNew.scala │ │ │ └── KafkaProtocol.scala │ │ │ ├── package.scala │ │ │ ├── client │ │ │ ├── KafkaMessageTracker.scala │ │ │ ├── KafkaSender.scala │ │ │ ├── TrackersPool.scala │ │ │ └── KafkaMessageTrackerActor.scala │ │ │ ├── checks │ │ │ ├── KafkaMessageCheck.scala │ │ │ ├── AvroBodyCheckBuilder.scala │ │ │ ├── KafkaMessagePreparer.scala │ │ │ ├── KafkaCheckMaterializer.scala │ │ │ └── KafkaCheckSupport.scala │ │ │ ├── actions │ │ │ ├── KafkaRequestActionBuilder.scala │ │ │ ├── KafkaRequestReplyActionBuilder.scala │ │ │ ├── KafkaRequestAvro4sActionBuilder.scala │ │ │ ├── KafkaAvro4sRequestAction.scala │ │ │ ├── KafkaRequestAction.scala │ │ │ └── KafkaRequestReplyAction.scala │ │ │ └── KafkaDsl.scala │ └── java │ │ └── ru │ │ └── tinkoff │ │ └── gatling │ │ └── kafka │ │ └── javaapi │ │ ├── checks │ │ ├── KafkaCheckType.java │ │ └── KafkaChecks.scala │ │ ├── protocol │ │ ├── KafkaProtocolBuilderNewBase.java │ │ ├── KPProducerSettingsStep.java │ │ ├── KafkaProtocolBuilderBase.java │ │ ├── KafkaProtocolBuilder.java │ │ ├── KafkaProtocolBuilderPropertiesStep.java │ │ ├── KafkaProtocolBuilderNew.java │ │ └── KPConsumeSettingsStep.java │ │ ├── request │ │ └── builder │ │ │ ├── ReqRepBase.java │ │ │ ├── RRInTopicStep.java │ │ │ ├── RequestBuilder.java │ │ │ ├── RequestReplyBuilder.java │ │ │ ├── KafkaRequestBuilderBase.java │ │ │ └── RROutTopicStep.java │ │ └── KafkaDsl.java └── test │ ├── scala │ └── ru │ │ └── tinkoff │ │ └── gatling │ │ └── kafka │ │ └── examples │ │ ├── GatlingRunner.scala │ │ ├── ProducerSimulation.scala │ │ ├── KafkaJavaapiMethodsGatlingTest.scala │ │ ├── BasicSimulation.scala │ │ ├── Avro4sSimulation.scala │ │ ├── MatchSimulation.scala │ │ ├── AvroClassWithRequestReplySimulation.scala │ │ └── KafkaGatlingTest.scala │ ├── resources │ └── logback.xml │ ├── java │ └── ru │ │ └── tinkoff │ │ └── gatling │ │ └── kafka │ │ └── javaapi │ │ └── examples │ │ ├── ProducerSimulation.java │ │ ├── BasicSimulation.java │ │ ├── AvroClassWithRequestReplySimulation.java │ │ └── MatchSimulation.java │ └── kotlin │ └── ru │ └── tinkoff │ └── gatling │ └── kafka │ └── javaapi │ └── examples │ ├── ProducerSimulation.kt │ ├── BasicSimulation.kt │ ├── AvroClassWithRequestReplySimulation.kt │ └── MatchSimulation.kt ├── .scalafmt.conf ├── publish.sbt ├── .gitignore ├── .github └── workflows │ └── ci.yml ├── README.md └── LICENSE /project/build.properties: -------------------------------------------------------------------------------- 1 | # suppress inspection "UnusedProperty" 2 | sbt.version=1.8.2 -------------------------------------------------------------------------------- /.git-blame-ignore-revs: -------------------------------------------------------------------------------- 1 | # Scala Steward: Reformat with scalafmt 3.7.1 2 | 3be9944a4bdcf03615c2357d5c32c62a713d2ce7 3 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/Predef.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka 2 | 3 | object Predef extends KafkaDsl 4 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | runner.dialect = "scala213" 2 | version=3.7.3 3 | binPack.parentConstructors = true 4 | maxColumn = 128 5 | includeCurlyBraceInSelectChains = false 6 | align.preset = most 7 | trailingCommas = always -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/RequestBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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/ru/tinkoff/gatling/kafka/javaapi/checks/KafkaCheckType.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.checks; 2 | 3 | import io.gatling.javaapi.core.CheckBuilder.CheckType; 4 | 5 | public enum KafkaCheckType implements CheckType { 6 | ResponseCode 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/protocol/KafkaProtocolBuilderPropertiesStep.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.protocol 2 | 3 | case class KafkaProtocolBuilderPropertiesStep(topic: String, props: Map[String, Object]) { 4 | 5 | def properties(props: Map[String, Object]): KafkaProtocolBuilder = 6 | KafkaProtocolBuilder(topic, props) 7 | } 8 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderNewBase.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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/scala/ru/tinkoff/gatling/kafka/request/builder/KafkaAttributes.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.session.Expression 4 | import org.apache.kafka.common.header.Headers 5 | 6 | case class KafkaAttributes[K, V]( 7 | requestName: Expression[String], 8 | key: Option[Expression[K]], 9 | payload: Expression[V], 10 | headers: Option[Expression[Headers]], 11 | ) 12 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/KafkaRequestBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.action.builder.ActionBuilder 4 | import ru.tinkoff.gatling.kafka.actions.KafkaRequestActionBuilder 5 | 6 | case class KafkaRequestBuilder[K, V](attr: KafkaAttributes[K, V]) extends RequestBuilder[K, V] { 7 | 8 | def build: ActionBuilder = new KafkaRequestActionBuilder(attr) 9 | 10 | } 11 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/KafkaAvro4sRequestBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.action.builder.ActionBuilder 4 | import ru.tinkoff.gatling.kafka.actions.KafkaRequestAvro4sActionBuilder 5 | 6 | case class KafkaAvro4sRequestBuilder[K, V](attr: Avro4sAttributes[K, V]) extends RequestBuilder[K, V] { 7 | 8 | def build: ActionBuilder = new KafkaRequestAvro4sActionBuilder(attr) 9 | 10 | } 11 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/ReqRepBase.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.request.builder; 2 | 3 | public class ReqRepBase { 4 | 5 | private final String requestName; 6 | 7 | public ReqRepBase(String requestName) { 8 | this.requestName = requestName; 9 | } 10 | 11 | public RRInTopicStep requestTopic(String inputTopic) { 12 | return new RRInTopicStep(inputTopic, this.requestName); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KPProducerSettingsStep.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderBase.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.protocol; 2 | 3 | import java.util.Collections; 4 | 5 | public class KafkaProtocolBuilderBase { 6 | 7 | public KafkaProtocolBuilderPropertiesStep topic(String name) { 8 | return new KafkaProtocolBuilderPropertiesStep(name, Collections.emptyMap()); 9 | } 10 | 11 | public KafkaProtocolBuilderNewBase requestReply() { 12 | return new KafkaProtocolBuilderNewBase(); 13 | } 14 | 15 | } 16 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/package.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling 2 | 3 | import com.typesafe.scalalogging.StrictLogging 4 | import io.gatling.core.check.Check 5 | import ru.tinkoff.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/test/scala/ru/tinkoff/gatling/kafka/examples/GatlingRunner.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.examples 2 | 3 | import io.gatling.app.Gatling 4 | import io.gatling.core.config.GatlingPropertiesBuilder 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 | val props = new GatlingPropertiesBuilder 14 | props.simulationClass(simulationClass) 15 | 16 | Gatling.fromMap(props.build) 17 | } 18 | 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/Avro4sAttributes.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import com.sksamuel.avro4s.{FromRecord, RecordFormat, SchemaFor} 4 | import io.gatling.core.session.Expression 5 | import org.apache.kafka.common.header.Headers 6 | 7 | case class Avro4sAttributes[K, V]( 8 | requestName: Expression[String], 9 | key: Option[Expression[K]], 10 | payload: Expression[V], 11 | schema: SchemaFor[V], 12 | format: RecordFormat[V], 13 | fromRecord: FromRecord[V], 14 | headers: Option[Expression[Headers]], 15 | ) 16 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/RRInTopicStep.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.request.builder; 2 | 3 | public class RRInTopicStep { 4 | 5 | private final String inputTopic; 6 | private final String requestName; 7 | 8 | public RRInTopicStep(String inputTopic, String requestName) { 9 | this.inputTopic = inputTopic; 10 | this.requestName = requestName; 11 | } 12 | 13 | public RROutTopicStep replyTopic(String outputTopic) { 14 | return new RROutTopicStep(this.inputTopic, outputTopic, this.requestName); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/RequestBuilder.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.javaapi.core.ActionBuilder; 4 | 5 | public class RequestBuilder implements ActionBuilder { 6 | 7 | private final ru.tinkoff.gatling.kafka.request.builder.RequestBuilder wrapped; 8 | 9 | public RequestBuilder(ru.tinkoff.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 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KafkaProtocolBuilder.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.protocol; 2 | 3 | import io.gatling.core.protocol.Protocol; 4 | import io.gatling.javaapi.core.ProtocolBuilder; 5 | 6 | public class KafkaProtocolBuilder implements ProtocolBuilder { 7 | 8 | private final ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilder wrapped; 9 | 10 | public KafkaProtocolBuilder(ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilder wrapped) { 11 | this.wrapped = wrapped; 12 | } 13 | 14 | @Override 15 | public Protocol protocol() { 16 | return wrapped.build(); 17 | } 18 | 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/KafkaProtocolMessage.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request 2 | 3 | import org.apache.kafka.clients.producer.ProducerRecord 4 | import org.apache.kafka.common.header.Headers 5 | 6 | case class KafkaProtocolMessage( 7 | key: Array[Byte], 8 | value: Array[Byte], 9 | inputTopic: String, 10 | outputTopic: String, 11 | headers: Option[Headers] = None, 12 | responseCode: Option[String] = None, 13 | ) { 14 | def toProducerRecord: ProducerRecord[Array[Byte], Array[Byte]] = { 15 | headers.fold(new ProducerRecord(inputTopic, key, value))(hs => new ProducerRecord(inputTopic, null, key, value, hs)) 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/protocol/KafkaComponents.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.client.{KafkaSender, TrackersPool} 7 | 8 | case class KafkaComponents( 9 | coreComponents: CoreComponents, 10 | kafkaProtocol: KafkaProtocol, 11 | trackersPool: TrackersPool, 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/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/ru/tinkoff/gatling/kafka/protocol/KafkaProtocolBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.protocol 2 | 3 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher 4 | 5 | import scala.concurrent.duration.DurationInt 6 | 7 | case object KafkaProtocolBuilder { 8 | 9 | def topic(name: String): KafkaProtocolBuilderPropertiesStep = 10 | KafkaProtocolBuilderPropertiesStep(name, Map.empty[String, Object]) 11 | 12 | def requestReply: KafkaProtocolBuilderNew.type = KafkaProtocolBuilderNew 13 | 14 | } 15 | 16 | case class KafkaProtocolBuilder(topic: String, props: Map[String, Object]) { 17 | 18 | def build = new KafkaProtocol(topic, props, props, 60.seconds, KafkaKeyMatcher) 19 | 20 | } 21 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/KafkaRequestReplyAttributes.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.Serializer 6 | import ru.tinkoff.gatling.kafka.KafkaCheck 7 | 8 | case class KafkaRequestReplyAttributes[K, V]( 9 | requestName: Expression[String], 10 | inputTopic: Expression[String], 11 | outputTopic: Expression[String], 12 | key: Expression[K], 13 | value: Expression[V], 14 | headers: Option[Expression[Headers]], 15 | keySerializer: Serializer[K], 16 | valueSerializer: Serializer[V], 17 | checks: List[KafkaCheck], 18 | ) 19 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers ++= Seq( 2 | Resolver.bintrayIvyRepo("rallyhealth", "sbt-plugins"), 3 | // need for load sbt-schema-registry-plugin dependencies 4 | "Confluent" at "https://packages.confluent.io/maven/", 5 | ) 6 | 7 | addSbtPlugin("com.github.sbt" % "sbt-ci-release" % "1.5.12") 8 | addSbtPlugin("io.gatling" % "gatling-sbt" % "4.3.2") 9 | addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.5.0") 10 | addSbtPlugin("com.github.sbt" % "sbt-avro" % "3.4.2") 11 | addSbtPlugin("ru.tinkoff" % "sbt-schema-registry-plugin" % "0.2.1") 12 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "2.0.7") 13 | 14 | libraryDependencies += "org.apache.avro" % "avro-compiler" % "1.11.1" 15 | -------------------------------------------------------------------------------- /publish.sbt: -------------------------------------------------------------------------------- 1 | ThisBuild / organization := "ru.tinkoff" 2 | ThisBuild / scmInfo := Some( 3 | ScmInfo( 4 | url("https://github.com/TinkoffCreditSystems/gatling-kafka-plugin"), 5 | "git@github.com:TinkoffCreditSystems/gatling-kafka-plugin.git", 6 | ), 7 | ) 8 | 9 | ThisBuild / developers := List( 10 | Developer( 11 | id = "jigarkhwar", 12 | name = "Ioann Akhaltsev", 13 | email = "i.akhaltsev@tinkoff.ru", 14 | url = url("https://github.com/jigarkhwar"), 15 | ), 16 | ) 17 | 18 | ThisBuild / description := "Plugin to support kafka performance testing in Gatling(3.9.x)." 19 | ThisBuild / licenses += ("Apache-2.0", url("http://www.apache.org/licenses/LICENSE-2.0")) 20 | ThisBuild / homepage := Some(url("https://github.com/TinkoffCreditSystems/gatling-kafka-plugin")) 21 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/client/KafkaMessageTracker.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.client 2 | 3 | import akka.actor.ActorRef 4 | import io.gatling.core.action.Action 5 | import io.gatling.core.session.Session 6 | import ru.tinkoff.gatling.kafka.KafkaCheck 7 | import ru.tinkoff.gatling.kafka.client.KafkaMessageTrackerActor.MessagePublished 8 | 9 | class KafkaMessageTracker(actor: ActorRef) { 10 | 11 | def track( 12 | matchId: Array[Byte], 13 | sent: Long, 14 | replyTimeout: Long, 15 | checks: List[KafkaCheck], 16 | session: Session, 17 | next: Action, 18 | requestName: String, 19 | ): Unit = 20 | actor ! MessagePublished( 21 | matchId, 22 | sent, 23 | replyTimeout, 24 | checks, 25 | session, 26 | next, 27 | requestName, 28 | ) 29 | } 30 | -------------------------------------------------------------------------------- /src/test/java/ru/tinkoff/gatling/kafka/javaapi/examples/ProducerSimulation.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.javaapi.protocol.KafkaProtocolBuilder; 7 | 8 | import java.util.Map; 9 | 10 | import static io.gatling.javaapi.core.CoreDsl.*; 11 | import static ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.kafka; 12 | 13 | public class ProducerSimulation extends Simulation { 14 | 15 | private final KafkaProtocolBuilder kafkaConsumerConf = 16 | kafka().topic("test.topic") 17 | .properties(Map.of(ProducerConfig.ACKS_CONFIG, "1")); 18 | 19 | private final ScenarioBuilder scn = scenario("Basic") 20 | .exec(kafka("BasicRequest").send("foo")) 21 | .exec(kafka("dld").send("true", 12.0)); 22 | 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/checks/KafkaMessageCheck.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.KafkaCheck 8 | import ru.tinkoff.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/ru/tinkoff/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderPropertiesStep.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.protocol; 2 | 3 | import java.util.Map; 4 | import static scala.jdk.javaapi.CollectionConverters.asScala; 5 | 6 | public class KafkaProtocolBuilderPropertiesStep { 7 | 8 | private final String topic; 9 | private Map props; 10 | 11 | public KafkaProtocolBuilderPropertiesStep(String topic, Map props){ 12 | this.topic = topic; 13 | this.props = props; 14 | } 15 | 16 | public KafkaProtocolBuilder properties(Map props) { 17 | this.props = props; 18 | scala.collection.immutable.Map scalaMap = scala.collection.immutable.Map.from(asScala(this.props)); 19 | return new KafkaProtocolBuilder( 20 | ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilderPropertiesStep.apply(this.topic, scalaMap) 21 | .properties(scalaMap) 22 | ); 23 | } 24 | 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/RequestReplyBuilder.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.request.builder; 2 | 3 | import io.gatling.javaapi.core.ActionBuilder; 4 | import ru.tinkoff.gatling.kafka.javaapi.checks.KafkaChecks; 5 | 6 | import java.util.Arrays; 7 | import java.util.List; 8 | 9 | public class RequestReplyBuilder implements ActionBuilder { 10 | 11 | private ru.tinkoff.gatling.kafka.actions.KafkaRequestReplyActionBuilder wrapped; 12 | 13 | public RequestReplyBuilder(ru.tinkoff.gatling.kafka.actions.KafkaRequestReplyActionBuilder wrapped) { 14 | this.wrapped = wrapped; 15 | } 16 | 17 | public RequestReplyBuilder check(Object... checks) { 18 | return check(Arrays.asList(checks)); 19 | } 20 | 21 | public RequestReplyBuilder check(List checks) { 22 | this.wrapped = wrapped.check(KafkaChecks.toScalaChecks(checks)); 23 | return this; 24 | } 25 | 26 | @Override 27 | public io.gatling.core.action.builder.ActionBuilder asScala() { 28 | return wrapped; 29 | } 30 | } -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KafkaProtocolBuilderNew.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.protocol; 2 | 3 | import io.gatling.core.protocol.Protocol; 4 | import io.gatling.javaapi.core.ProtocolBuilder; 5 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage; 6 | import scala.Function1; 7 | 8 | public class KafkaProtocolBuilderNew implements ProtocolBuilder { 9 | 10 | private ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilderNew wrapped; 11 | 12 | public KafkaProtocolBuilderNew(ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilderNew wrapped) { 13 | this.wrapped = wrapped; 14 | } 15 | 16 | public KafkaProtocolBuilderNew matchByValue() { 17 | this.wrapped = wrapped.matchByValue(); 18 | return this; 19 | } 20 | 21 | public KafkaProtocolBuilderNew 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 | 33 | 34 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/LowPriorSender.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import io.gatling.core.session.Expression 4 | import org.apache.kafka.common.header.Headers 5 | 6 | trait LowPriorSender { 7 | implicit def noSchemaSender[K, V]: Sender[K, V] = 8 | new Sender[K, V] { 9 | 10 | override def send(requestName: Expression[String], payload: Expression[V]): RequestBuilder[Nothing, V] = 11 | KafkaRequestBuilder[Nothing, V](KafkaAttributes(requestName, None, payload, None)) 12 | 13 | override def send( 14 | requestName: Expression[String], 15 | key: Option[Expression[K]], 16 | payload: Expression[V], 17 | ): RequestBuilder[K, V] = 18 | KafkaRequestBuilder[K, V](KafkaAttributes(requestName, key, payload, None)) 19 | 20 | override def send( 21 | requestName: Expression[String], 22 | key: Option[Expression[K]], 23 | payload: Expression[V], 24 | headers: Option[Expression[Headers]], 25 | ): RequestBuilder[K, V] = 26 | KafkaRequestBuilder[K, V](KafkaAttributes(requestName, key, payload, headers)) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaRequestActionBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.apache.kafka.clients.producer.KafkaProducer 7 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 8 | import ru.tinkoff.gatling.kafka.request.builder.KafkaAttributes 9 | 10 | import scala.jdk.CollectionConverters._ 11 | 12 | class KafkaRequestActionBuilder[K, V](attr: KafkaAttributes[K, V]) extends ActionBuilder { 13 | 14 | override def build(ctx: ScenarioContext, next: Action): Action = { 15 | 16 | import ctx._ 17 | 18 | val kafkaComponents = 19 | protocolComponentsRegistry.components(KafkaProtocol.kafkaProtocolKey) 20 | 21 | val producer = new KafkaProducer[K, V](kafkaComponents.kafkaProtocol.producerProperties.asJava) 22 | 23 | coreComponents.actorSystem.registerOnTermination(producer.close()) 24 | 25 | new KafkaRequestAction( 26 | producer, 27 | attr, 28 | coreComponents, 29 | kafkaComponents.kafkaProtocol, 30 | throttled, 31 | next, 32 | ) 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /.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 -------------------------------------------------------------------------------- /src/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples/ProducerSimulation.kt: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.* 7 | import java.time.Duration 8 | 9 | class ProducerSimulation : Simulation() { 10 | 11 | private val kafkaConsumerConf = kafka().topic("test.topic") 12 | .properties(mapOf(ProducerConfig.ACKS_CONFIG to "1", 13 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG to "localhost:9092", 14 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG to "org.apache.kafka.common.serialization.StringSerializer", 15 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG to "org.apache.kafka.common.serialization.StringSerializer" 16 | )) 17 | 18 | private val scn = scenario("Basic") 19 | .exec(kafka("BasicRequest").send("foo")) 20 | .exec(kafka("dld").send("true", "12.0")) 21 | 22 | init { 23 | setUp( 24 | scn.injectOpen(atOnceUsers(1)) 25 | ) 26 | .protocols(kafkaConsumerConf) 27 | .maxDuration(Duration.ofSeconds(120)) 28 | } 29 | 30 | } -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaRequestReplyActionBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.actions 2 | 3 | import com.softwaremill.quicklens.ModifyPimp 4 | import io.gatling.core.action.Action 5 | import io.gatling.core.action.builder.ActionBuilder 6 | import io.gatling.core.structure.ScenarioContext 7 | import ru.tinkoff.gatling.kafka.KafkaCheck 8 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 9 | import ru.tinkoff.gatling.kafka.request.builder.KafkaRequestReplyAttributes 10 | 11 | import scala.reflect.ClassTag 12 | 13 | case class KafkaRequestReplyActionBuilder[K: ClassTag, V: ClassTag](attributes: KafkaRequestReplyAttributes[K, V]) 14 | extends ActionBuilder { 15 | def check(checks: KafkaCheck*): KafkaRequestReplyActionBuilder[K, V] = 16 | this.modify(_.attributes.checks).using(_ ::: checks.toList) 17 | 18 | override def build(ctx: ScenarioContext, next: Action): Action = { 19 | val kafkaComponents = ctx.protocolComponentsRegistry.components(KafkaProtocol.kafkaProtocolKey) 20 | new KafkaRequestReplyAction[K, V]( 21 | kafkaComponents, 22 | attributes, 23 | ctx.coreComponents.statsEngine, 24 | ctx.coreComponents.clock, 25 | next, 26 | ctx.coreComponents.throttler, 27 | ) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/ru/tinkoff/gatling/kafka/examples/ProducerSimulation.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.Predef._ 7 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 8 | 9 | class ProducerSimulation extends Simulation { 10 | 11 | val kafkaConsumerConf: KafkaProtocol = 12 | kafka 13 | .topic("test.topic") 14 | .properties( 15 | Map( 16 | ProducerConfig.ACKS_CONFIG -> "1", 17 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 18 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.DoubleSerializer", 19 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 20 | ), 21 | ) 22 | 23 | val scn: ScenarioBuilder = scenario("Basic") 24 | .exec( 25 | kafka("BasicRequest") 26 | .send[Double](1.16423), 27 | ) 28 | .exec(kafka("BasicRequestWithKey").send[String, Double]("true", 12.0)) 29 | 30 | setUp(scn.inject(atOnceUsers(5))).protocols(kafkaConsumerConf) 31 | 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaRequestAvro4sActionBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.core.util.NameGen 7 | import org.apache.avro.generic.GenericRecord 8 | import org.apache.kafka.clients.producer.KafkaProducer 9 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 10 | import ru.tinkoff.gatling.kafka.request.builder.Avro4sAttributes 11 | 12 | import scala.jdk.CollectionConverters._ 13 | 14 | class KafkaRequestAvro4sActionBuilder[K, V](attr: Avro4sAttributes[K, V]) extends ActionBuilder with NameGen { 15 | override def build(ctx: ScenarioContext, next: Action): Action = { 16 | import ctx._ 17 | 18 | val kafkaComponents: KafkaProtocol.Components = protocolComponentsRegistry.components(KafkaProtocol.kafkaProtocolKey) 19 | 20 | val producer = new KafkaProducer[K, GenericRecord](kafkaComponents.kafkaProtocol.producerProperties.asJava) 21 | 22 | coreComponents.actorSystem.registerOnTermination(producer.close()) 23 | 24 | new KafkaAvro4sRequestAction( 25 | producer, 26 | attr, 27 | coreComponents, 28 | kafkaComponents.kafkaProtocol, 29 | throttled, 30 | next, 31 | ) 32 | 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/KafkaDsl.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi; 2 | 3 | import static io.gatling.javaapi.core.internal.Expressions.*; 4 | 5 | import io.gatling.core.check.CheckBuilder; 6 | import org.apache.avro.generic.GenericRecord; 7 | import ru.tinkoff.gatling.kafka.javaapi.checks.KafkaChecks; 8 | import ru.tinkoff.gatling.kafka.javaapi.protocol.*; 9 | import ru.tinkoff.gatling.kafka.javaapi.request.builder.*; 10 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage; 11 | import scala.Function1; 12 | 13 | public final class KafkaDsl { 14 | 15 | public static KafkaProtocolBuilderBase kafka() { 16 | return new KafkaProtocolBuilderBase(); 17 | } 18 | 19 | public static KafkaRequestBuilderBase kafka(String requestName) { 20 | return new KafkaRequestBuilderBase(ru.tinkoff.gatling.kafka.Predef.kafka(toStringExpression(requestName)), requestName); 21 | } 22 | 23 | public static KafkaChecks.KafkaCheckTypeWrapper simpleCheck(Function1 f) { 24 | return new KafkaChecks.KafkaCheckTypeWrapper(new KafkaChecks.SimpleChecksScala().simpleCheck(f.andThen(Boolean::valueOf))); 25 | } 26 | 27 | public static CheckBuilder.Find avroBody() { 28 | return new KafkaChecks.SimpleChecksScala().avroBody(ru.tinkoff.gatling.kafka.javaapi.checks.KafkaChecks.avroSerde()); 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/client/KafkaSender.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.client 2 | 3 | import org.apache.kafka.clients.producer.{KafkaProducer, Producer, RecordMetadata} 4 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 5 | 6 | import scala.concurrent.{ExecutionContext, Future} 7 | import scala.jdk.CollectionConverters._ 8 | import scala.util.{Failure, Success} 9 | 10 | trait KafkaSender { 11 | def send(protocolMessage: KafkaProtocolMessage)( 12 | onSuccess: RecordMetadata => Unit, 13 | onFailure: Throwable => Unit, 14 | ): Unit 15 | def close(): Unit 16 | } 17 | 18 | object KafkaSender { 19 | private final class Impl(producer: Producer[Array[Byte], Array[Byte]])(implicit ec: ExecutionContext) extends KafkaSender { 20 | override def send( 21 | protocolMessage: KafkaProtocolMessage, 22 | )(onSuccess: RecordMetadata => Unit, onFailure: Throwable => Unit): Unit = { 23 | Future(producer.send(protocolMessage.toProducerRecord).get()).onComplete { 24 | case Success(value) => onSuccess(value) 25 | case Failure(exception) => onFailure(exception) 26 | } 27 | 28 | } 29 | 30 | override def close(): Unit = 31 | producer.close() 32 | 33 | } 34 | 35 | def apply(producerSettings: Map[String, AnyRef])(implicit ec: ExecutionContext): KafkaSender = { 36 | val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerSettings.asJava) 37 | new Impl(producer) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /project/Dependencies.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | 3 | object Dependencies { 4 | private object Versions { 5 | val kafka = "7.4.0-ce" 6 | val gatling = "3.9.5" 7 | val avro4s = "4.1.1" 8 | val avro = "1.11.1" 9 | } 10 | 11 | lazy val gatling: Seq[ModuleID] = Seq( 12 | "io.gatling" % "gatling-core" % Versions.gatling % "provided", 13 | "io.gatling" % "gatling-core-java" % Versions.gatling % "provided", 14 | ) 15 | 16 | lazy val gatlingTest: Seq[ModuleID] = Seq( 17 | "io.gatling.highcharts" % "gatling-charts-highcharts" % Versions.gatling % "it,test", 18 | "io.gatling" % "gatling-test-framework" % Versions.gatling % "it,test", 19 | ) 20 | 21 | lazy val kafka: Seq[ModuleID] = Seq( 22 | ("org.apache.kafka" % "kafka-clients" % Versions.kafka) 23 | .exclude("org.slf4j", "slf4j-api"), 24 | ("org.apache.kafka" %% "kafka-streams-scala" % Versions.kafka) 25 | .exclude("org.slf4j", "slf4j-api"), 26 | ) 27 | 28 | lazy val avro4s: ModuleID = "com.sksamuel.avro4s" %% "avro4s-core" % Versions.avro4s % "provided" 29 | 30 | lazy val avroCompiler: ModuleID = "org.apache.avro" % "avro-compiler" % Versions.avro 31 | lazy val avroCore: ModuleID = "org.apache.avro" % "avro" % Versions.avro 32 | lazy val avroSerdes: ModuleID = 33 | ("io.confluent" % "kafka-streams-avro-serde" % "7.4.0").exclude("org.apache.kafka", "kafka-streams-scala") 34 | lazy val avroSerializers: ModuleID = "io.confluent" % "kafka-avro-serializer" % "7.4.0" 35 | 36 | } 37 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/checks/AvroBodyCheckBuilder.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.KafkaCheck 10 | import ru.tinkoff.gatling.kafka.checks.KafkaCheckMaterializer.KafkaMessageCheckType 11 | import ru.tinkoff.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 <: GenericRecord: 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.outputTopic, prepared.value))).toValidation 24 | } 25 | }.expressionSuccess 26 | 27 | new Find.Default[KafkaMessageCheckType, KafkaProtocolMessage, T](tExtractor, displayActualValue = true) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/ru/tinkoff/gatling/kafka/examples/KafkaJavaapiMethodsGatlingTest.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.apache.kafka.common.header.internals.RecordHeaders 7 | import ru.tinkoff.gatling.kafka.javaapi.KafkaDsl._ 8 | 9 | class KafkaJavaapiMethodsGatlingTest extends Simulation { 10 | 11 | val kafkaConfwoKey = kafka 12 | .topic("myTopic3") 13 | .properties( 14 | java.util.Map.of( 15 | ProducerConfig.ACKS_CONFIG, 16 | "1", 17 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 18 | "localhost:9093", 19 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 20 | "org.apache.kafka.common.serialization.StringSerializer", 21 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 22 | "org.apache.kafka.common.serialization.StringSerializer", 23 | ), 24 | ) 25 | .protocol() 26 | 27 | setUp( 28 | scenario("Request String without key") 29 | .exec( 30 | kafka("Request String without headers and key") 31 | .send("testJavaWithoutKeyAndHeaders") 32 | .asScala(), 33 | ) 34 | .exec( 35 | kafka("Request String with headers without key") 36 | .send("testJavaWithHeadersWithoutKey", new RecordHeaders().add("test-header", "test_value".getBytes())) 37 | .asScala(), 38 | ) 39 | .inject(nothingFor(1), atOnceUsers(1)) 40 | .protocols(kafkaConfwoKey), 41 | ) 42 | 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/KafkaDsl.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.checks.KafkaCheckSupport 8 | import ru.tinkoff.gatling.kafka.protocol.{KafkaProtocol, KafkaProtocolBuilder, KafkaProtocolBuilderNew} 9 | import ru.tinkoff.gatling.kafka.request.KafkaSerdesImplicits 10 | import ru.tinkoff.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 | val kafkaConsumer: KafkaProtocolBuilderNew.type = KafkaProtocolBuilderNew 19 | 20 | def kafka(requestName: Expression[String]): KafkaRequestBuilderBase = 21 | KafkaRequestBuilderBase(requestName) 22 | 23 | implicit def kafkaProtocolBuilder2kafkaProtocol(builder: KafkaProtocolBuilder): KafkaProtocol = builder.build 24 | 25 | implicit def kafkaProtocolBuilderNew2kafkaProtocol(builder: KafkaProtocolBuilderNew): 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/java/ru/tinkoff/gatling/kafka/javaapi/protocol/KPConsumeSettingsStep.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.protocol; 2 | 3 | import java.time.Duration; 4 | import java.util.Map; 5 | 6 | import static scala.jdk.javaapi.CollectionConverters.asScala; 7 | import scala.jdk.javaapi.DurationConverters; 8 | 9 | public class KPConsumeSettingsStep { 10 | 11 | private final Map producerSettings; 12 | private final Map consumeSettings; 13 | 14 | public KPConsumeSettingsStep(Map producerSettings, Map consumeSettings) { 15 | this.producerSettings = producerSettings; 16 | this.consumeSettings = consumeSettings; 17 | } 18 | 19 | public KafkaProtocolBuilderNew timeout(Duration timeout) { 20 | scala.collection.immutable.Map ps = scala.collection.immutable.Map.from(asScala(this.producerSettings)); 21 | scala.collection.immutable.Map cs = scala.collection.immutable.Map.from(asScala(this.consumeSettings)); 22 | return new KafkaProtocolBuilderNew(ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilderNew.apply(ps, cs, DurationConverters.toScala(timeout), ru.tinkoff.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher$.MODULE$)); 23 | } 24 | 25 | public KafkaProtocolBuilderNew withDefaultTimeout() { 26 | scala.collection.immutable.Map ps = scala.collection.immutable.Map.from(asScala(this.producerSettings)); 27 | scala.collection.immutable.Map cs = scala.collection.immutable.Map.from(asScala(this.consumeSettings)); 28 | return new KafkaProtocolBuilderNew(ru.tinkoff.gatling.kafka.protocol.KafkaProtocolBuilderNew.apply(ps, cs, DurationConverters.toScala(Duration.ofSeconds(60)), ru.tinkoff.gatling.kafka.protocol.KafkaProtocol.KafkaKeyMatcher$.MODULE$)); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/test/scala/ru/tinkoff/gatling/kafka/examples/BasicSimulation.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.apache.kafka.common.header.Headers 8 | import org.apache.kafka.common.header.internals.RecordHeaders 9 | import ru.tinkoff.gatling.kafka.Predef._ 10 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 11 | 12 | import java.util.concurrent.atomic.AtomicInteger 13 | import scala.concurrent.duration.DurationInt 14 | 15 | class BasicSimulation extends Simulation { 16 | 17 | val kafkaConf: KafkaProtocol = kafka 18 | .topic("test.topic") 19 | .properties(Map(ProducerConfig.ACKS_CONFIG -> "1")) 20 | 21 | val kafkaProtocolC: KafkaProtocol = kafka.requestReply 22 | .producerSettings( 23 | Map( 24 | ProducerConfig.ACKS_CONFIG -> "1", 25 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 26 | ), 27 | ) 28 | .consumeSettings( 29 | Map( 30 | "bootstrap.servers" -> "localhost:9092", 31 | ), 32 | ) 33 | .timeout(5.seconds) 34 | val c = new AtomicInteger(0) 35 | val feeder: Feeder[Int] = Iterator.continually(Map("kekey" -> c.incrementAndGet())) 36 | 37 | val headers: Headers = new RecordHeaders().add("test-header", "test_value".getBytes) 38 | 39 | val scn: ScenarioBuilder = scenario("Basic") 40 | .feed(feeder) 41 | .exec( 42 | kafka("ReqRep").requestReply 43 | .requestTopic("test.t") 44 | .replyTopic("test.t") 45 | .send[String, String]("#{kekey}", """{ "m": "dkf" }""", headers) 46 | .check(jsonPath("$.m").is("dkf")), 47 | ) 48 | 49 | setUp(scn.inject(atOnceUsers(5))).protocols(kafkaProtocolC).maxDuration(120.seconds) 50 | 51 | } 52 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/Sender.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.request.builder 2 | 3 | import com.sksamuel.avro4s.{FromRecord, RecordFormat, SchemaFor} 4 | import io.gatling.core.session.Expression 5 | import org.apache.kafka.common.header.Headers 6 | 7 | trait Sender[K, V] { 8 | 9 | def send(requestName: Expression[String], payload: Expression[V]): RequestBuilder[Nothing, V] 10 | 11 | def send(requestName: Expression[String], key: Option[Expression[K]], payload: Expression[V]): RequestBuilder[K, V] 12 | 13 | def send( 14 | requestName: Expression[String], 15 | key: Option[Expression[K]], 16 | payload: Expression[V], 17 | headers: Option[Expression[Headers]], 18 | ): RequestBuilder[K, V] 19 | 20 | } 21 | 22 | object Sender extends LowPriorSender { 23 | 24 | implicit def Avro4sSender[K, V](implicit 25 | schema: SchemaFor[V], 26 | format: RecordFormat[V], 27 | fromRecord: FromRecord[V], 28 | headers: Headers, 29 | ): Sender[K, V] = new Sender[K, V] { 30 | 31 | override def send(requestName: Expression[String], payload: Expression[V]): RequestBuilder[Nothing, V] = 32 | new KafkaAvro4sRequestBuilder[Nothing, V](Avro4sAttributes(requestName, None, payload, schema, format, fromRecord, None)) 33 | 34 | override def send( 35 | requestName: Expression[String], 36 | key: Option[Expression[K]], 37 | payload: Expression[V], 38 | ): RequestBuilder[K, V] = 39 | new KafkaAvro4sRequestBuilder[K, V](Avro4sAttributes(requestName, key, payload, schema, format, fromRecord, None)) 40 | 41 | override def send( 42 | requestName: Expression[String], 43 | key: Option[Expression[K]], 44 | payload: Expression[V], 45 | headers: Option[Expression[Headers]], 46 | ): RequestBuilder[K, V] = 47 | new KafkaAvro4sRequestBuilder[K, V](Avro4sAttributes(requestName, key, payload, schema, format, fromRecord, headers)) 48 | } 49 | 50 | } 51 | -------------------------------------------------------------------------------- /src/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples/BasicSimulation.kt: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.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/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.kt: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.javaapi.* 7 | import ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.* 8 | import ru.tinkoff.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/ru/tinkoff/gatling/kafka/examples/Avro4sSimulation.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.examples 2 | 3 | import com.sksamuel.avro4s._ 4 | import io.gatling.core.Predef._ 5 | import io.gatling.core.structure.ScenarioBuilder 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 ru.tinkoff.gatling.kafka.Predef._ 10 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 11 | 12 | class Avro4sSimulation extends Simulation { 13 | 14 | val kafkaAclConf: KafkaProtocol = kafka 15 | .topic("my.acl.topic") 16 | .properties( 17 | Map( 18 | ProducerConfig.ACKS_CONFIG -> "1", 19 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9092", 20 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 21 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 22 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 23 | "schema.registry.url" -> "http://localhost:9094", 24 | ), 25 | ) 26 | 27 | case class Ingredient(name: String, sugar: Double, fat: Double) 28 | 29 | implicit val ingridientToRecord: ToRecord[Ingredient] = ToRecord.apply 30 | implicit val ingridientFromRecord: FromRecord[Ingredient] = FromRecord.apply 31 | implicit val ingridientSchemaFor: SchemaFor[Ingredient] = SchemaFor.apply 32 | implicit val ingridientFormat: RecordFormat[Ingredient] = RecordFormat.apply 33 | implicit val ingredientHeaders: Headers = new RecordHeaders() 34 | 35 | val scn: ScenarioBuilder = scenario("Kafka Test") 36 | .exec( 37 | kafka("Simple Avro4s Request") 38 | // message to send 39 | .send[Ingredient](Ingredient("Cheese", 0d, 70d)), 40 | ) 41 | .exec( 42 | kafka("Simple Avro4s Request with Key") 43 | // message to send 44 | .send[String, Ingredient]("Key", Ingredient("Cheese", 0d, 70d)), 45 | ) 46 | 47 | setUp(scn.inject(atOnceUsers(1))).protocols(kafkaAclConf) 48 | } 49 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/builder/KafkaRequestBuilderBase.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.actions.KafkaRequestReplyActionBuilder 7 | 8 | import scala.reflect.ClassTag 9 | 10 | case class KafkaRequestBuilderBase(requestName: Expression[String]) { 11 | 12 | import ru.tinkoff.gatling.kafka.Predef._ 13 | def send[K, V]( 14 | key: Expression[K], 15 | payload: Expression[V], 16 | headers: Expression[Headers] = List.empty[Header], 17 | )(implicit 18 | sender: Sender[K, V], 19 | ): RequestBuilder[K, V] = { 20 | if (key == null) 21 | sender.send(requestName, None, payload, Some(headers)) 22 | else 23 | sender.send(requestName, Some(key), payload, Some(headers)) 24 | } 25 | 26 | def send[V](payload: Expression[V])(implicit sender: Sender[Nothing, V]): RequestBuilder[_, V] = 27 | sender.send(requestName, None, payload) 28 | 29 | def requestReply: ReqRepBase.type = ReqRepBase 30 | 31 | object ReqRepBase { 32 | case class RROutTopicStep(inputTopic: Expression[String], outputTopic: Expression[String]) { 33 | def send[K: Serde: ClassTag, V: Serde: ClassTag]( 34 | key: Expression[K], 35 | payload: Expression[V], 36 | headers: Expression[Headers] = List.empty[Header].expressionSuccess, 37 | ): KafkaRequestReplyActionBuilder[K, V] = { 38 | KafkaRequestReplyActionBuilder[K, V]( 39 | new KafkaRequestReplyAttributes[K, V]( 40 | requestName, 41 | inputTopic, 42 | outputTopic, 43 | key, 44 | payload, 45 | Some(headers), 46 | implicitly[Serde[K]].serializer(), 47 | implicitly[Serde[V]].serializer(), 48 | List.empty, 49 | ), 50 | ) 51 | } 52 | } 53 | 54 | case class RRInTopicStep(inputTopic: Expression[String]) { 55 | def replyTopic(outputTopic: Expression[String]): RROutTopicStep = RROutTopicStep(inputTopic, outputTopic) 56 | } 57 | def requestTopic(rt: Expression[String]): RRInTopicStep = RRInTopicStep(rt) 58 | 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /src/test/scala/ru/tinkoff/gatling/kafka/examples/MatchSimulation.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.Predef._ 8 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 9 | import ru.tinkoff.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.requestReply 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.requestReply 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/ru/tinkoff/gatling/kafka/javaapi/examples/BasicSimulation.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.examples; 2 | 3 | import io.gatling.javaapi.core.ScenarioBuilder; 4 | import io.gatling.javaapi.core.Simulation; 5 | import org.apache.kafka.common.header.Headers; 6 | import org.apache.kafka.common.header.internals.RecordHeaders; 7 | import ru.tinkoff.gatling.kafka.javaapi.protocol.*; 8 | import org.apache.kafka.clients.producer.ProducerConfig; 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.*; 19 | import static ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.kafka; 20 | 21 | public class BasicSimulation extends Simulation { 22 | 23 | private final KafkaProtocolBuilder kafkaConf = kafka() 24 | .topic("test.topic") 25 | .properties(Map.of(ProducerConfig.ACKS_CONFIG, "1")); 26 | 27 | private final KafkaProtocolBuilderNew kafkaProtocolC = kafka().requestReply() 28 | .producerSettings( 29 | Map.of( 30 | ProducerConfig.ACKS_CONFIG, "1", 31 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 32 | ) 33 | ) 34 | .consumeSettings( 35 | Map.of("bootstrap.servers", "localhost:9092") 36 | ).timeout(Duration.ofSeconds(5)); 37 | 38 | private final AtomicInteger c = new AtomicInteger(0); 39 | 40 | private final Iterator> feeder = 41 | Stream.generate((Supplier>) () -> Collections.singletonMap("kekey", c.incrementAndGet()) 42 | ).iterator(); 43 | 44 | private final Headers headers = new RecordHeaders().add("test-header", "test_value".getBytes()); 45 | 46 | private final ScenarioBuilder scn = scenario("Basic") 47 | .feed(feeder) 48 | .exec( 49 | kafka("ReqRep").requestReply() 50 | .requestTopic("test.t") 51 | .replyTopic("test.t") 52 | .send("#{kekey}", """ 53 | { "m": "dkf" } 54 | """, headers, String.class, String.class) 55 | .check(jsonPath("$.m").is("dkf")) 56 | ); 57 | 58 | { 59 | setUp(scn.injectOpen(atOnceUsers(5))).protocols(kafkaProtocolC).maxDuration(Duration.ofSeconds(120)); 60 | } 61 | 62 | } 63 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/protocol/KafkaProtocolBuilderNew.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.protocol 2 | 3 | import io.gatling.core.session.Expression 4 | import org.apache.kafka.clients.producer.ProducerConfig 5 | import org.apache.kafka.common.serialization.Serdes 6 | import org.apache.kafka.streams.StreamsConfig 7 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol._ 8 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 9 | 10 | import scala.concurrent.duration.{DurationInt, FiniteDuration} 11 | 12 | object KafkaProtocolBuilderNew { 13 | def producerSettings(ps: Map[String, AnyRef]): KPProducerSettingsStep = KPProducerSettingsStep(ps) 14 | 15 | case class KPProducerSettingsStep(producerSettings: Map[String, AnyRef]) { 16 | def consumeSettings(cs: Map[String, AnyRef]): KPConsumeSettingsStep = KPConsumeSettingsStep(producerSettings, cs) 17 | } 18 | 19 | case class KPConsumeSettingsStep(producerSettings: Map[String, AnyRef], consumeSettings: Map[String, AnyRef]) { 20 | def timeout(t: FiniteDuration): KafkaProtocolBuilderNew = KafkaProtocolBuilderNew(producerSettings, consumeSettings, t) 21 | def withDefaultTimeout: KafkaProtocolBuilderNew = KafkaProtocolBuilderNew(producerSettings, consumeSettings, 60.seconds) 22 | } 23 | } 24 | 25 | case class KafkaProtocolBuilderNew( 26 | producerSettings: Map[String, AnyRef], 27 | consumeSettings: Map[String, AnyRef], 28 | timeout: FiniteDuration, 29 | messageMatcher: KafkaMatcher = KafkaKeyMatcher, 30 | ) extends { 31 | 32 | def matchByValue: KafkaProtocolBuilderNew = 33 | messageMatcher(KafkaValueMatcher) 34 | 35 | def matchByMessage(keyExtractor: KafkaProtocolMessage => Array[Byte]): KafkaProtocolBuilderNew = 36 | messageMatcher(KafkaMessageMatcher(keyExtractor)) 37 | 38 | private def messageMatcher(matcher: KafkaMatcher): KafkaProtocolBuilderNew = 39 | copy(messageMatcher = matcher) 40 | 41 | def build: KafkaProtocol = { 42 | 43 | val serializers = Map( 44 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 45 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 46 | ) 47 | 48 | val consumeDefaults = Map( 49 | StreamsConfig.APPLICATION_ID_CONFIG -> s"gatling-test-${java.util.UUID.randomUUID()}", 50 | StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG -> Serdes.ByteArray().getClass.getName, 51 | StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG -> Serdes.ByteArray().getClass.getName, 52 | ) 53 | 54 | KafkaProtocol("test", producerSettings ++ serializers, consumeDefaults ++ consumeSettings, timeout, messageMatcher) 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/checks/KafkaMessagePreparer.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.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), bodyCharset) 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/java/ru/tinkoff/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.examples; 2 | 3 | import io.gatling.javaapi.core.*; 4 | import org.apache.kafka.clients.producer.ProducerConfig; 5 | import ru.tinkoff.gatling.kafka.javaapi.protocol.KafkaProtocolBuilderNew; 6 | import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; 7 | import io.confluent.kafka.serializers.*; 8 | import org.apache.kafka.common.serialization.*; 9 | import ru.tinkoff.gatling.kafka.javaapi.request.builder.RequestReplyBuilder; 10 | 11 | import java.time.Duration; 12 | import java.util.*; 13 | 14 | import static io.gatling.javaapi.core.CoreDsl.*; 15 | import static ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.kafka; 16 | 17 | public class AvroClassWithRequestReplySimulation extends Simulation { 18 | 19 | // example of using custom serde 20 | public static Serializer ser = 21 | (Serializer) new KafkaAvroSerializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 22 | public static Deserializer de = 23 | (Deserializer) new KafkaAvroDeserializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 24 | 25 | // protocol 26 | private final KafkaProtocolBuilderNew kafkaProtocolRRAvro = kafka().requestReply() 27 | .producerSettings( 28 | Map.of( 29 | ProducerConfig.ACKS_CONFIG, "1", 30 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9093", 31 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer", 32 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer", 33 | // schema registry url is required for KafkaAvroSerializer and KafkaAvroDeserializer 34 | "schema.registry.url", "http://localhost:9094" 35 | ) 36 | ) 37 | .consumeSettings( 38 | Map.of("bootstrap.servers", "localhost:9093") 39 | ) 40 | .timeout(Duration.ofSeconds(5)); 41 | 42 | // message 43 | public static RequestReplyBuilder kafkaMessage = kafka("RequestReply").requestReply() 44 | .requestTopic("request.t") 45 | .replyTopic("reply.t") 46 | .send("key", new MyAvroClass(), String.class, MyAvroClass.class, ser, de); 47 | 48 | // simulation 49 | { 50 | setUp(scenario("Kafka RequestReply Avro").exec(kafkaMessage).injectOpen(atOnceUsers(1))).protocols(kafkaProtocolRRAvro); 51 | } 52 | 53 | private static class MyAvroClass { 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/request/KafkaSerdesImplicits.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 | 10 | import java.nio.ByteBuffer 11 | import java.util.UUID 12 | 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 | } 48 | -------------------------------------------------------------------------------- /src/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples/MatchSimulation.kt: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.javaapi.* 7 | import ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.* 8 | import ru.tinkoff.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/test/scala/ru/tinkoff/gatling/kafka/examples/AvroClassWithRequestReplySimulation.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.Predef._ 9 | import ru.tinkoff.gatling.kafka.actions.KafkaRequestReplyActionBuilder 10 | import ru.tinkoff.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.requestReply 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/main/scala/ru/tinkoff/gatling/kafka/checks/KafkaCheckMaterializer.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.KafkaCheck 18 | import ru.tinkoff.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/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/KafkaRequestBuilderBase.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 scala.Function1; 8 | 9 | import static io.gatling.javaapi.core.internal.Expressions.*; 10 | import static io.gatling.javaapi.core.internal.Expressions.toStaticValueExpression; 11 | 12 | public class KafkaRequestBuilderBase { 13 | 14 | private final ru.tinkoff.gatling.kafka.request.builder.KafkaRequestBuilderBase wrapped; 15 | private final String requestName; 16 | 17 | private Function1> calculateExpression(T obj) { 18 | Function1> expression; 19 | 20 | if (obj instanceof String || obj.getClass().isPrimitive() || obj instanceof CharSequence || obj instanceof byte[]) { 21 | expression = toExpression(obj.toString(), obj.getClass()); 22 | } else { 23 | expression = toStaticValueExpression(obj); 24 | } 25 | return expression; 26 | } 27 | 28 | public KafkaRequestBuilderBase(ru.tinkoff.gatling.kafka.request.builder.KafkaRequestBuilderBase wrapped, String requestName) { 29 | this.wrapped = wrapped; 30 | this.requestName = requestName; 31 | } 32 | 33 | public RequestBuilder send(K key, V payload) { 34 | return new RequestBuilder<>( 35 | wrapped.send( 36 | calculateExpression(key), 37 | calculateExpression(payload), 38 | toStaticValueExpression(new RecordHeaders()), 39 | ru.tinkoff.gatling.kafka.request.builder.Sender.noSchemaSender() 40 | )); 41 | } 42 | 43 | public RequestBuilder send(K key, V payload, Headers headers) { 44 | return new RequestBuilder<>( 45 | wrapped.send( 46 | calculateExpression(key), 47 | calculateExpression(payload), 48 | toStaticValueExpression(headers), 49 | ru.tinkoff.gatling.kafka.request.builder.Sender.noSchemaSender() 50 | )); 51 | } 52 | 53 | public RequestBuilder send(V payload) { 54 | return new RequestBuilder<>(wrapped.send( 55 | calculateExpression(payload), 56 | ru.tinkoff.gatling.kafka.request.builder.Sender.noSchemaSender())); 57 | } 58 | 59 | public RequestBuilder send(V payload, Headers headers) { 60 | return new RequestBuilder<>( 61 | wrapped.send( 62 | null, 63 | calculateExpression(payload), 64 | toStaticValueExpression(headers), 65 | ru.tinkoff.gatling.kafka.request.builder.Sender.noSchemaSender() 66 | )); 67 | } 68 | 69 | public ReqRepBase requestReply() { 70 | return new ReqRepBase(requestName); 71 | } 72 | 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaAvro4sRequestAction.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.actions 2 | 3 | import io.gatling.commons.stats.{KO, OK} 4 | import io.gatling.commons.util.DefaultClock 5 | import io.gatling.commons.validation.Validation 6 | import io.gatling.core.CoreComponents 7 | import io.gatling.core.action.{Action, ExitableAction} 8 | import io.gatling.core.session.Session 9 | import io.gatling.core.stats.StatsEngine 10 | import io.gatling.core.util.NameGen 11 | import org.apache.avro.generic.GenericRecord 12 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} 13 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 14 | import ru.tinkoff.gatling.kafka.request.builder.Avro4sAttributes 15 | 16 | class KafkaAvro4sRequestAction[K, V]( 17 | val producer: KafkaProducer[K, GenericRecord], 18 | val attr: Avro4sAttributes[K, V], 19 | val coreComponents: CoreComponents, 20 | val kafkaProtocol: KafkaProtocol, 21 | val throttled: Boolean, 22 | val next: Action, 23 | ) extends ExitableAction with NameGen { 24 | 25 | val statsEngine: StatsEngine = coreComponents.statsEngine 26 | val clock = new DefaultClock 27 | override val name: String = genName("kafkaAvroRequest") 28 | 29 | override def execute(session: Session): Unit = recover(session) { 30 | attr requestName session flatMap { requestName => 31 | val outcome = sendRequest(requestName, producer, attr, throttled, session) 32 | 33 | outcome.onFailure(errorMessage => 34 | statsEngine.reportUnbuildableRequest(session.scenario, session.groups, requestName, errorMessage), 35 | ) 36 | 37 | outcome 38 | } 39 | } 40 | 41 | def sendRequest( 42 | requestName: String, 43 | producer: KafkaProducer[K, GenericRecord], 44 | attr: Avro4sAttributes[K, V], 45 | throttled: Boolean, 46 | session: Session, 47 | ): Validation[Unit] = { 48 | 49 | attr payload session map { payload => 50 | val headers = attr.headers 51 | .map(h => h(session).toOption.get) 52 | .orNull 53 | val key = attr.key 54 | .map(k => k(session).toOption.get) 55 | .getOrElse(null.asInstanceOf[K]) 56 | 57 | val record: ProducerRecord[K, GenericRecord] = 58 | new ProducerRecord(kafkaProtocol.producerTopic, null, key, attr.format.to(payload), headers) 59 | 60 | val requestStartDate = clock.nowMillis 61 | 62 | producer.send( 63 | record, 64 | (_: RecordMetadata, e: Exception) => { 65 | 66 | val requestEndDate = clock.nowMillis 67 | statsEngine.logResponse( 68 | session.scenario, 69 | session.groups, 70 | requestName, 71 | requestStartDate, 72 | requestEndDate, 73 | if (e == null) OK else KO, 74 | None, 75 | if (e == null) None else Some(e.getMessage), 76 | ) 77 | 78 | coreComponents.throttler match { 79 | case Some(th) if throttled => th.throttle(session.scenario, () => next ! session) 80 | case _ => next ! session 81 | } 82 | 83 | }, 84 | ) 85 | 86 | } 87 | 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /.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 | jobs: 11 | test: 12 | name: Test Release 13 | runs-on: ubuntu-20.04 14 | services: 15 | zookeeper: 16 | image: wurstmeister/zookeeper 17 | env: 18 | ZOO_MY_ID: "1" 19 | ZOO_PORT: "2181" 20 | ZOO_SERVERS: server.1=zoo1:2888:3888 21 | ports: 22 | - '2181:2181' 23 | kafka: 24 | image: wurstmeister/kafka:2.13-2.6.3 25 | env: 26 | KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 27 | KAFKA_ADVERTISED_HOST_NAME: kafka 28 | KAFKA_LISTENERS: BROKER://:9092,EXTERNAL://:9093 29 | KAFKA_ADVERTISED_LISTENERS: BROKER://kafka:9092,EXTERNAL://localhost:9093 30 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: BROKER:PLAINTEXT,EXTERNAL:PLAINTEXT 31 | KAFKA_INTER_BROKER_LISTENER_NAME: BROKER 32 | KAFKA_BROKER_ID: "1" 33 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: "1" 34 | KAFKA_CREATE_TOPICS: "myTopic1:1:1, test.t1:1:1, myTopic2:1:1, test.t2:1:1, myTopic3:1:1, test.t3:1:1" 35 | ports: 36 | - '9092:9092' 37 | - '9093:9093' 38 | schema-registry: 39 | image: confluentinc/cp-schema-registry:7.2.1 40 | env: 41 | SCHEMA_REGISTRY_HOST_NAME: schema-registry 42 | SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: 'kafka:9092,localhost:9093' 43 | SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:9094 44 | ports: 45 | - '9094:9094' 46 | 47 | steps: 48 | - name: Checkout 49 | uses: actions/checkout@v2.4.0 50 | with: 51 | fetch-depth: 0 52 | 53 | - name: Setup Scala 54 | uses: olafurpg/setup-scala@v13 55 | with: 56 | java-version: openjdk@1.17 57 | 58 | - name: Test Release 59 | run: sbt clean scalafmtCheckAll scalafmtSbtCheck compile coverage "Gatling / testOnly ru.tinkoff.gatling.kafka.examples.KafkaGatlingTest" test coverageOff 60 | 61 | - name: Test Javaapi Methods 62 | run: sbt compile "Gatling / testOnly ru.tinkoff.gatling.kafka.examples.KafkaJavaapiMethodsGatlingTest" 63 | 64 | - name: Coverage Report 65 | run: sbt coverageReport 66 | 67 | - name: Upload coverage reports to Codecov 68 | uses: codecov/codecov-action@v3 69 | 70 | publish: 71 | name: Publish Release 72 | needs: [test] 73 | if: github.event_name != 'pull_request' && (github.ref == 'refs/heads/master' || startsWith(github.ref, 'refs/tags/v')) 74 | runs-on: ubuntu-20.04 75 | steps: 76 | - name: Checkout 77 | uses: actions/checkout@v2.4.0 78 | with: 79 | fetch-depth: 0 80 | 81 | - name: Setup Scala 82 | uses: olafurpg/setup-scala@v13 83 | with: 84 | java-version: openjdk@1.17 85 | 86 | - name: Publish Release 87 | run: sbt ci-release 88 | env: 89 | PGP_PASSPHRASE: ${{ secrets.PGP_PASSPHRASE }} 90 | PGP_SECRET: ${{ secrets.PGP_SECRET }} 91 | SONATYPE_PASSWORD: ${{ secrets.SONATYPE_PASSWORD }} 92 | SONATYPE_USERNAME: ${{ secrets.SONATYPE_USERNAME }} 93 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/client/TrackersPool.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.client 2 | 3 | import akka.actor.{ActorSystem, CoordinatedShutdown} 4 | import io.gatling.commons.util.Clock 5 | import io.gatling.core.stats.StatsEngine 6 | import io.gatling.core.util.NameGen 7 | import org.apache.kafka.streams.KafkaStreams 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala.StreamsBuilder 10 | import org.apache.kafka.streams.scala.serialization.Serdes._ 11 | import ru.tinkoff.gatling.kafka.KafkaLogging 12 | import ru.tinkoff.gatling.kafka.client.KafkaMessageTrackerActor.MessageConsumed 13 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol.KafkaMatcher 14 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 15 | 16 | import java.util.concurrent.ConcurrentHashMap 17 | import scala.jdk.CollectionConverters._ 18 | 19 | class TrackersPool( 20 | streamsSettings: Map[String, AnyRef], 21 | system: ActorSystem, 22 | statsEngine: StatsEngine, 23 | clock: Clock, 24 | ) extends KafkaLogging with NameGen { 25 | 26 | private val trackers = new ConcurrentHashMap[String, KafkaMessageTracker] 27 | private val props = new java.util.Properties() 28 | props.putAll(streamsSettings.asJava) 29 | 30 | def tracker( 31 | inputTopic: String, 32 | outputTopic: String, 33 | messageMatcher: KafkaMatcher, 34 | responseTransformer: Option[KafkaProtocolMessage => KafkaProtocolMessage], 35 | ): KafkaMessageTracker = 36 | trackers.computeIfAbsent( 37 | outputTopic, 38 | _ => { 39 | val actor = 40 | system.actorOf(KafkaMessageTrackerActor.props(statsEngine, clock), genName("kafkaTrackerActor")) 41 | 42 | val builder = new StreamsBuilder 43 | 44 | builder.stream[Array[Byte], Array[Byte]](outputTopic).foreach { case (k, v) => 45 | val message = KafkaProtocolMessage(k, v, inputTopic, outputTopic) 46 | if (messageMatcher.responseMatch(message) == null) { 47 | logger.error(s"no messageMatcher key for read message") 48 | } else { 49 | if (k == null || v == null) 50 | logger.info(s" --- received message with null key or value") 51 | else 52 | logger.info(s" --- received ${new String(k)} ${new String(v)}") 53 | val receivedTimestamp = clock.nowMillis 54 | val replyId = messageMatcher.responseMatch(message) 55 | if (k != null) 56 | logMessage( 57 | s"Record received key=${new String(k)}", 58 | message, 59 | ) 60 | else 61 | logMessage( 62 | s"Record received key=null", 63 | message, 64 | ) 65 | 66 | actor ! MessageConsumed( 67 | replyId, 68 | receivedTimestamp, 69 | responseTransformer.map(_(message)).getOrElse(message), 70 | ) 71 | } 72 | } 73 | 74 | val streams = new KafkaStreams(builder.build(), props) 75 | 76 | streams.cleanUp() 77 | streams.start() 78 | 79 | CoordinatedShutdown(system).addJvmShutdownHook(streams.close()) 80 | 81 | new KafkaMessageTracker(actor) 82 | }, 83 | ) 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaRequestAction.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.actions 2 | 3 | import io.gatling.commons.stats.{KO, OK} 4 | import io.gatling.commons.util.DefaultClock 5 | import io.gatling.commons.validation.Validation 6 | import io.gatling.core.CoreComponents 7 | import io.gatling.core.action.{Action, ExitableAction} 8 | import io.gatling.core.session._ 9 | import io.gatling.core.stats.StatsEngine 10 | import io.gatling.core.util.NameGen 11 | import org.apache.kafka.clients.producer._ 12 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 13 | import ru.tinkoff.gatling.kafka.request.builder.KafkaAttributes 14 | 15 | class KafkaRequestAction[K, V]( 16 | val producer: KafkaProducer[K, V], 17 | val attr: KafkaAttributes[K, V], 18 | val coreComponents: CoreComponents, 19 | val kafkaProtocol: KafkaProtocol, 20 | val throttled: Boolean, 21 | val next: Action, 22 | ) extends ExitableAction with NameGen { 23 | 24 | override val name: String = genName("kafkaRequest") 25 | val statsEngine: StatsEngine = coreComponents.statsEngine 26 | val clock = new DefaultClock 27 | 28 | override def execute(session: Session): Unit = recover(session) { 29 | 30 | attr requestName session flatMap { requestName => 31 | val outcome = 32 | sendRequest(requestName, producer, attr, throttled, session) 33 | 34 | outcome.onFailure(errorMessage => 35 | statsEngine.reportUnbuildableRequest(session.scenario, session.groups, requestName, errorMessage), 36 | ) 37 | 38 | outcome 39 | 40 | } 41 | 42 | } 43 | 44 | private def sendRequest( 45 | requestName: String, 46 | producer: Producer[K, V], 47 | kafkaAttributes: KafkaAttributes[K, V], 48 | throttled: Boolean, 49 | session: Session, 50 | ): Validation[Unit] = { 51 | 52 | kafkaAttributes payload session map { payload => 53 | val key = kafkaAttributes.key 54 | .map(k => k(session).toOption.get) 55 | .getOrElse(null.asInstanceOf[K]) 56 | 57 | val headers = kafkaAttributes.headers 58 | .map(h => h(session).toOption.get) 59 | .orNull 60 | 61 | val record = new ProducerRecord[K, V]( 62 | kafkaProtocol.producerTopic, 63 | null, 64 | key, 65 | payload, 66 | headers, 67 | ) 68 | 69 | val requestStartDate = clock.nowMillis 70 | 71 | producer.send( 72 | record, 73 | (_: RecordMetadata, e: Exception) => { 74 | 75 | val requestEndDate = clock.nowMillis 76 | 77 | statsEngine.logResponse( 78 | session.scenario, 79 | session.groups, 80 | requestName, 81 | startTimestamp = requestStartDate, 82 | endTimestamp = requestEndDate, 83 | if (e == null) OK else KO, 84 | None, 85 | if (e == null) None else Some(e.getMessage), 86 | ) 87 | 88 | coreComponents.throttler match { 89 | case Some(th) if throttled => th.throttle(session.scenario, () => next ! session) 90 | case _ => next ! session 91 | } 92 | 93 | }, 94 | ) 95 | 96 | } 97 | 98 | } 99 | 100 | } 101 | -------------------------------------------------------------------------------- /src/test/java/ru/tinkoff/gatling/kafka/javaapi/examples/MatchSimulation.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.javaapi.protocol.KafkaProtocolBuilderNew; 7 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage; 8 | 9 | import java.time.Duration; 10 | import java.util.Collections; 11 | import java.util.Iterator; 12 | import java.util.Map; 13 | import java.util.concurrent.atomic.AtomicInteger; 14 | import java.util.function.Supplier; 15 | import java.util.stream.Stream; 16 | 17 | import static io.gatling.javaapi.core.CoreDsl.*; 18 | import static ru.tinkoff.gatling.kafka.javaapi.KafkaDsl.kafka; 19 | 20 | public class MatchSimulation extends Simulation { 21 | 22 | private final KafkaProtocolBuilderNew kafkaProtocolMatchByValue = kafka().requestReply() 23 | .producerSettings( 24 | Map.of( 25 | ProducerConfig.ACKS_CONFIG, "1", 26 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 27 | ) 28 | ) 29 | .consumeSettings( 30 | Map.of("bootstrap.servers", "localhost:9092") 31 | ) 32 | .timeout(Duration.ofSeconds(5)) 33 | // for match by message value 34 | .matchByValue(); 35 | 36 | private byte[] matchByOwnVal(KafkaProtocolMessage message) { 37 | // do something with the message and extract the values you are interested in 38 | // method is called: 39 | // - for each message which will be sent out 40 | // - for each message which has been received 41 | return "Custom Message".getBytes(); // just returning something 42 | } 43 | 44 | private final KafkaProtocolBuilderNew kafkaProtocolMatchByMessage = kafka().requestReply() 45 | .producerSettings( 46 | Map.of( 47 | ProducerConfig.ACKS_CONFIG, "1", 48 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" 49 | ) 50 | ) 51 | .consumeSettings( 52 | Map.of( 53 | "bootstrap.servers", "localhost:9092" 54 | ) 55 | ) 56 | .timeout(Duration.ofSeconds(5)) 57 | .matchByMessage(this::matchByOwnVal); 58 | 59 | private final AtomicInteger c = new AtomicInteger(0); 60 | private final Iterator> feeder = 61 | Stream.generate((Supplier>) () -> Collections.singletonMap("kekey", c.incrementAndGet()) 62 | ).iterator(); 63 | 64 | private final ScenarioBuilder scn = scenario("Basic") 65 | .feed(feeder) 66 | .exec( 67 | kafka("ReqRep").requestReply() 68 | .requestTopic("test.t") 69 | .replyTopic("test.t") 70 | .send("#{kekey}", """ 71 | { "m": "dkf" } 72 | """, String.class, String.class)); 73 | 74 | { 75 | setUp( 76 | scn.injectOpen(atOnceUsers(1))) 77 | .protocols(kafkaProtocolMatchByMessage) 78 | .maxDuration(Duration.ofSeconds(120)); 79 | } 80 | 81 | } 82 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/protocol/KafkaProtocol.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.client.{KafkaSender, TrackersPool} 7 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol.KafkaMatcher 8 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 9 | 10 | import java.util.concurrent.Executors 11 | import scala.concurrent.duration.FiniteDuration 12 | import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService} 13 | 14 | object KafkaProtocol { 15 | 16 | trait KafkaMatcher { 17 | def requestMatch(msg: KafkaProtocolMessage): Array[Byte] 18 | def responseMatch(msg: KafkaProtocolMessage): Array[Byte] 19 | } 20 | 21 | object KafkaKeyMatcher extends KafkaMatcher { 22 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.key 23 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.key 24 | } 25 | 26 | object KafkaValueMatcher extends KafkaMatcher { 27 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.value 28 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = msg.value 29 | } 30 | 31 | case class KafkaMessageMatcher(keyExtractor: KafkaProtocolMessage => Array[Byte]) extends KafkaMatcher { 32 | override def requestMatch(msg: KafkaProtocolMessage): Array[Byte] = keyExtractor(msg) 33 | override def responseMatch(msg: KafkaProtocolMessage): Array[Byte] = keyExtractor(msg) 34 | } 35 | 36 | type Components = KafkaComponents 37 | 38 | val kafkaProtocolKey: ProtocolKey[KafkaProtocol, Components] = new ProtocolKey[KafkaProtocol, Components] { 39 | override def protocolClass: Class[Protocol] = 40 | classOf[KafkaProtocol].asInstanceOf[Class[Protocol]] 41 | 42 | override def defaultProtocolValue(configuration: GatlingConfiguration): KafkaProtocol = 43 | throw new IllegalStateException("Can't provide a default value for KafkaProtocol") 44 | 45 | override def newComponents(coreComponents: CoreComponents): KafkaProtocol => KafkaComponents = 46 | kafkaProtocol => { 47 | val blockingPool = Executors.newCachedThreadPool() 48 | implicit val ec: ExecutionContextExecutorService = ExecutionContext.fromExecutorService(blockingPool) 49 | 50 | val sender = KafkaSender(kafkaProtocol.producerProperties) 51 | val trackersPool = new TrackersPool( 52 | kafkaProtocol.consumeProperties, 53 | coreComponents.actorSystem, 54 | coreComponents.statsEngine, 55 | coreComponents.clock, 56 | ) 57 | 58 | KafkaComponents(coreComponents, kafkaProtocol, trackersPool, sender) 59 | } 60 | } 61 | } 62 | 63 | case class KafkaProtocol( 64 | producerTopic: String, 65 | producerProperties: Map[String, AnyRef], 66 | consumeProperties: Map[String, AnyRef], 67 | timeout: FiniteDuration, 68 | messageMatcher: KafkaMatcher, 69 | ) extends Protocol { 70 | 71 | def topic(t: String): KafkaProtocol = copy(producerTopic = t) 72 | 73 | def properties(properties: Map[String, AnyRef]): KafkaProtocol = 74 | copy(producerProperties = properties) 75 | 76 | def producerProperties(properties: Map[String, AnyRef]): KafkaProtocol = copy(producerProperties = properties) 77 | def consumeProperties(properties: Map[String, AnyRef]): KafkaProtocol = copy(consumeProperties = properties) 78 | def timeout(t: FiniteDuration): KafkaProtocol = copy(timeout = t) 79 | 80 | } 81 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/checks/KafkaChecks.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.javaapi.checks 2 | 3 | import io.gatling.core.check.Check 4 | import io.gatling.core.check._ 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.javaapi.core.internal.CoreCheckType 12 | import ru.tinkoff.gatling.kafka.checks.{KafkaCheckMaterializer, KafkaCheckSupport} 13 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 14 | import ru.tinkoff.gatling.kafka.{KafkaCheck, checks} 15 | import net.sf.saxon.s9api.XdmNode 16 | import com.fasterxml.jackson.databind.JsonNode 17 | import io.confluent.kafka.streams.serdes.avro.GenericAvroSerde 18 | import org.apache.avro.generic.GenericRecord 19 | import org.apache.kafka.common.serialization.Serde 20 | 21 | import java.{util => ju} 22 | import scala.jdk.CollectionConverters._ 23 | 24 | object KafkaChecks { 25 | case class KafkaCheckTypeWrapper(value: Check[KafkaProtocolMessage]) 26 | class SimpleChecksScala extends KafkaCheckSupport {} 27 | 28 | val avroSerde: Serde[GenericRecord] = new GenericAvroSerde() 29 | 30 | private def toScalaCheck(javaCheck: Object): KafkaCheck = { 31 | javaCheck match { 32 | case _: io.gatling.javaapi.core.CheckBuilder => 33 | val checkBuilder = javaCheck.asInstanceOf[io.gatling.javaapi.core.CheckBuilder] 34 | val scalaCheck = checkBuilder.asScala 35 | checkBuilder.`type` match { 36 | case CoreCheckType.BodyBytes => 37 | scalaCheck 38 | .asInstanceOf[CheckBuilder[BodyBytesCheckType, Array[Byte]]] 39 | .build(KafkaCheckMaterializer.bodyBytes) 40 | case CoreCheckType.BodyString => 41 | scalaCheck 42 | .asInstanceOf[CheckBuilder[BodyStringCheckType, String]] 43 | .build(KafkaCheckMaterializer.bodyString(io.gatling.core.Predef.configuration)) 44 | case CoreCheckType.Substring => 45 | scalaCheck 46 | .asInstanceOf[CheckBuilder[SubstringCheckType, String]] 47 | .build(KafkaCheckMaterializer.substring(io.gatling.core.Predef.configuration)) 48 | case CoreCheckType.XPath => 49 | scalaCheck 50 | .asInstanceOf[CheckBuilder[XPathCheckType, XdmNode]] 51 | .build(KafkaCheckMaterializer.xpath(io.gatling.core.Predef.configuration)) 52 | case CoreCheckType.JsonPath => 53 | scalaCheck 54 | .asInstanceOf[CheckBuilder[JsonPathCheckType, JsonNode]] 55 | .build( 56 | KafkaCheckMaterializer.jsonPath(io.gatling.core.Predef.defaultJsonParsers, io.gatling.core.Predef.configuration), 57 | ) 58 | case CoreCheckType.JmesPath => 59 | scalaCheck 60 | .asInstanceOf[CheckBuilder[JmesPathCheckType, JsonNode]] 61 | .build( 62 | KafkaCheckMaterializer.jmesPath(io.gatling.core.Predef.defaultJsonParsers, io.gatling.core.Predef.configuration), 63 | ) 64 | case KafkaCheckType.ResponseCode => 65 | scalaCheck 66 | .asInstanceOf[CheckBuilder[checks.KafkaCheckMaterializer.KafkaMessageCheckType, KafkaProtocolMessage]] 67 | .build( 68 | KafkaCheckMaterializer.kafkaStatusCheck, 69 | ) 70 | case unknown => throw new IllegalArgumentException(s"Kafka DSL doesn't support $unknown") 71 | } 72 | } 73 | } 74 | 75 | def toScalaChecks(javaChecks: ju.List[Object]): Seq[KafkaCheck] = 76 | javaChecks.asScala.map(toScalaCheck).toSeq 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/ru/tinkoff/gatling/kafka/javaapi/request/builder/RROutTopicStep.java: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.*; 6 | import ru.tinkoff.gatling.kafka.request.builder.KafkaRequestBuilderBase; 7 | import scala.reflect.ClassTag; 8 | 9 | import static io.gatling.javaapi.core.internal.Expressions.*; 10 | 11 | public class RROutTopicStep { 12 | 13 | private final String inputTopic; 14 | private final String outputTopic; 15 | private final String requestName; 16 | 17 | public RROutTopicStep(String inputTopic, String outputTopic, String requestName) { 18 | this.inputTopic = inputTopic; 19 | this.outputTopic = outputTopic; 20 | this.requestName = requestName; 21 | } 22 | 23 | public RequestReplyBuilder send(K key, V payload, Headers headers, Class keyClass, Class payloadClass) { 24 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 25 | .requestTopic(toStringExpression(this.inputTopic)) 26 | .replyTopic(toStringExpression(this.outputTopic)) 27 | .send( 28 | toStaticValueExpression(key), 29 | toStaticValueExpression(payload), 30 | toStaticValueExpression(headers), 31 | Serdes.serdeFrom(keyClass), 32 | ClassTag.apply(keyClass), 33 | Serdes.serdeFrom(payloadClass), 34 | ClassTag.apply(payloadClass) 35 | )); 36 | } 37 | 38 | public RequestReplyBuilder send(K key, V payload, Class keyClass, Class payloadClass) { 39 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 40 | .requestTopic(toStringExpression(this.inputTopic)) 41 | .replyTopic(toStringExpression(this.outputTopic)) 42 | .send( 43 | toStaticValueExpression(key), 44 | toStaticValueExpression(payload), 45 | toStaticValueExpression(new RecordHeaders()), 46 | Serdes.serdeFrom(keyClass), 47 | ClassTag.apply(keyClass), 48 | Serdes.serdeFrom(payloadClass), 49 | ClassTag.apply(payloadClass) 50 | )); 51 | } 52 | 53 | public RequestReplyBuilder send(K key, V payload, Class keyClass, Class payloadClass, Serializer ser, Deserializer de) { 54 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 55 | .requestTopic(toStringExpression(this.inputTopic)) 56 | .replyTopic(toStringExpression(this.outputTopic)) 57 | .send( 58 | toStaticValueExpression(key), 59 | toStaticValueExpression(payload), 60 | toStaticValueExpression(new RecordHeaders()), 61 | Serdes.serdeFrom(keyClass), 62 | ClassTag.apply(keyClass), 63 | Serdes.serdeFrom(ser, de), 64 | ClassTag.apply(payloadClass) 65 | )); 66 | } 67 | 68 | public RequestReplyBuilder send(K key, V payload, Headers headers, Class keyClass, Class payloadClass, Serializer ser, Deserializer de) { 69 | return new RequestReplyBuilder(KafkaRequestBuilderBase.apply(toStringExpression(this.requestName)).requestReply() 70 | .requestTopic(toStringExpression(this.inputTopic)) 71 | .replyTopic(toStringExpression(this.outputTopic)) 72 | .send( 73 | toStaticValueExpression(key), 74 | toStaticValueExpression(payload), 75 | toStaticValueExpression(headers), 76 | Serdes.serdeFrom(keyClass), 77 | ClassTag.apply(keyClass), 78 | Serdes.serdeFrom(ser, de), 79 | ClassTag.apply(payloadClass) 80 | )); 81 | } 82 | 83 | } 84 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/checks/KafkaCheckSupport.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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 ru.tinkoff.gatling.kafka.KafkaCheck 20 | import ru.tinkoff.gatling.kafka.checks.KafkaCheckMaterializer.KafkaMessageCheckType 21 | import ru.tinkoff.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 <: GenericRecord: 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 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Gatling Kafka Plugin 2 | 3 | ![Build](https://github.com/TinkoffCreditSystems/gatling-kafka-plugin/workflows/Build/badge.svg) [![Maven Central](https://img.shields.io/maven-central/v/ru.tinkoff/gatling-kafka-plugin_2.13.svg?color=success)](https://search.maven.org/search?q=ru.tinkoff.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/Tinkoff/gatling-kafka-plugin/coverage.svg?branch=master)](https://codecov.io/github/Tinkoff/gatling-kafka-plugin?branch=master) 5 | 6 | # Introduction 7 | 8 | Plugin to support Kafka in Gatling (3.9.x) 9 | 10 | # Usage 11 | 12 | ### Getting Started 13 | 14 | Plugin is currently available for Scala 2.13, Java 17, Kotlin. 15 | 16 | You may include plugin as dependency in project with your tests. Write 17 | 18 | ### Scala 19 | 20 | ```scala 21 | libraryDependencies += "ru.tinkoff" %% "gatling-kafka-plugin" % % Test 22 | ``` 23 | 24 | ### Java 25 | 26 | Write this to your dependencies block in build.gradle: 27 | 28 | ```java 29 | gatling "ru.tinkoff:gatling-kafka-plugin_2.13:" 30 | ``` 31 | 32 | ### Kotlin 33 | 34 | Write this to your dependencies block in build.gradle: 35 | 36 | ```kotlin 37 | gatling("ru.tinkoff:gatling-kafka-plugin_2.13:") 38 | ``` 39 | 40 | ## Example Scenarios 41 | 42 | ### Scala 43 | 44 | Examples [here](src/test/scala/ru/tinkoff/gatling/kafka/examples) 45 | 46 | ### Java 47 | 48 | Examples [here](src/test/java/ru/tinkoff/gatling/kafka/javaapi/examples) 49 | 50 | ### Kotlin 51 | 52 | Examples [here](src/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples) 53 | 54 | ## Download and create Avro schema 55 | 56 | Avro schema is downloaded using the plugin [sbt-schema-registry-plugin](https://github.com/Tinkoff/sbt-schema-registry-plugin) 57 | and for that you need to configure schemas and url in `build.sbt` and run the command: 58 | 59 | ```bash 60 | sbt schemaRegistryDownload 61 | ``` 62 | 63 | To create java classes you should add use capabilities, that provide plugin [sbt-avro](https://github.com/sbt/sbt-avro). 64 | This plugin is included in project and will do all needed for creating java classes in compile stage. 65 | To run you should create scala object in root project directory and type `sbt run`. 66 | 67 | ### Example download avro-schema 68 | 69 | Example [here](https://github.com/TinkoffCreditSystems/gatling-kafka-plugin/tree/master/src/test/scala/ru/tinkoff/gatling/kafka/examples) 70 | 71 | ## Avro support in Request-Reply 72 | 73 | ### Scala 74 | 75 | To use avro messages as payload in key or value, you must: 76 | 77 | - define implicit for schema registry url: 78 | 79 | ```scala 80 | implicit val schemaRegUrl: String = "http://localhost:9094" 81 | ``` 82 | 83 | - or define serde for your class: 84 | 85 | ```scala 86 | val ser = 87 | new KafkaAvroSerializer( 88 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 89 | ) 90 | 91 | val de = 92 | new KafkaAvroDeserializer( 93 | new CachedSchemaRegistryClient("schRegUrl".split(',').toList.asJava, 16), 94 | ) 95 | 96 | implicit val serdeClass: Serde[MyAvroClass] = new Serde[MyAvroClass] { 97 | override def serializer(): Serializer[MyAvroClass] = ser.asInstanceOf[Serializer[MyAvroClass]] 98 | override def deserializer(): Deserializer[MyAvroClass] = de.asInstanceOf[Deserializer[MyAvroClass]] 99 | } 100 | ``` 101 | 102 | ### Java 103 | 104 | To use avro messages as payload in key or value, you must define serde for your class: 105 | 106 | ```java 107 | public static Serializer ser = (Serializer) new KafkaAvroSerializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 108 | public static Deserializer de = (Deserializer) new KafkaAvroDeserializer(new CachedSchemaRegistryClient(Arrays.asList("schRegUrl".split(",")), 16)); 109 | ``` 110 | 111 | ### Kotlin 112 | 113 | To use avro messages as payload in key or value, you must define serde for your class: 114 | 115 | ```kotlin 116 | val ser = KafkaAvroSerializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Serializer 117 | val de = KafkaAvroDeserializer(CachedSchemaRegistryClient("schRegUrl".split(','), 16),) as Deserializer 118 | ``` 119 | 120 | ### Example usage Avro in Request-Reply 121 | 122 | Example [scala](src/test/scala/ru/tinkoff/gatling/kafka/examples/AvroClassWithRequestReplySimulation.scala) 123 | 124 | Example [java](src/test/java/ru/tinkoff/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.java) 125 | 126 | Example [kotlin](src/test/kotlin/ru/tinkoff/gatling/kafka/javaapi/examples/AvroClassWithRequestReplySimulation.kt) 127 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/actions/KafkaRequestReplyAction.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.actions 2 | 3 | import io.gatling.commons.stats.KO 4 | import io.gatling.commons.util.Clock 5 | import io.gatling.commons.validation._ 6 | import io.gatling.core.action.{Action, RequestAction} 7 | import io.gatling.core.controller.throttle.Throttler 8 | import io.gatling.core.session.el._ 9 | import io.gatling.core.session.{Expression, Session} 10 | import io.gatling.core.stats.StatsEngine 11 | import io.gatling.core.util.NameGen 12 | import org.apache.kafka.common.serialization.Serializer 13 | import ru.tinkoff.gatling.kafka.KafkaLogging 14 | import ru.tinkoff.gatling.kafka.protocol.KafkaComponents 15 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 16 | import ru.tinkoff.gatling.kafka.request.builder.KafkaRequestReplyAttributes 17 | 18 | import scala.reflect.{ClassTag, classTag} 19 | 20 | class KafkaRequestReplyAction[K: ClassTag, V: ClassTag]( 21 | components: KafkaComponents, 22 | attributes: KafkaRequestReplyAttributes[K, V], 23 | val statsEngine: StatsEngine, 24 | val clock: Clock, 25 | val next: Action, 26 | throttler: Option[Throttler], 27 | ) extends RequestAction with KafkaLogging with NameGen { 28 | override def requestName: Expression[String] = attributes.requestName 29 | 30 | override def sendRequest(session: Session): Validation[Unit] = { 31 | for { 32 | rn <- requestName(session) 33 | msg <- resolveToProtocolMessage(session) 34 | } yield throttler 35 | .fold(publishAndLogMessage(rn, msg, session))( 36 | _.throttle(session.scenario, () => publishAndLogMessage(rn, msg, session)), 37 | ) 38 | 39 | } 40 | 41 | private def serializeKey( 42 | serializer: Serializer[K], 43 | keyE: Expression[K], 44 | topicE: Expression[String], 45 | ): Expression[Array[Byte]] = s => 46 | // need for work gatling Expression Language 47 | if (classTag[K].runtimeClass.getCanonicalName == "java.lang.String") 48 | for { 49 | topic <- topicE(s) 50 | key <- keyE.asInstanceOf[Expression[String]](s).flatMap(_.el[String].apply(s)) 51 | } yield serializer.asInstanceOf[Serializer[String]].serialize(topic, key) 52 | else 53 | for { 54 | topic <- topicE(s) 55 | key <- keyE(s) 56 | } yield serializer.serialize(topic, key) 57 | 58 | private def optToVal[T](ovt: Option[Validation[T]]): Validation[Option[T]] = 59 | ovt.fold(Option.empty[T].success)(_.map(Option[T])) 60 | 61 | private def resolveToProtocolMessage: Expression[KafkaProtocolMessage] = s => 62 | // need for work gatling Expression Language 63 | if (classTag[V].runtimeClass.getCanonicalName == "java.lang.String") 64 | for { 65 | key <- serializeKey(attributes.keySerializer, attributes.key, attributes.inputTopic)(s) 66 | inputTopic <- attributes.inputTopic(s) 67 | outputTopic <- attributes.outputTopic(s) 68 | value <- attributes.value 69 | .asInstanceOf[Expression[String]](s) 70 | .flatMap(_.el[String].apply(s)) 71 | .map(v => attributes.valueSerializer.asInstanceOf[Serializer[String]].serialize(inputTopic, v)) 72 | headers <- optToVal(attributes.headers.map(_(s))) 73 | } yield KafkaProtocolMessage(key, value, inputTopic, outputTopic, headers) 74 | else 75 | for { 76 | key <- serializeKey(attributes.keySerializer, attributes.key, attributes.inputTopic)(s) 77 | inputTopic <- attributes.inputTopic(s) 78 | outputTopic <- attributes.outputTopic(s) 79 | value <- attributes.value(s).map(v => attributes.valueSerializer.serialize(inputTopic, v)) 80 | headers <- optToVal(attributes.headers.map(_(s))) 81 | } yield KafkaProtocolMessage(key, value, inputTopic, outputTopic, headers) 82 | 83 | private def publishAndLogMessage(requestNameString: String, msg: KafkaProtocolMessage, session: Session): Unit = { 84 | val now = clock.nowMillis 85 | components.sender.send(msg)( 86 | rm => { 87 | if (logger.underlying.isDebugEnabled) { 88 | logMessage(s"Record sent user=${session.userId} key=${new String(msg.key)} topic=${rm.topic()}", msg) 89 | } 90 | val id = components.kafkaProtocol.messageMatcher.requestMatch(msg) 91 | components.trackersPool 92 | .tracker(msg.inputTopic, msg.outputTopic, components.kafkaProtocol.messageMatcher, None) 93 | .track( 94 | id, 95 | clock.nowMillis, 96 | components.kafkaProtocol.timeout.toMillis, 97 | attributes.checks, 98 | session, 99 | next, 100 | requestNameString, 101 | ) 102 | }, 103 | e => { 104 | logger.error(e.getMessage, e) 105 | statsEngine.logResponse( 106 | session.scenario, 107 | session.groups, 108 | requestNameString, 109 | now, 110 | clock.nowMillis, 111 | KO, 112 | Some("500"), 113 | Some(e.getMessage), 114 | ) 115 | }, 116 | ) 117 | } 118 | 119 | override def name: String = genName("kafkaRequestReply") 120 | } 121 | -------------------------------------------------------------------------------- /src/main/scala/ru/tinkoff/gatling/kafka/client/KafkaMessageTrackerActor.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.gatling.kafka.client 2 | 3 | import akka.actor.{Actor, Props, Timers} 4 | import com.typesafe.scalalogging.LazyLogging 5 | import io.gatling.commons.stats.{KO, OK, Status} 6 | import io.gatling.commons.util.Clock 7 | import io.gatling.commons.validation.Failure 8 | import io.gatling.core.action.Action 9 | import io.gatling.core.check.Check 10 | import io.gatling.core.session.Session 11 | import io.gatling.core.stats.StatsEngine 12 | import ru.tinkoff.gatling.kafka.KafkaCheck 13 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 14 | 15 | import scala.collection.mutable 16 | import scala.concurrent.duration.DurationInt 17 | 18 | object KafkaMessageTrackerActor { 19 | 20 | def props(statsEngine: StatsEngine, clock: Clock): Props = 21 | Props(new KafkaMessageTrackerActor(statsEngine, clock)) 22 | 23 | case class MessagePublished( 24 | matchId: Array[Byte], 25 | sent: Long, 26 | replyTimeout: Long, 27 | checks: List[KafkaCheck], 28 | session: Session, 29 | next: Action, 30 | requestName: String, 31 | ) 32 | 33 | case class MessageConsumed( 34 | replyId: Array[Byte], 35 | received: Long, 36 | message: KafkaProtocolMessage, 37 | ) 38 | 39 | case object TimeoutScan 40 | 41 | private def makeKeyForSentMessages(m: Array[Byte]): String = 42 | Option(m).map(java.util.Base64.getEncoder.encodeToString).getOrElse("") 43 | } 44 | 45 | class KafkaMessageTrackerActor(statsEngine: StatsEngine, clock: Clock) extends Actor with Timers with LazyLogging { 46 | import KafkaMessageTrackerActor._ 47 | def triggerPeriodicTimeoutScan( 48 | periodicTimeoutScanTriggered: Boolean, 49 | sentMessages: mutable.HashMap[String, MessagePublished], 50 | timedOutMessages: mutable.ArrayBuffer[MessagePublished], 51 | ): Unit = 52 | if (!periodicTimeoutScanTriggered) { 53 | context.become(onMessage(periodicTimeoutScanTriggered = true, sentMessages, timedOutMessages)) 54 | timers.startTimerWithFixedDelay("timeoutTimer", TimeoutScan, 1000.millis) 55 | } 56 | 57 | override def receive: Receive = 58 | onMessage( 59 | periodicTimeoutScanTriggered = false, 60 | mutable.HashMap.empty[String, MessagePublished], 61 | mutable.ArrayBuffer.empty[MessagePublished], 62 | ) 63 | 64 | private def executeNext( 65 | session: Session, 66 | sent: Long, 67 | received: Long, 68 | status: Status, 69 | next: Action, 70 | requestName: String, 71 | responseCode: Option[String], 72 | message: Option[String], 73 | ): Unit = { 74 | statsEngine.logResponse( 75 | session.scenario, 76 | session.groups, 77 | requestName, 78 | sent, 79 | received, 80 | status, 81 | responseCode, 82 | message, 83 | ) 84 | next ! session.logGroupRequestTimings(sent, received) 85 | } 86 | 87 | /** Processes a matched message 88 | */ 89 | private def processMessage( 90 | session: Session, 91 | sent: Long, 92 | received: Long, 93 | checks: List[KafkaCheck], 94 | message: KafkaProtocolMessage, 95 | next: Action, 96 | requestName: String, 97 | ): Unit = { 98 | val (newSession, error) = Check.check(message, session, checks) 99 | error match { 100 | case Some(Failure(errorMessage)) => 101 | executeNext( 102 | newSession.markAsFailed, 103 | sent, 104 | received, 105 | KO, 106 | next, 107 | requestName, 108 | message.responseCode, 109 | Some(errorMessage), 110 | ) 111 | case _ => 112 | executeNext(newSession, sent, received, OK, next, requestName, None, None) 113 | } 114 | } 115 | 116 | private def onMessage( 117 | periodicTimeoutScanTriggered: Boolean, 118 | sentMessages: mutable.HashMap[String, MessagePublished], 119 | timedOutMessages: mutable.ArrayBuffer[MessagePublished], 120 | ): Receive = { 121 | // message was sent; add the timestamps to the map 122 | case messageSent: MessagePublished => 123 | val key = makeKeyForSentMessages(messageSent.matchId) 124 | sentMessages += key -> messageSent 125 | if (messageSent.replyTimeout > 0) { 126 | triggerPeriodicTimeoutScan(periodicTimeoutScanTriggered, sentMessages, timedOutMessages) 127 | } 128 | 129 | // message was received; publish stats and remove from the map 130 | case MessageConsumed(replyId, received, message) => 131 | // if key is missing, message was already acked and is a dup, or request timeout 132 | val key = makeKeyForSentMessages(replyId) 133 | sentMessages.remove(key).foreach { case MessagePublished(_, sent, _, checks, session, next, requestName) => 134 | processMessage(session, sent, received, checks, message, next, requestName) 135 | } 136 | 137 | case TimeoutScan => 138 | val now = clock.nowMillis 139 | sentMessages.valuesIterator.foreach { messagePublished => 140 | val replyTimeout = messagePublished.replyTimeout 141 | if (replyTimeout > 0 && (now - messagePublished.sent) > replyTimeout) { 142 | timedOutMessages += messagePublished 143 | } 144 | } 145 | for (MessagePublished(matchId, sent, receivedTimeout, _, session, next, requestName) <- timedOutMessages) { 146 | sentMessages.remove(makeKeyForSentMessages(matchId)) 147 | executeNext( 148 | session.markAsFailed, 149 | sent, 150 | now, 151 | KO, 152 | next, 153 | requestName, 154 | None, 155 | Some(s"Reply timeout after $receivedTimeout ms"), 156 | ) 157 | } 158 | timedOutMessages.clear() 159 | } 160 | } 161 | -------------------------------------------------------------------------------- /src/test/scala/ru/tinkoff/gatling/kafka/examples/KafkaGatlingTest.scala: -------------------------------------------------------------------------------- 1 | package ru.tinkoff.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.structure.ScenarioBuilder 8 | import org.apache.kafka.clients.producer.ProducerConfig 9 | import ru.tinkoff.gatling.kafka.Predef._ 10 | import ru.tinkoff.gatling.kafka.protocol.KafkaProtocol 11 | import org.apache.kafka.common.header.Headers 12 | import org.apache.kafka.common.header.internals.RecordHeaders 13 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer} 14 | import ru.tinkoff.gatling.kafka.request.KafkaProtocolMessage 15 | 16 | import scala.concurrent.duration.DurationInt 17 | 18 | class KafkaGatlingTest extends Simulation { 19 | 20 | case class Ingredient(name: String, sugar: Double, fat: Double) 21 | 22 | implicit val ingridientToRecord: ToRecord[Ingredient] = ToRecord.apply 23 | implicit val ingridientFromRecord: FromRecord[Ingredient] = FromRecord.apply 24 | implicit val ingridientSchemaFor: SchemaFor[Ingredient] = SchemaFor.apply 25 | implicit val ingridientFormat: RecordFormat[Ingredient] = RecordFormat.apply 26 | implicit val ingredientHeaders: Headers = new RecordHeaders() 27 | 28 | val kafkaConf: KafkaProtocol = kafka 29 | .topic("test.t1") 30 | .properties( 31 | Map( 32 | ProducerConfig.ACKS_CONFIG -> "1", 33 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 34 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 35 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 36 | ), 37 | ) 38 | 39 | val kafkaConfwoKey: KafkaProtocol = kafka 40 | .topic("myTopic3") 41 | .properties( 42 | Map( 43 | ProducerConfig.ACKS_CONFIG -> "1", 44 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 45 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 46 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 47 | ), 48 | ) 49 | 50 | val kafkaConfBytes: KafkaProtocol = kafka 51 | .topic("test.t2") 52 | .properties( 53 | Map( 54 | ProducerConfig.ACKS_CONFIG -> "1", 55 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 56 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 57 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 58 | ), 59 | ) 60 | 61 | val kafkaProtocolRRString: KafkaProtocol = kafka.requestReply 62 | .producerSettings( 63 | Map( 64 | ProducerConfig.ACKS_CONFIG -> "1", 65 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 66 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 67 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 68 | ), 69 | ) 70 | .consumeSettings( 71 | Map( 72 | "bootstrap.servers" -> "localhost:9093", 73 | ), 74 | ) 75 | .withDefaultTimeout 76 | 77 | val kafkaProtocolRRBytes: KafkaProtocol = kafka.requestReply 78 | .producerSettings( 79 | Map( 80 | ProducerConfig.ACKS_CONFIG -> "1", 81 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 82 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 83 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 84 | ), 85 | ) 86 | .consumeSettings( 87 | Map( 88 | "bootstrap.servers" -> "localhost:9093", 89 | ), 90 | ) 91 | .timeout(5.seconds) 92 | .matchByValue 93 | 94 | val kafkaProtocolRRBytes2: KafkaProtocol = kafka.requestReply 95 | .producerSettings( 96 | Map( 97 | ProducerConfig.ACKS_CONFIG -> "1", 98 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 99 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 100 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", 101 | ), 102 | ) 103 | .consumeSettings( 104 | Map( 105 | "bootstrap.servers" -> "localhost:9093", 106 | ), 107 | ) 108 | .timeout(1.seconds) 109 | .matchByValue 110 | 111 | val kafkaAvro4sConf: KafkaProtocol = kafka 112 | .topic("test.t3") 113 | .properties( 114 | Map( 115 | ProducerConfig.ACKS_CONFIG -> "1", 116 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 117 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 118 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 119 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 120 | "schema.registry.url" -> "http://localhost:9094", 121 | ), 122 | ) 123 | 124 | def matchByOwnVal(message: KafkaProtocolMessage): Array[Byte] = { 125 | message.key 126 | } 127 | 128 | val kafkaProtocolRRAvro: KafkaProtocol = kafka.requestReply 129 | .producerSettings( 130 | Map( 131 | ProducerConfig.ACKS_CONFIG -> "1", 132 | ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", 133 | ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringSerializer", 134 | ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "io.confluent.kafka.serializers.KafkaAvroSerializer", 135 | "value.subject.name.strategy" -> "io.confluent.kafka.serializers.subject.RecordNameStrategy", 136 | "schema.registry.url" -> "http://localhost:9094", 137 | ), 138 | ) 139 | .consumeSettings( 140 | Map( 141 | "bootstrap.servers" -> "localhost:9093", 142 | ), 143 | ) 144 | .timeout(7.seconds) 145 | .matchByMessage(matchByOwnVal) 146 | 147 | val scnRR: ScenarioBuilder = scenario("RequestReply String") 148 | .exec( 149 | kafka("Request Reply String").requestReply 150 | .requestTopic("myTopic1") 151 | .replyTopic("test.t1") 152 | .send[String, String]("testCheckJson", """{ "m": "dkf" }""") 153 | .check(jsonPath("$.m").is("dkf")), 154 | ) 155 | 156 | val scnwokey: ScenarioBuilder = scenario("Request String without key") 157 | .exec( 158 | kafka("Request String") 159 | .send[String]("foo"), 160 | ) 161 | .exec( 162 | kafka("Request Stirng With null key") 163 | .send[Any, String](null, "nullkey"), 164 | ) 165 | 166 | val scn: ScenarioBuilder = scenario("Request String") 167 | .exec(kafka("Request String 2").send[String, String]("testCheckJson", """{ "m": "dkf" }""")) 168 | 169 | val scn2: ScenarioBuilder = scenario("Request Byte") 170 | .exec( 171 | kafka("Request Byte") 172 | .send[Array[Byte], Array[Byte]]("key".getBytes(), "tstBytes".getBytes()), 173 | ) 174 | 175 | val scnRR2: ScenarioBuilder = scenario("RequestReply Bytes") 176 | .exec( 177 | kafka("Request Reply Bytes").requestReply 178 | .requestTopic("myTopic2") 179 | .replyTopic("test.t2") 180 | .send[Array[Byte], Array[Byte]]("test".getBytes(), "tstBytes".getBytes()) 181 | .check(bodyBytes.is("tstBytes".getBytes()).saveAs("bodyInfo")), 182 | ) 183 | 184 | val scnAvro4s: ScenarioBuilder = scenario("Request Avro4s") 185 | .exec( 186 | kafka("Request Simple Avro4s") 187 | .send(Ingredient("Cheese", 1d, 50d)), 188 | ) 189 | .exec( 190 | kafka("Request Avro4s") 191 | .send[String, Ingredient]("key4s", Ingredient("Cheese", 0d, 70d)), 192 | ) 193 | 194 | val scnRRwo: ScenarioBuilder = scenario("RequestReply w/o answer") 195 | .exec( 196 | kafka("Request Reply Bytes wo").requestReply 197 | .requestTopic("myTopic2") 198 | .replyTopic("test.t2") 199 | .send[Array[Byte], Array[Byte]]("testWO".getBytes(), "tstBytesWO".getBytes()), 200 | ) 201 | 202 | setUp( 203 | scnRR.inject(atOnceUsers(1)).protocols(kafkaProtocolRRString), 204 | scn.inject(nothingFor(1), atOnceUsers(1)).protocols(kafkaConf), 205 | scnRR2.inject(atOnceUsers(1)).protocols(kafkaProtocolRRBytes), 206 | scn2.inject(nothingFor(2), atOnceUsers(1)).protocols(kafkaConfBytes), 207 | scnAvro4s.inject(atOnceUsers(1)).protocols(kafkaAvro4sConf), 208 | scnRRwo.inject(atOnceUsers(1)).protocols(kafkaProtocolRRBytes2), 209 | scnwokey.inject(nothingFor(1), atOnceUsers(1)).protocols(kafkaConfwoKey), 210 | ).assertions( 211 | global.failedRequests.percent.lt(15.0), 212 | ) 213 | 214 | } 215 | -------------------------------------------------------------------------------- /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 | --------------------------------------------------------------------------------