-
Notifications
You must be signed in to change notification settings - Fork 83
Introducing support for Kafka streams #39
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
66353f7
Moved embedded kafka to its own sub module. Changed root project to a…
adamosloizou 9d04fb6
Introduced kafka-streams module for testing Kafka Streams using Embed…
adamosloizou e661bdc
Disabling verbose logging for tests to make it easier to trace failed…
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
File renamed without changes.
51 changes: 51 additions & 0 deletions
51
embedded-kafka/src/main/scala/net/manub/embeddedkafka/ConsumerExtensions.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,51 @@ | ||
| package net.manub.embeddedkafka | ||
|
|
||
| import org.apache.kafka.clients.consumer.KafkaConsumer | ||
| import org.apache.kafka.common.KafkaException | ||
| import org.apache.log4j.Logger | ||
|
|
||
| import scala.util.Try | ||
|
|
||
| /** Method extensions for Kafka's [[KafkaConsumer]] API allowing easy testing.*/ | ||
| object ConsumerExtensions { | ||
| val MaximumAttempts = 3 | ||
| implicit class ConsumerOps[K, V](val consumer: KafkaConsumer[K, V]) { | ||
|
|
||
| private val logger = Logger.getLogger(classOf[ConsumerOps[K, V]]) | ||
|
|
||
| /** Consume messages from a given topic and return them as a lazily evaluated Scala Stream. | ||
| * Depending on how many messages are taken from the Scala Stream it will try up to 3 times | ||
| * to consume batches from the given topic, until it reaches the number of desired messages or | ||
| * return otherwise. | ||
| * | ||
| * @param topic the topic from which to consume messages | ||
| * @return the stream of consumed messages that you can do `.take(n: Int).toList` | ||
| * to evaluate the requested number of messages. | ||
| */ | ||
| def consumeLazily(topic: String): Stream[(K, V)] = { | ||
| val attempts = 1 to MaximumAttempts | ||
| attempts.toStream.flatMap { attempt => | ||
| val batch: Seq[(K, V)] = getNextBatch(topic) | ||
| logger.debug(s"----> Batch $attempt ($topic) | ${batch.mkString("|")}") | ||
| batch | ||
| } | ||
| } | ||
|
|
||
| /** Get the next batch of messages from Kafka. | ||
| * | ||
| * @param topic the topic to consume | ||
| * @return the next batch of messages | ||
| */ | ||
| def getNextBatch(topic: String): Seq[(K, V)] = Try { | ||
| import scala.collection.JavaConversions._ | ||
| consumer.subscribe(List(topic)) | ||
| consumer.partitionsFor(topic) | ||
| val records = consumer.poll(2000) | ||
| // use toList to force eager evaluation. toSeq is lazy | ||
| records.iterator().toList.map(r => r.key -> r.value) | ||
| }.recover { | ||
| case ex: KafkaException => throw new KafkaUnavailableException(ex) | ||
| }.get | ||
| } | ||
| } | ||
|
|
59 changes: 59 additions & 0 deletions
59
embedded-kafka/src/main/scala/net/manub/embeddedkafka/Consumers.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,59 @@ | ||
| package net.manub.embeddedkafka | ||
|
|
||
| import java.util.Properties | ||
|
|
||
| import org.apache.kafka.clients.consumer.KafkaConsumer | ||
| import org.apache.kafka.common.serialization.Deserializer | ||
|
|
||
| /** Utility trait for easily creating Kafka consumers and accessing their consumed messages. */ | ||
| trait Consumers { | ||
| /** Loaner pattern that allows running a code block with a newly created consumer. | ||
| * The consumer's lifecycle will be automatically handled and closed at the end of the | ||
| * given code block. | ||
| * | ||
| * @param block the code block to be executed with the instantiated consumer | ||
| * passed as an argument | ||
| * @tparam K the type of the consumer's Key | ||
| * @tparam V the type of the consumer's Value | ||
| * @tparam T the type of the block's returning result | ||
| * @return the result of the executed block | ||
| */ | ||
| def withConsumer[K: Deserializer, V: Deserializer, T](block: KafkaConsumer[K, V] => T) | ||
| (implicit config: EmbeddedKafkaConfig): T = { | ||
| val consumer = newConsumer[K, V]() | ||
| try { | ||
| val result = block(consumer) | ||
| result | ||
| } finally { | ||
| consumer.close() | ||
| } | ||
| } | ||
|
|
||
| /** Convenience alternative to `withConsumer` that offers a consumer for String keys and values. | ||
| * | ||
| * @param block the block to be executed with the consumer | ||
| * @tparam T the type of the result of the code block | ||
| * @return the code block result | ||
| */ | ||
| def withStringConsumer[T](block: KafkaConsumer[String, String] => T) | ||
| (implicit config: EmbeddedKafkaConfig): T = { | ||
| import net.manub.embeddedkafka.Codecs.stringDeserializer | ||
| withConsumer(block) | ||
| } | ||
|
|
||
| /** Create a new Kafka consumer. | ||
| * | ||
| * @tparam K the type of the consumer's Key | ||
| * @tparam V the type of the consumer's Value | ||
| * @return the new consumer | ||
| */ | ||
| def newConsumer[K: Deserializer, V: Deserializer]() | ||
| (implicit config: EmbeddedKafkaConfig): KafkaConsumer[K, V] = { | ||
| val props = new Properties() | ||
| props.put("group.id", UUIDs.newUuid().toString) | ||
| props.put("bootstrap.servers", s"localhost:${config.kafkaPort}") | ||
| props.put("auto.offset.reset", "earliest") | ||
|
|
||
| new KafkaConsumer[K, V](props, implicitly[Deserializer[K]], implicitly[Deserializer[V]]) | ||
| } | ||
| } |
File renamed without changes.
File renamed without changes.
14 changes: 14 additions & 0 deletions
14
embedded-kafka/src/main/scala/net/manub/embeddedkafka/UUIDs.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,14 @@ | ||
| package net.manub.embeddedkafka | ||
|
|
||
| import java.util.UUID | ||
|
|
||
| /** Utility object for creating unique test IDs. | ||
| * Useful for separating IDs and directories across test cases. | ||
| */ | ||
| object UUIDs { | ||
| /** Create a new unique ID. | ||
| * | ||
| * @return the unique ID | ||
| */ | ||
| def newUuid(): UUID = UUID.randomUUID() | ||
| } |
File renamed without changes.
File renamed without changes.
File renamed without changes.
4 changes: 2 additions & 2 deletions
4
src/test/resources/log4j.properties → ...kafka/src/test/resources/log4j.properties
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,7 +1,7 @@ | ||
| log4j.rootLogger=info, stdout | ||
| log4j.rootLogger=off | ||
|
|
||
| log4j.appender.stdout=org.apache.log4j.ConsoleAppender | ||
| log4j.appender.stdout.layout=org.apache.log4j.PatternLayout | ||
|
|
||
| # Pattern to output the caller's file name and line number. | ||
| log4j.appender.stdout.layout.ConversionPattern=%5p [%t] (%F:%L) - %m%n | ||
| log4j.appender.stdout.layout.ConversionPattern=%5p [%t] (%F:%L) - %m%n |
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
56 changes: 56 additions & 0 deletions
56
kafka-streams/src/main/scala/net/manub/embeddedkafka/streams/EmbeddedKafkaStreams.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,56 @@ | ||
| package net.manub.embeddedkafka.streams | ||
|
|
||
| import net.manub.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig, UUIDs} | ||
| import org.apache.kafka.streams.KafkaStreams | ||
| import org.apache.kafka.streams.processor.TopologyBuilder | ||
| import org.apache.log4j.Logger | ||
| import org.scalatest.Suite | ||
|
|
||
| /** Helper trait for testing Kafka Streams. | ||
| * It creates an embedded Kafka Instance for each test case. | ||
| * Use `runStreams` to execute your streams. | ||
| */ | ||
| trait EmbeddedKafkaStreams extends EmbeddedKafka with TestStreamsConfig { | ||
| this: Suite => | ||
|
|
||
| private val logger = Logger.getLogger(classOf[EmbeddedKafkaStreams]) | ||
|
|
||
| /** Execute Kafka streams and pass a block of code that can | ||
| * operate while the streams are active. | ||
| * The code block can be used for publishing and consuming messages in Kafka. | ||
| * The block gets a pre-initialized kafka consumer that can be used implicitly for | ||
| * util methods such as `consumeLazily(String)`. | ||
| * | ||
| * e.g. | ||
| * | ||
| * {{{ | ||
| *runStreams(Seq("inputTopic", "outputTopic", streamBuilder) { | ||
| * // here you can publish and consume messages and make assertions | ||
| * publishToKafka(in, Seq("one-string", "another-string")) | ||
| * consumeFirstStringMessageFrom(in) should be ("one-string") | ||
| *} | ||
| * }}} | ||
| * | ||
| * @param topicsToCreate the topics that should be created in Kafka before launching the streams. | ||
| * @param builder the streams builder that will be used to instantiate the streams with | ||
| * a default configuration (all state directories are different and | ||
| * in temp folders) | ||
| * @param block the code block that will executed while the streams are active. | ||
| * Once the block has been executed the streams will be closed. | ||
| */ | ||
| def runStreams(topicsToCreate: Seq[String], builder: TopologyBuilder) | ||
| (block: => Any) | ||
| (implicit config: EmbeddedKafkaConfig): Any = | ||
| withRunningKafka { | ||
| topicsToCreate.foreach(topic => createCustomTopic(topic)) | ||
| val streamId = UUIDs.newUuid().toString | ||
| logger.debug(s"Creating stream with Application ID: [$streamId]") | ||
| val streams = new KafkaStreams(builder, streamConfig(streamId)) | ||
| streams.start() | ||
| try { | ||
| block | ||
| } finally { | ||
| streams.close() | ||
| } | ||
| }(config) | ||
| } | ||
46 changes: 46 additions & 0 deletions
46
...streams/src/main/scala/net/manub/embeddedkafka/streams/EmbeddedKafkaStreamsAllInOne.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,46 @@ | ||
| package net.manub.embeddedkafka.streams | ||
|
|
||
| import net.manub.embeddedkafka.{Consumers, EmbeddedKafkaConfig} | ||
| import org.apache.kafka.clients.consumer.KafkaConsumer | ||
| import org.apache.kafka.streams.processor.TopologyBuilder | ||
| import org.scalatest.Suite | ||
|
|
||
| /** Convenience trait for testing Kafka Streams with ScalaTest. | ||
| * It exposes `EmbeddedKafkaStreams.runStreams` as well as `Consumers` api | ||
| * for easily creating and querying consumers in tests. | ||
| * | ||
| * e.g. | ||
| * {{{ | ||
| *runStreams(Seq("inputTopic", "outputTopic", streamBuilder) { | ||
| * withConsumer[String, String, Unit] { consumer => | ||
| * // here you can publish and consume messages and make assertions | ||
| * publishToKafka(in, Seq("one-string", "another-string")) | ||
| * consumeLazily(out).take(2).toList should be ( | ||
| * Seq("one-string" -> "true", "another-string" -> "true") | ||
| * ) | ||
| * } | ||
| *} | ||
| * }}} | ||
| * | ||
| * @see [[Consumers]] | ||
| * @see [[EmbeddedKafkaStreams]] | ||
| */ | ||
| trait EmbeddedKafkaStreamsAllInOne extends EmbeddedKafkaStreams with Consumers { | ||
| this: Suite => | ||
|
|
||
| /** Run Kafka Streams while offering a String-based consumer for easy testing of stream output. | ||
| * | ||
| * @param topicsToCreate the topics that should be created. Usually these should be the topics | ||
| * that the Streams-under-test use for inputs and outputs. They need to be | ||
| * created before running the streams and | ||
| * this is automatically taken care of. | ||
| * @param builder the streams builder that contains the stream topology that will be instantiated | ||
| * @param block the block of testing code that will be executed by passing the simple | ||
| * String-based consumer. | ||
| * @return the result of the testing code | ||
| */ | ||
| def runStreamsWithStringConsumer(topicsToCreate: Seq[String], builder: TopologyBuilder) | ||
| (block: KafkaConsumer[String, String] => Any) | ||
| (implicit config: EmbeddedKafkaConfig): Any = | ||
| runStreams(topicsToCreate, builder)(withStringConsumer[Any](block))(config) | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
in my opinion it would be more idiomatic
.foreach(createCustomTopic)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree. I tried that originally but unfortunately it does not work:
An alternative is
foreach(createCustomTopic(_)). In my opinion that's not particularly pretty and I prefer being explicit as per above. I think it reads better.What do you think?