From da587a884207416593f3f07d794bb6d09751415d Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 9 Feb 2025 14:55:30 +0100 Subject: [PATCH 1/2] Deprecate accessor methods in zio-kafka tutorial Remove accessor methods from the zio-kafka tutorial. It also fixes lots of small and larger issues and adds links to more recent documentation. --- ...oduce-consume-data-to-from-kafka-topics.md | 573 +++++++++--------- 1 file changed, 275 insertions(+), 298 deletions(-) diff --git a/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md b/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md index 328e3ad9f4f7..d350049f32b1 100644 --- a/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md +++ b/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md @@ -6,16 +6,16 @@ sidebar_label: "Producing/Consuming Data To/From Kafka Topics" ## Introduction -Kafka is a distributed, fault-tolerant, message-oriented event-store platform. It is used as a message broker for distributed applications. ZIO Kafka is a library that provides a way to consume and produce data from Kafka topics and it also supports the ability to have streaming consumers and producers. +Kafka is a distributed, fault-tolerant, message-oriented event-store platform. It is used as a message broker for distributed applications. Zio-kafka is a library that provides a way to consume and produce data from Kafka topics, and it also supports the ability to have streaming consumers and producers. -In this tutorial, we will learn how to use ZIO Streams and ZIO Kafka to produce and consume data from Kafka topics. +In this tutorial, we will learn how to use zio-streams and zio-kafka to produce and consume data from Kafka topics. ## Running Examples To access the code examples, you can clone the [ZIO Quickstarts](http://github.com/zio/zio-quickstarts) project: ```bash -$ git clone git@github.com:zio/zio-quickstarts.git +$ git clone https://github.com/zio/zio-quickstarts.git $ cd zio-quickstarts/zio-quickstart-kafka ``` @@ -31,78 +31,73 @@ In this tutorial, we will be using the following dependencies. So, let's add the ```scala libraryDependencies += Seq( - "dev.zio" %% "zio" % "2.0.9", - "dev.zio" %% "zio-streams" % "2.0.9", - "dev.zio" %% "zio-kafka" % "2.1.1", - "dev.zio" %% "zio-json" % "0.3.0-RC10" + "dev.zio" %% "zio" % "2.1.15", + "dev.zio" %% "zio-streams" % "2.1.15", + "dev.zio" %% "zio-kafka" % "2.10.0", + "dev.zio" %% "zio-json" % "0.7.16" ) ``` -1. **ZIO Kafka** is a ZIO native client for Apache Kafka. It has a high-level streaming API on top of the Java client. So we can produce and consume events using the declarative concurrency model of ZIO Streams. +1. **Zio-kafka** is a ZIO native client for Apache Kafka. It provides a high-level streaming API on top of the Java client. So we can produce and consume events using the declarative concurrency model of zio-streams. -2. **ZIO Stream** introduces a high-level API for working with streams of values. It is designated to work in a highly concurrent environment. It has seamless integration with ZIO, so we have the ability to use all the features of the ZIO along with the streams, e.g. `Scope`, `Schedule`, `ZLayer`, `Queue`, `Hub` etc. To learn more about ZIO Stream, we have a comprehensive section in on that [here](../../reference/stream/index.md). +2. **Zio-streams** introduces a high-level API for working with streams of values. It is designated to work in a highly concurrent environment. It has seamless integration with ZIO, so we have the ability to use all the features of the ZIO along with the streams, e.g. `Scope`, `Schedule`, `ZLayer`, `Queue`, `Hub` etc. To learn more about zio-stream, we have a comprehensive section in on that [here](../../reference/stream/index.md). -3. **ZIO JSON** is a library to serialize and deserialize data from/to JSON data type. We will be using this library to serialize and deserialize data when reading and writing JSON data from/to Kafka topics. +3. **Zio-json** is a library to serialize and deserialize data from/to JSON data type. We will be using this library to serialize and deserialize data when reading and writing JSON data from/to Kafka topics. ## Setting Up The Kafka Cluster -Before we start, we need to set up a Kafka cluster. To set up the kafka cluster for testing purposes we can use the following `docker-compose.yml` file: +Before we start, we need to set up a Kafka cluster. To start a kafka cluster for testing purposes we can use the following `docker-compose.yml` file: ```docker-compose -version: '2' services: - zookeeper: - image: confluentinc/cp-zookeeper:latest - environment: - ZOOKEEPER_CLIENT_PORT: 2181 - ZOOKEEPER_TICK_TIME: 2000 - ports: - - 22181:2181 - - kafka: - image: confluentinc/cp-kafka:latest - depends_on: - - zookeeper + broker: + image: apache/kafka:3.9.0 + container_name: broker ports: - - 29092:29092 + - "9092:9092" environment: + KAFKA_NODE_ID: 1 KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 - KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092,PLAINTEXT_HOST://localhost:29092 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + KAFKA_PROCESS_ROLES: broker,controller + KAFKA_LISTENERS: PLAINTEXT://broker:29092,CONTROLLER://broker:29093,PLAINTEXT_HOST://0.0.0.0:9092 + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://broker:29092,PLAINTEXT_HOST://localhost:9092 + KAFKA_CONTROLLER_LISTENER_NAMES: CONTROLLER + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT,CONTROLLER:PLAINTEXT + KAFKA_CONTROLLER_QUORUM_VOTERS: 1@broker:29093 KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT ``` -Now we can run the `docker-compose up` command to start the Kafka cluster: +Now we can run the `docker compose up -d` command to start the Kafka cluster: ```bash -$ docker-compose up +$ docker compose up -d ``` -This will create a Kafka cluster with one instance of Kafka broker and one instance of Zookeeper. Zookeeper is a distributed service that is used to coordinate broker instances inside the cluster. - ## Writing a Simple Producer and Consumer Using ZIO Workflows -To write producers and consumers, using the ZIO Kafka library, we have two choices: +To write producers and consumers, using the zio-kafka library, we have two choices: -1. Using ZIO Workflows -2. Using ZIO Streams Workflows +1. Using ZIO workflows +2. Using zio-streams workflows In this section, we will try the first option. ### 1. Serializing and Deserializing Data -Before we can write a producer and consumer, let's talk about how data is stored in Kafka. Kafka is an event-store platform that stores key-value pairs as raw bytes. So a Kafka broker knows nothing about its records, it just appends the records to its internal log file. +Before we can write a producer and consumer, let's talk about how data is stored in Kafka. Kafka is an event-store platform that stores records, records are key-value pairs as raw bytes. So a Kafka broker knows nothing about its records, it just appends the records to its internal log file. -So to produce and consume data from Kafka, we need a way to serialize our data to a byte array and deserialize byte arrays to our data types. This is where the `Serde` data type comes in handy. A `Serde[R, T]` is a serializer and deserializer for values of type `T`, which can use the environment `R` to serialize and deserialize values. +To produce and consume data from Kafka, we need a way to serialize our data to a byte array and deserialize byte arrays to our data types. This is where the `Serde` data type comes in handy. A `Serde[R, A]` is a `Serializer` and `Deserializer` for values of type `A`, which can use the environment `R` to serialize and deserialize values. Here is the simplified definition of the `Serde` data type: ```scala -trait Serde[-R, T] { - def deserialize(data: Array[Byte]): RIO[R, T] - def serialize(value: T) : RIO[R, Array[Byte]] +trait Serde[-R, A] { + def deserialize(data: Array[Byte]): RIO[R, A] + def serialize(value: A) : RIO[R, Array[Byte]] } ``` @@ -123,25 +118,27 @@ In this example, the type of the `key` is `Int` and the type of the `value` is ` ### 2. Creating a Producer -ZIO Kafka has several producers that can be used to produce data on Kafka topics. In this example, we will be using the `Producer.produce` method: +Zio-kafka has several producers that can be used to produce data on Kafka topics. In this example, we will be using the `Producer.produce` method: ```scala -object Producer { +trait Producer { def produce[R, K, V]( topic: String, key: K, value: V, keySerializer: Serializer[R, K], valueSerializer: Serializer[R, V] - ): RIO[R with Producer, RecordMetadata] + ): RIO[R, RecordMetadata] } ``` -So let's create a helper function that takes a topic name, key, and value and then returns a ZIO workflow that if we run it, will produce a record to the specified topic: +Notice how the type parameters of `produce` method ensure that the value's type, corresponds to the type produced by the value's `Serializer` (and the same for the key and the `Serializer` for the key). + +Here is a helper function that shows how we could use the `produce` method: ```scala -def produce(topic: String, key: Long, value: String): RIO[Any with Producer, RecordMetadata] = - Producer.produce[Any, Long, String]( +def produceRecord(producer: Producer, topic: String, key: Long, value: String): RIO[Any, RecordMetadata] = + producer.produce[Any, Long, String]( topic = topic, key = key, value = value, @@ -150,26 +147,22 @@ def produce(topic: String, key: Long, value: String): RIO[Any with Producer, Rec ) ``` -The `produce` function is polymorphic in the type of key and value of the record. Based on what type of key and value we pass, we should provide the appropriate `Serde` for the key and value. +### 3. Creating a Producer -### 3. Creating a Producer Layer - -The `produce` workflow requires the `Producer` from the ZIO environment. So we need to provide a `Producer` instance to it. So let's create a `producer` layer: +Now we can create a `Producer`: ```scala mdoc:compile-only import zio._ import zio.kafka._ import zio.kafka.producer._ -val producer: ZLayer[Any, Throwable, Producer] = - ZLayer.scoped( - Producer.make( - ProducerSettings(List("localhost:29092")) - ) +val producer: ZIO[Any, Throwable, Producer] = + Producer.make( + ProducerSettings(List("localhost:9092")) ) ``` -It is sufficient for this example, although the following helper methods are available for more customization: +This is already sufficient for this example, although more helper methods are available for further customization: ```scala class ProducerSettings { @@ -184,125 +177,135 @@ class ProducerSettings { ### 4. Creating a Consumer -ZIO Kafka also has several consumers that can be used to consume data from Kafka topics including the support for ZIO Streams which we will discuss later. In this example, we will use the `Consumer.consumeWith` function. +Zio-kafka provides several ways to consume data from Kafka topics. In this example, we will use the `Consumer.consumeWith` function. -The following helper function will create a ZIO workflow that if we run it, will run forever and consume records from the given topic and finally print them to the console: +The following helper function creates a ZIO workflow that if we run it, runs forever and consumes records from the given topic and prints them to the console, and then commits the offsets of the consumed records: ```scala -def consumeAndPrintEvents(groupId: String, topic: String, topics: String*): RIO[Any, Unit] = - Consumer.consumeWith( +def consumeAndPrintEvents(consumer: Consumer, groupId: String, topic: String, topics: String*): RIO[Any, Unit] = + consumer.consumeWith( settings = ConsumerSettings(BOOSTRAP_SERVERS).withGroupId(groupId), subscription = Subscription.topics(topic, topics: _*), keyDeserializer = Serde.long, valueDeserializer = Serde.string, - )((k, v) => Console.printLine((k, v)).orDie) + ) { (k, v) => + Console.printLine(s"Consumed key: $k, value: $v").orDie + } ``` +For performance reasons, records are always consumed in batches. The `consumeWith` method commits the offsets of consumed records, as soon all records of a batch have been processed. + ### 5. The Complete Example Now it's time to combine all the above steps to create a ZIO workflow that will produce and consume data from the Kafka cluster: ```scala mdoc:compile-only -import org.apache.kafka.clients.producer.RecordMetadata import zio._ import zio.kafka.consumer._ import zio.kafka.producer.{Producer, ProducerSettings} import zio.kafka.serde._ -object SimpleApp extends ZIOAppDefault { - private val BOOSTRAP_SERVERS = List("localhost:29092") - private val KAFKA_TOPIC = "hello" - - private def produce( - topic: String, - key: Long, - value: String - ): RIO[Any with Producer, RecordMetadata] = - Producer.produce[Any, Long, String]( - topic = topic, - key = key, - value = value, - keySerializer = Serde.long, - valueSerializer = Serde.string - ) - - private def consumeAndPrintEvents( - groupId: String, - topic: String, - topics: String* - ): RIO[Any, Unit] = - Consumer.consumeWith( - settings = ConsumerSettings(BOOSTRAP_SERVERS) - .withGroupId(groupId), - subscription = Subscription.topics(topic, topics: _*), - keyDeserializer = Serde.long, - valueDeserializer = Serde.string - )(record => Console.printLine((record.key(), record.value())).orDie) - - private val producer: ZLayer[Any, Throwable, Producer] = - ZLayer.scoped( - Producer.make( - ProducerSettings(BOOSTRAP_SERVERS) - ) - ) - - def run = +/** A simple app that produces and consumes messages from a kafka cluster + * without using ZIO Streams. + */ +object SimpleKafkaApp extends ZIOAppDefault { + private val BOOSTRAP_SERVERS = List("localhost:9092") + private val KAFKA_TOPIC = "hello" + + def run: ZIO[Scope, Throwable, Unit] = { for { - f <- consumeAndPrintEvents("my-consumer-group", KAFKA_TOPIC).fork - _ <- - Clock.currentDateTime - .flatMap { time => - produce(KAFKA_TOPIC, time.getHour, s"$time -- Hello, World!") - } - .schedule(Schedule.spaced(1.second)) - .provide(producer) - _ <- f.join + c <- Consumer + .consumeWith( + settings = + ConsumerSettings(BOOSTRAP_SERVERS).withGroupId("simple-kafka-app"), + subscription = Subscription.topics(KAFKA_TOPIC), + keyDeserializer = Serde.long, + valueDeserializer = Serde.string + ) { record => + Console.printLine(s"Consumed ${record.key()}, ${record.value()}").orDie + } + .fork + + producer <- Producer.make(ProducerSettings(BOOSTRAP_SERVERS)) + p <- Clock.currentDateTime + .flatMap { time => + producer.produce[Any, Long, String]( + topic = KAFKA_TOPIC, + key = time.getHour.toLong, + value = s"$time -- Hello, World!", + keySerializer = Serde.long, + valueSerializer = Serde.string + ) + } + .schedule(Schedule.spaced(1.second)) + .fork + + _ <- (c <*> p).join } yield () + } } ``` -## ZIO Kafka With ZIO Streams +## Zio-kafka With Zio-streams -As we said before, to write producers and consumers using the ZIO Kafka library, we have two choices: +As we said before, to write producers and consumers using the zio-kafka library, we have two choices: -1. Using ZIO Workflows -2. Using ZIO Streams Workflows +1. Using ZIO workflows +2. Using zio-streams workflows -In the previous section, we used the ZIO Kafka with the ZIO Workflow. The ZIO Kafka also works with the ZIO Streams seamlessly. So instead of using the `Producer.produce` and `Consumer.consumeWith` functions, we can use the streaming APIs provided by ZIO Kafka. +In this section we show zio-kafka's seamless integration with zio-streams. ### 1. Streaming Producer API -To produce data using ZIO Streams, ZIO Kafka has a `Producer.produceAll` API, which is a `ZPipeline`. It takes streams of `ProducerRecord[K, V]` as upstream and uses the `Producer` from the environment to produce streams to the Kafka topic and then returns a stream of `RecordMetadata` as downstream: +With zio-kafka's `Producer.produceAll` we get a `ZPipeline`. It takes streams of `ProducerRecord[K, V]`, produces these records to a Kafka topic, and then returns a stream of `RecordMetadata`: ```scala -object Producer { +trait Producer { def produceAll[R, K, V]( keySerializer: Serializer[R, K], valueSerializer: Serializer[R, V] - ): ZPipeline[R with Producer, Throwable, ProducerRecord[K, V], RecordMetadata] = ??? + ): ZPipeline[R, Throwable, ProducerRecord[K, V], RecordMetadata] } ``` -Note that the `ZStream` implicitly chunks the records into batches for the sake of performance. So the `produceAll` produces records in batches instead of one at a time. +Here is an example that uses `Producer.produceAll`: + +```scala +ZStream + .fromIterator(Iterator.from(0), maxChunkSize = 50) // ZStream[Any, Throwable, Int] + .mapChunksZIO { chunk => + chunk.map { i => + new ProducerRecord(topic, key = i, value = s"record $i") + } + } // ZStream[Any, Throwable, ProducerRecord] + .via(producer.produceAll(Serde.int, Serde.string)) // ZStream[Any, Throwable, RecordMetadata] + .runDrain +``` + +For performance reasons, method `produceAll` produces records in batches, every chunk of the input stream results in a batch. ### 2. Streaming Consumer API -Creating a streaming consumer is also simple. We can use the `Consumer.plainStream` API to create a `ZStream` that if we run it, will use the `Consumer` from the environment to consume records from a Kafka topic and then returns a stream of `CommittableRecord[K, V]` as downstream: +The `Consumer.plainStream` method gives a `ZStream` that, when run, consumes records from a Kafka topic and gives a stream of `CommittableRecord[K, V]`: ```scala -object Consumer { +trait Consumer { def plainStream[R, K, V]( + subscription: Subscription, keyDeserializer: Deserializer[R, K], - valueDeserializer: Deserializer[R, V], - bufferSize: Int = 4 - ): ZStream[R with Consumer, Throwable, CommittableRecord[K, V]] = ??? + valueDeserializer: Deserializer[R, V] + ): ZStream[R, Throwable, CommittableRecord[K, V]] } ``` -The `CommittableRecord` is a record that can be committed to Kafka via `CommittableRecord#commit` to indicate that the record has been consumed successfully. After we commit the record that we have consumed, if our application crashes, when we restart, we can resume consuming from the last record that we have committed. +Parameter `subscription` indicates which topics and partitions should be consumed from. For example `Subscription.topics("events")` to consume from the `events` topic. + +Parameters `keyDeserializer` and `valueDeserializer` are the `Serde` that will be used to deserialize the raw record bytes to whatever type you want them to be in. + +To indicate that a record has been consumed successfully, and make sure that no other consumer in the same group will consume this record again (even when the application crashes and restarts), we need to commit the offset of the record. This is how it works: first get the `Offset` of the `CommittableRecord` with the `offset` method, then call `commit` on the returned `Offset`. -For example, if we want to consume records and then save them to a file system, we can run the `CommittableRecord#commit` function after we wrote the record to the file system. So we are sure that the record has been persisted in the file system: +Here is an example: ```scala mdoc:compile-only import zio._ @@ -312,17 +315,18 @@ import zio.kafka.consumer._ import zio.kafka.serde._ val KAFKA_TOPIC = "my-topic" +val consumer: Consumer = ??? -val c: ZStream[Consumer, Throwable, Nothing] = - Consumer +val c: ZIO[Any, Throwable, Unit] = + consumer .plainStream(Subscription.topics(KAFKA_TOPIC), Serde.int, Serde.string) .tap(e => Console.printLine(e.value)) - .map(_.offset) - .mapZIO(_.commit) - .drain + .map(_.offset) // Get the offset of the record + .mapZIO(_.commit) // Commit the offset + .runDrain ``` -The problem with this approach is that we are committing offsets for each record that we consume. This will cause a lot of overhead and will slow down the consumption of the records. To avoid this, we can aggregate the offsets into batches and commit them all at once. This can be done by using the `ZStream#aggregateAsync` along with the `Consumer.offsetBatches` sink: +While this works, it is not very performant. The problem with this approach is that we are committing offsets for each record separately. This causes a lot of overhead and slows down the consumption of records. To avoid this, we can aggregate offsets into batches and commit them all at once. This can be done by using the `ZStream#aggregateAsync` along with the `Consumer.offsetBatches` sink: ```scala mdoc:compile-only import zio._ @@ -332,49 +336,29 @@ import zio.kafka.consumer._ import zio.kafka.serde._ val KAFKA_TOPIC = "my-topic" +val consumer: Consumer = ??? -val c: ZStream[Consumer, Throwable, Nothing] = - Consumer +val c: ZIO[Any, Throwable, Unit] = + consumer .plainStream(Subscription.topics(KAFKA_TOPIC), Serde.int, Serde.string) .tap(e => Console.printLine(e.value)) - .map(_.offset) - .aggregateAsync(Consumer.offsetBatches) - .mapZIO(_.commit) - .drain + .map(_.offset) // Get the offset of the record + .aggregateAsync(Consumer.offsetBatches) // Group offsets in an OffsetBatch + .mapZIO(_.commit) // Commit the batch of offsets + .runDrain ``` -The `Consumer.offsetBatches` sink folds `Offset`s into `OffsetBatch` which contains the maximum offset we have seen so far. So instead of committing the offset for each record, we commit the maximum offset of all the records in the batch. +Notice that because we are using `aggregateAsync`, the commits run asynchronously with the upstream of `aggregateAsync`. Every time a commit finishes, the next `OffsetBatch` is taken and committed. -### 3. Creating a Consumer and Producer Layer +:::caution +Keeping the chunking structure intact is important. -```scala mdoc:compile-only -import zio._ -import zio.stream._ -import zio.kafka._ -import zio.kafka.producer._ -import zio.kafka.consumer._ +In the example so far we have used `tap` to print the records as they are consumed. Unfortunately, methods like `tap` and `mapZIO` destroy the chunking structure and lead to much lower throughput. Please read [a warning about mapZIO](https://zio.dev/zio-kafka/serialization-and-deserialization#a-warning-about-mapzio) for more details and alternatives. +::: -val BOOSTRAP_SERVERS = List("localhost:29092") +### 3. The Complete Streaming Example -private val producer: ZLayer[Any, Throwable, Producer] = - ZLayer.scoped( - Producer.make( - ProducerSettings(BOOSTRAP_SERVERS) - ) - ) - -private val consumer: ZLayer[Any, Throwable, Consumer] = - ZLayer.scoped( - Consumer.make( - ConsumerSettings(BOOSTRAP_SERVERS) - .withGroupId("streaming-kafka-app") - ) - ) -``` - -### 4. The Complete Streaming Example - -It's time to create a full working example of ZIO Kafka with ZIO streams: +It's time to create a full working example of zio-kafka with zio-streams: ```scala mdoc:compile-only import org.apache.kafka.clients.producer.ProducerRecord @@ -385,43 +369,54 @@ import zio.kafka.serde._ import zio.stream.ZStream object StreamingKafkaApp extends ZIOAppDefault { - private val BOOSTRAP_SERVERS = List("localhost:29092") + private val BOOSTRAP_SERVERS = List("localhost:9092") private val KAFKA_TOPIC = "streaming-hello" - private val producer: ZLayer[Any, Throwable, Producer] = - ZLayer.scoped( - Producer.make( - ProducerSettings(BOOSTRAP_SERVERS) - ) - ) - - private val consumer: ZLayer[Any, Throwable, Consumer] = - ZLayer.scoped( - Consumer.make( - ConsumerSettings(BOOSTRAP_SERVERS) - .withGroupId("streaming-kafka-app") - ) - ) - - def run = { - val p: ZStream[Producer, Throwable, Nothing] = - ZStream - .repeatZIO(Clock.currentDateTime) - .schedule(Schedule.spaced(1.second)) - .map(time => new ProducerRecord(KAFKA_TOPIC, time.getMinute, s"$time -- Hello, World!")) - .via(Producer.produceAll(Serde.int, Serde.string)) - .drain - - val c: ZStream[Consumer, Throwable, Nothing] = - Consumer - .plainStream(Subscription.topics(KAFKA_TOPIC), Serde.int, Serde.string) - .tap(e => Console.printLine(e.value)) - .map(_.offset) - .aggregateAsync(Consumer.offsetBatches) - .mapZIO(_.commit) - .drain - - (p merge c).runDrain.provide(producer, consumer) + private val producerSettings = ProducerSettings(BOOSTRAP_SERVERS) + private val consumerSettings = + ConsumerSettings(BOOSTRAP_SERVERS).withGroupId("streaming-kafka-app") + + def run: ZIO[Any, Throwable, Unit] = { + val p: ZIO[Any, Throwable, Unit] = + ZIO.scoped { + for { + producer <- Producer.make(producerSettings) + _ <- ZStream + .repeatZIO(Clock.currentDateTime) + .schedule(Schedule.spaced(1.second)) + .map { time => + new ProducerRecord( + KAFKA_TOPIC, + time.getMinute, + s"$time -- Hello, World!" + ) + } + .via(producer.produceAll(Serde.int, Serde.string)) + .runDrain + } yield () + } + + val c: ZIO[Any, Throwable, Unit] = + ZIO.scoped { + for { + consumer <- Consumer.make(consumerSettings) + _ <- consumer + .plainStream( + Subscription.topics(KAFKA_TOPIC), + Serde.int, + Serde.string + ) + // do not use `tap` in prod because it destroys the chunking structure and leads to lower performance + // See https://zio.dev/zio-kafka/serialization-and-deserialization#a-warning-about-mapzio + .tap(r => Console.printLine("Consumed: " + r.value)) + .map(_.offset) + .aggregateAsync(Consumer.offsetBatches) + .mapZIO(_.commit) + .runDrain + } yield () + } + + p <&> c } } @@ -429,30 +424,29 @@ object StreamingKafkaApp extends ZIOAppDefault { ## Producing and Consuming JSON Data -Until now, we learned how to work with simple primitive types like `Int`, `String`, etc and how to use their `Serde` instances to encode and decode the data. +Until now, we learned how to work with simple types like `Int`, `String`, and how to use their `Serde` instances to serialize and deserialize the data. -In this section, we are going to learn how to work with user-defined data types (like case classes e.g. `Event`), and how to produce and consume the JSON data representing our user-defined data types. We also will learn how to use the `Serde` built-in instances to create more complex `Serde` instances. +In this section, we are going to learn how to serialize/deserialize user-defined data types (like case classes), and in particular how to use the JSON format. We also will learn how to use the `Serde` built-in instances to create more complex `Serde` instances. ### 1. Writing Custom Serializer and Deserializer -In ZIO Kafka all of the built-in serializers/deserializers are instances of the `Serde` trait, which has two useful methods: +In zio-kafka all the built-in serializers/deserializers are instances of the `Serde` trait. All `Serde`s offers several combinators with which we can create new `Serde`s. We take a closer look at 2 of them: + +- Method `inmap` transforms the `Serde` with pure transformations from the source type to the target type and back. +- Method `inmapZIO` transforms the `Serde` with effectful transformations from the source type to the target type and back. As it accepts effectful transformations, we can encode any parsing failure with a `ZIO` workflow. + +In this example, we use `inmap` to transform the build-in `Serde.long` to a `Serde[Any, Instant]`. The `inmap` method gets 2 pure functions, the first converts from `Long` to `Instant`, the second from `Instant` to `Long`: ```scala -trait Serde[-R, T] extends Deserializer[R, T] with Serializer[R, T] { - def inmap[U](f: T => U)(g: U => T): Serde[R, U] = - Serde(map(f))(contramap(g)) +import java.time.Instant - def inmapM[R1 <: R, U](f: T => RIO[R1, U])(g: U => RIO[R1, T]): Serde[R1, U] = - Serde(mapM(f))(contramapM(g)) -} +val instantSerde: Serde[Any, Instant] = + Serde.long.inmap[Instant](Instant.ofEpochMilli)(_.toEpochMilli) ``` -Using the `inmap` and `inmapM` combinators, we can create our own serializers and deserializers on top of the built-in ones: +In the following example, we will use `inmapZIO` to transform the built-in `Serde.string` to a `Serde[Any, Event]` where `Event` is a case class that is serialized to/deserialized from a String containing JSON. -- The `inmap` is used to transform the `Serde` type `U` with pure transformations of `f` and `g`. -- The `inmapM` is used to transform the `Serde` type `U` with effectful transformations of `f` and `g`. As it accepts effectful transformations, we can encode any parsing failure with a `ZIO` workflow. - -Let's say we have a case class `Event` with the following fields: +Let's say we have the `Event` case class with the following fields: ```scala mdoc:silent import java.time.OffsetDateTime @@ -479,53 +473,29 @@ object Event { } ``` -Then we need to create a `Serde` for the `Event` type. To convert `Event` to JSON and back, we will use the ZIO JSON library, and to define `Serde` for the `Event` type, we will use the `Serde#inmapM` combinator: +Then we need to create a `Serde` for the `Event` type. To convert `Event` to JSON and back, we will use the zio-json library. To define a `Serde` for the `Event` type, we will use the `Serde.string.inmapZIO` combinator: ```scala mdoc:silent import zio._ import zio.kafka.serde._ -object KafkaSerde { - val key: Serde[Any, Int] = - Serde.int - - val value: Serde[Any, Event] = - Serde.string.inmapM[Any, Event](s => - ZIO.fromEither(s.fromJson[Event]) +object EventKafkaSerde { + val event: Serde[Any, Event] = + Serde.string.inmapZIO[Any, Event](s => + ZIO + .fromEither(s.fromJson[Event]) .mapError(e => new RuntimeException(e)) )(r => ZIO.succeed(r.toJson)) } ``` -As we can see, we use the `String#fromJson` to convert the string to an `Event` object and we also encode any parsing failure with a `RuntimeException` in the `ZIO` workflow. - -### 2. Using the Custom Serde - -After we have defined our custom `Serde` for the `Event` type, we can use it in our Kafka producer and consumer streams: - -```scala mdoc:compile-only -import zio._ -import zio.stream._ -import zio.kafka.serde._ -import zio.kafka.producer._ -import zio.kafka.consumer._ -import org.apache.kafka.clients.producer.ProducerRecord +As we can see, we use the `String#fromJson` to convert the string to an `Event` object, and we also encode any parsing failure with a `RuntimeException` in the `ZIO` workflow. -val KAFKA_TOPIC = "json-streaming-hello" +See [](https://zio.dev/zio-kafka/serialization-and-deserialization) -val events: UStream[ProducerRecord[Int, Event]] = ??? +### 2. The Complete JSON Streaming Example -val producer = - events.via(Producer.produceAll(KafkaSerde.key, KafkaSerde.value)) - -val consumer = - Consumer - .plainStream(Subscription.topics(KAFKA_TOPIC), KafkaSerde.key, KafkaSerde.value) -``` - -### 3. The Complete JSON Streaming Example - -Here is a full working example of producing and consuming JSON data with ZIO Kafka, ZIO Streams and ZIO JSON: +Here is a full working example of producing and consuming JSON data with zio-kafka, zio-streams and zio-json: ```scala mdoc:compile-only import org.apache.kafka.clients.producer.ProducerRecord @@ -539,8 +509,10 @@ import zio.stream.ZStream import java.time.OffsetDateTime import java.util.UUID +/** This is the data we will be sending to Kafka in JSON format. */ case class Event(uuid: UUID, timestamp: OffsetDateTime, message: String) +/** A zio-json encoder/decoder for [[Event]]. */ object Event { implicit val encoder: JsonEncoder[Event] = DeriveJsonEncoder.gen[Event] @@ -549,61 +521,66 @@ object Event { DeriveJsonDecoder.gen[Event] } -object KafkaSerde { - val key: Serde[Any, Int] = - Serde.int - - val value: Serde[Any, Event] = - Serde.string.inmapM[Any, Event](s => - ZIO.fromEither(s.fromJson[Event]) +/** A zio-kafka serializer/deserializer for [[Event]]. */ +object EventKafkaSerde { + val event: Serde[Any, Event] = + Serde.string.inmapZIO[Any, Event](s => + ZIO + .fromEither(s.fromJson[Event]) .mapError(e => new RuntimeException(e)) )(r => ZIO.succeed(r.toJson)) } object JsonStreamingKafkaApp extends ZIOAppDefault { - private val BOOSTRAP_SERVERS = List("localhost:29092") + private val BOOSTRAP_SERVERS = List("localhost:9092") private val KAFKA_TOPIC = "json-streaming-hello" - private val producer: ZLayer[Any, Throwable, Producer] = - ZLayer.scoped( - Producer.make( - ProducerSettings(BOOSTRAP_SERVERS) - ) - ) - - private val consumer: ZLayer[Any, Throwable, Consumer] = - ZLayer.scoped( - Consumer.make( - ConsumerSettings(BOOSTRAP_SERVERS) - .withGroupId("streaming-kafka-app") - ) - ) - - def run = { - val p: ZStream[Producer, Throwable, Nothing] = - ZStream - .repeatZIO(Random.nextUUID <*> Clock.currentDateTime) - .schedule(Schedule.spaced(1.second)) - .map { case (uuid, time) => - new ProducerRecord( - KAFKA_TOPIC, - time.getMinute, - Event(uuid, time, "Hello, World!") + def run: ZIO[Any, Throwable, Unit] = { + val p: ZIO[Any, Throwable, Unit] = + ZIO.scoped { + for { + producer <- Producer.make(ProducerSettings(BOOSTRAP_SERVERS)) + _ <- ZStream + .repeatZIO(Random.nextUUID <*> Clock.currentDateTime) + .schedule(Schedule.spaced(1.second)) + .map { case (uuid, time) => + new ProducerRecord( + KAFKA_TOPIC, + time.getMinute, + Event(uuid, time, "Hello, World!") + ) + } + .via(producer.produceAll(Serde.int, EventKafkaSerde.event)) + .runDrain + } yield () + } + + val c: ZIO[Any, Throwable, Unit] = + ZIO.scoped { + for { + consumer <- Consumer.make( + ConsumerSettings(BOOSTRAP_SERVERS).withGroupId("streaming-kafka-app") ) - } - .via(Producer.produceAll(KafkaSerde.key, KafkaSerde.value)) - .drain - - val c: ZStream[Consumer, Throwable, Nothing] = - Consumer - .plainStream(Subscription.topics(KAFKA_TOPIC), KafkaSerde.key, KafkaSerde.value) - .tap(e => Console.printLine(e.value)) - .map(_.offset) - .aggregateAsync(Consumer.offsetBatches) - .mapZIO(_.commit) - .drain - - (p merge c).runDrain.provide(producer, consumer) + _ <- consumer + .plainStream( + Subscription.topics(KAFKA_TOPIC), + Serde.int, + EventKafkaSerde.event + ) + .tap { r => + val event: Event = r.value + Console.printLine( + s"Event ${event.uuid} was sent at ${event.timestamp} with message ${event.message}" + ) + } + .map(_.offset) + .aggregateAsync(Consumer.offsetBatches) + .mapZIO(_.commit) + .runDrain + } yield () + } + + p <&> c } } @@ -611,6 +588,6 @@ object JsonStreamingKafkaApp extends ZIOAppDefault { ## Conclusion -In this tutorial first, we learned how to create a producer and consumer for Kafka using the ZIO workflow with ZIO Kafka. Then we learned how to do the same with ZIO Streams. We also learned how to create a custom serializer and deserializer for the Kafka records and how to produce and consume JSON data using the ZIO JSON library. +In this tutorial we first learned how to create a producer and consumer for Kafka using the ZIO workflow with zio-kafka. Then we learned how to do the same with zio-streams. We also learned how to create a custom serializer and deserializer for the Kafka records and how to produce and consume JSON data using the zio-json library. -All the source code associated with this article is available on the [ZIO Quickstart](http://github.com/zio/zio-quickstarts) project on Github. +All the source code associated with this article is available on the [ZIO Quickstart](http://github.com/zio/zio-quickstarts) project on GitHub. From c05ff1083af09552c377a744c6818e71424e5e0c Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 9 Feb 2025 17:19:07 +0100 Subject: [PATCH 2/2] Fix compilation error, fill in some gaps --- ...oduce-consume-data-to-from-kafka-topics.md | 45 +++++++++++++------ 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md b/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md index 9bf41c2a151c..5a970e3cb0cd 100644 --- a/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md +++ b/docs/guides/tutorials/produce-consume-data-to-from-kafka-topics.md @@ -168,24 +168,16 @@ import zio._ import zio.kafka._ import zio.kafka.producer._ -val producer: ZIO[Any, Throwable, Producer] = +val producer: ZIO[Scope, Throwable, Producer] = Producer.make( ProducerSettings(List("localhost:9092")) ) ``` -This is already sufficient for this example, although more helper methods are available for further customization: +The `ProducerSettings` as constructed in this code fragment is already sufficient for many applications. However, more configuration options are available on `ProducerSettings`. -```scala -class ProducerSettings { - def withBootstrapServers(servers: List[String]): ProducerSettings - def withClientId(clientId: String) : ProducerSettings - def withCloseTimeout(duration: Duration) : ProducerSettings - def withProperty(key: String, value: AnyRef) : ProducerSettings - def withProperties(kvs: (String, AnyRef)*) : ProducerSettings - def withProperties(kvs: Map[String, AnyRef]) : ProducerSettings -} -``` +Notice that the created producer requires a `Scope` in the environment. When this scope closes, the producer closes its +connection with the Kafka cluster. An explicit scope can be created with the `ZIO.scoped` method. ### 4. Creating a Consumer @@ -207,6 +199,8 @@ def consumeAndPrintEvents(consumer: Consumer, groupId: String, topic: String, to For performance reasons, records are always consumed in batches. The `consumeWith` method commits the offsets of consumed records, as soon all records of a batch have been processed. +For more options see [consumer tuning](https://zio.dev/zio-kafka/consumer-tuning). + ### 5. The Complete Example Now it's time to combine all the above steps to create a ZIO workflow that will produce and consume data from the Kafka cluster: @@ -297,7 +291,28 @@ ZStream For performance reasons, method `produceAll` produces records in batches, every chunk of the input stream results in a batch. -### 2. Streaming Consumer API +### 2. Creating a Consumer + +When we use the streaming API we need to construct a Consumer: + +```scala mdoc:compile-only +import zio._ +import zio.kafka._ +import zio.kafka.consumer._ + +val consumer: ZIO[Scope, Throwable, Consumer] = + Consumer.make( + ConsumerSettings(List("localhost:9092")) + .withGroupId("streaming-kafka-app") + ) +``` + +Notice that the consumer requires a `Scope` in the environment. When this scope closes, the consumer closes its +connection with the Kafka cluster. An explicit scope can be created with the `ZIO.scoped` method. + +For more options see [creating a consumer](https://zio.dev/zio-kafka/creating-a-consumer) and [consumer tuning](https://zio.dev/zio-kafka/consumer-tuning). + +### 3. Streaming Consumer API The `Consumer.plainStream` method gives a `ZStream` that, when run, consumes records from a Kafka topic and gives a stream of `CommittableRecord[K, V]`: @@ -368,7 +383,9 @@ Keeping the chunking structure intact is important. In the example so far we have used `tap` to print the records as they are consumed. Unfortunately, methods like `tap` and `mapZIO` destroy the chunking structure and lead to much lower throughput. Please read [a warning about mapZIO](https://zio.dev/zio-kafka/serialization-and-deserialization#a-warning-about-mapzio) for more details and alternatives. ::: -### 3. The Complete Streaming Example +For more details see [consuming Kafka topics using ZIO Streams](https://zio.dev/zio-kafka/consuming-kafka-topics-using-zio-streams). + +### 4. The Complete Streaming Example It's time to create a full working example of zio-kafka with zio-streams: