From 9a60ccdde0f4c5da69333731b8d22f9e3be9424b Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 14 Jul 2024 12:01:34 +0200 Subject: [PATCH 01/42] Small improvements to the Producer (#1272) By using ZIO.async, we no longer need a reference to the zio runtime, nor do we need the `exec` trickery anymore. --- .../scala/zio/kafka/producer/Producer.scala | 85 +++++++++---------- .../producer/TransactionalProducer.scala | 5 +- 2 files changed, 43 insertions(+), 47 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index 645f5f7ae..939f36d31 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -220,13 +220,12 @@ object Producer { settings: ProducerSettings ): ZIO[Scope, Throwable, Producer] = for { - runtime <- ZIO.runtime[Any] sendQueue <- Queue.bounded[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])]( settings.sendBufferSize ) - producer = new ProducerLive(javaProducer, runtime, sendQueue) - _ <- ZIO.blocking(producer.sendFromQueue).forkScoped + producer = new ProducerLive(javaProducer, sendQueue) + _ <- producer.sendFromQueue.forkScoped } yield producer /** @@ -362,7 +361,6 @@ object Producer { private[producer] final class ProducerLive( private[producer] val p: JProducer[Array[Byte], Array[Byte]], - runtime: Runtime[Any], sendQueue: Queue[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])] ) extends Producer { @@ -460,51 +458,52 @@ private[producer] final class ProducerLive( /** * Calls to send may block when updating metadata or when communication with the broker is (temporarily) lost, - * therefore this stream is run on a the blocking thread pool + * therefore this stream is run on the blocking thread pool. */ val sendFromQueue: ZIO[Any, Nothing, Any] = - ZStream - .fromQueueWithShutdown(sendQueue) - .mapZIO { case (serializedRecords, done) => - ZIO.succeed { - try { - val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex - val res: Array[Either[Throwable, RecordMetadata]] = - new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) - val count: AtomicLong = new AtomicLong - val length = serializedRecords.length - - while (it.hasNext) { - val (rec, idx): (ByteRecord, Int) = it.next() - - val _ = p.send( - rec, - (metadata: RecordMetadata, err: Exception) => - Unsafe.unsafe { implicit u => - exec { - if (err != null) res(idx) = Left(err) - else res(idx) = Right(metadata) - - if (count.incrementAndGet == length) { - exec { - runtime.unsafe.run(done.succeed(Chunk.fromArray(res))).getOrThrowFiberFailure() - } - } - } - } - ) - } - } catch { - case NonFatal(e) => - Unsafe.unsafe { implicit u => - exec { - runtime.unsafe.run(done.succeed(Chunk.fill(serializedRecords.size)(Left(e)))).getOrThrowFiberFailure() + ZIO.blocking { + ZStream + .fromQueueWithShutdown(sendQueue) + .mapZIO { case (serializedRecords, done) => + sendChunk(serializedRecords) + .flatMap(done.succeed(_)) + } + .runDrain + } + + private def sendChunk( + serializedRecords: Chunk[ByteRecord] + ): ZIO[Any, Nothing, Chunk[Either[Throwable, RecordMetadata]]] = + ZIO + .async[Any, Nothing, Chunk[Either[Throwable, RecordMetadata]]] { callback => + try { + val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex + val res: Array[Either[Throwable, RecordMetadata]] = + new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) + val count: AtomicLong = new AtomicLong + val length = serializedRecords.length + + while (it.hasNext) { + val (rec, idx): (ByteRecord, Int) = it.next() + + // Since we might be sending to multiple partitions, the callbacks + // are _not_ necessarily called in order. + val _ = p.send( + rec, + (metadata: RecordMetadata, err: Exception) => { + res(idx) = Either.cond(err == null, metadata, err) + + if (count.incrementAndGet == length) { + callback(ZIO.succeed(Chunk.fromArray(res))) } } + ) } + } catch { + case NonFatal(e) => + callback(ZIO.succeed(Chunk.fill(serializedRecords.size)(Left(e)))) } } - .runDrain private def serialize[R, K, V]( r: ProducerRecord[K, V], @@ -516,6 +515,4 @@ private[producer] final class ProducerLive( value <- valueSerializer.serialize(r.topic, r.headers, r.value()) } yield new ProducerRecord(r.topic, r.partition(), r.timestamp(), key, value, r.headers) - /** Used to prevent warnings about not using the result of an expression. */ - @inline private def exec[A](f: => A): Unit = { val _ = f } } diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala index 173f43100..c27f90633 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala @@ -96,12 +96,11 @@ object TransactionalProducer { )(p => ZIO.attemptBlocking(p.close(settings.producerSettings.closeTimeout)).orDie) _ <- ZIO.attemptBlocking(rawProducer.initTransactions()) semaphore <- Semaphore.make(1) - runtime <- ZIO.runtime[Any] sendQueue <- Queue.bounded[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])]( settings.producerSettings.sendBufferSize ) - live = new ProducerLive(rawProducer, runtime, sendQueue) - _ <- ZIO.blocking(live.sendFromQueue).forkScoped + live = new ProducerLive(rawProducer, sendQueue) + _ <- live.sendFromQueue.forkScoped } yield new LiveTransactionalProducer(live, semaphore) } From e6af87433d59e203845a93b8bce7a5d805c8f975 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 14 Jul 2024 19:52:21 +0200 Subject: [PATCH 02/42] Document metrics and consumer tuning based on metrics (#1280) Also: fix typo and make metric descriptions consistent. --- docs/consumer-internals.svg | 4 + docs/consumer-tuning.md | 36 +++++++ docs/metrics.md | 94 +++++++++++++++++++ docs/sidebars.js | 1 + .../consumer/internal/ConsumerMetrics.scala | 14 +-- 5 files changed, 142 insertions(+), 7 deletions(-) create mode 100644 docs/consumer-internals.svg create mode 100644 docs/metrics.md diff --git a/docs/consumer-internals.svg b/docs/consumer-internals.svg new file mode 100644 index 000000000..efe38baad --- /dev/null +++ b/docs/consumer-internals.svg @@ -0,0 +1,4 @@ + + + +
Runloop

2. store request
3. resume partition
4. poll broker
5. send records
Runloop...
kafka broker
kafka br...
Zstream
Zstream
records
records
1. Out of data? Send a request
1. Out of data?...
6. Emit records
6. Emit records
zio-kafka consumer internals for a single partition
zio-kafka consumer internals for a single partition
requests
requests
Text is not SVG - cannot display
\ No newline at end of file diff --git a/docs/consumer-tuning.md b/docs/consumer-tuning.md index 3e34c521d..5282ee543 100644 --- a/docs/consumer-tuning.md +++ b/docs/consumer-tuning.md @@ -90,3 +90,39 @@ On older zio-kafka versions `withMaxPollInterval` is not available. Use the foll ⚠️In zio-kafka versions 2.2 up to 2.5.0 it may also be necessary to increase the `runloopTimeout` setting. When no stream is processing data for this amount of time (while new data is available), the consumer will halt with a failure. In zio-kafka 2.5.0 `runloopTimeout` defaults to 4 minutes, a little bit lower than `max.poll.interval.ms`. + +## Using metrics to tune the consumer + +Zio-Kafka exposes [metrics](metrics.md) that can be used to further tune the consumer. To interpret these metrics you need to know how zio-kafka works internally. + +![](consumer-internals.svg) + +The runloop is at the heart of every zio-kafka consumer. +It creates a zstream for each partition, eventually this is the zstream your applications consumes from. +When the zstream starts, and every time the records queue is empty, it sends a request for data to the runloop. +The request causes the runloop to resume the partition so that the next poll may receive records. +Any received records are put in the records queue. +When the queue reaches a certain size (as determined by the configured `FetchStrategy`), the partition is paused. +Meanwhile, the zstream reads from the queue and emits the records to your application. + +An optimally configured consumer has the following properties: + +- the zstreams never have to wait for new records (to get high throughput), +- most of the time, the record queues are empty (to get low latency and low heap usage). + +The following strategy can help you get to this state: + +1. First make sure that `pollTimeout` and `max.poll.records` make sense for the latency and throughput requirements + of your application. +2. Configure `partitionPreFetchBufferLimit` to `0`. +3. Observe metric `ziokafka_consumer_queue_polls` which gives the number of polls during which records are idling in + the queue. +4. Increase `partitionPreFetchBufferLimit` in steps until most measurements of the `ziokafka_consumer_queue_polls` + histogram are in the `0` bucket . + +During this process, it is useful to observe metric `ziokafka_consumer_queue_size` (number of records in the queues) to +see if the queues are indeed increasing in size. + +When many (hundreds of) partitions need to be consumed, the metric `ziokafka_consumer_all_queue_size` should also be +observed as increasing `partitionPreFetchBufferLimit` can lead to high heap usage. (See 'High number of partitions' +above.) diff --git a/docs/metrics.md b/docs/metrics.md new file mode 100644 index 000000000..98479038c --- /dev/null +++ b/docs/metrics.md @@ -0,0 +1,94 @@ +--- +id: metrics +title: "Zio-Kafka Metrics" +--- + +Zio-kafka exposes all the metrics of the wrapped Java based consumer and producer, plus some more metrics about the +zio-kafka consumer itself. + +## Java client metrics + +The metrics from the Java metrics can be obtained via the `Consumer.metrics` and `Producer.metrics` methods. Both +return a live view on the internal metrics of the consumer/producer. We currently do not expose these metrics elsewhere, +a PR to copy them to the zio-metrics API is welcome. + +## Zio-kafka consumer metrics + +The zio-kafka consumer collects some additional metrics using the zio-metrics API. This allows any zio-metrics backend +to access and process the observed values. + +By default, no tags are added. Tags can be configured via `ConsumerSettings.withMetricsLabels`. + +Like the zio-metrics we follow Prometheus conventions. This means that: + +- durations are expressed in seconds, +- counters can only increase, +- metric names use snake_case and end in the unit where possible. + +The histograms each use 10 buckets. To reach a decent range while keeping sufficient accuracy at the low end, most +bucket boundaries use an exponential series based on 𝑒. + +### Poll metrics + +| Type | Name | Description | +|-----------|-------------------------------------------------------|------------------------------------------------------------------------------------| +| counter | `ziokafka_consumer_polls` | The number of polls. | +| histogram | `ziokafka_consumer_poll_latency_seconds` | The duration of a single poll in seconds. | +| histogram | `ziokafka_consumer_poll_size` | The number of records fetched by a single poll. | +| gauge | `ziokafka_consumer_partitions_resumed_in_latest_poll` | The number of partitions resumed in the latest poll call. | +| gauge | `ziokafka_consumer_partitions_paused_in_latest_poll` | The number of partitions paused in the latest poll call (because of backpressure). | +| counter | `ziokafka_consumer_poll_auth_errors` | The number of polls that ended with an authentication or authorization error. | + +### Partition stream metrics + +These metrics are updated after every poll. + +| Type | Name | Description | +|-----------|--------------------------------------|------------------------------------------------------------------------| +| histogram | `ziokafka_consumer_pending_requests` | The number of partitions that ran out of records (the queue is empty). | +| histogram | `ziokafka_consumer_queue_size` | The number of records queued for a partition. | +| histogram | `ziokafka_consumer_all_queue_size` | The total number of records queued for all partitions. | +| histogram | `ziokafka_consumer_queue_polls` | The number of polls during which records are idling in a queue. | + +### Commit metrics + +These metrics measure the separate commit requests issued through zio-kafka's api. + +| Type | Name | Description | +|------------|--------------------------------------------|-----------------------------------------------------| +| histogram | `ziokafka_consumer_pending_commits` | The number of commits that are awaiting completion. | +| counterInt | `ziokafka_consumer_commits` | The number of commits. | +| histogram | `ziokafka_consumer_commit_latency_seconds` | The duration of a commit in seconds. | + + +### Aggregated commit metrics + +After every poll zio-kafka combines all outstanding commit requests into 1 aggregated commit. These metrics are for the aggregated commits. + +| Type | Name | Description | +|------------|-------------------------------------------------------|----------------------------------------------------------------------------| +| counterInt | `ziokafka_consumer_aggregated_commits` | The number of aggregated commits. | +| histogram | `ziokafka_consumer_aggregated_commit_latency_seconds` | The duration of an aggregated commit in seconds. | +| histogram | `ziokafka_consumer_aggregated_commit_size` | An approximation of the number of records (offsets) per aggregated commit. | + +### Rebalance metrics + +| Type | Name | Description | +|------------|---------------------------------------------------|--------------------------------------------------------------| +| counterInt | `ziokafka_consumer_rebalances` | The number of rebalances. | +| gauge | `ziokafka_consumer_partitions_currently_assigned` | The number of partitions currently assigned to the consumer. | +| counterInt | `ziokafka_consumer_partitions_assigned` | The number of partitions assigned to the consumer. | +| counterInt | `ziokafka_consumer_partitions_revoked` | The number of partitions revoked to the consumer. | +| counterInt | `ziokafka_consumer_partitions_lost` | The number of partitions lost to the consumer. | + +### Runloop metrics + +These metrics are updated after every poll. + +| Type | Name | Description | +|-----------|----------------------------------------|----------------------------------------------------| +| gauge | `ziokafka_consumer_subscription_state` | Whether the consumer is subscribed (1) or not (0). | +| histogram | `ziokafka_consumer_command_queue_size` | The number of commands queued in the consumer. | +| histogram | `ziokafka_consumer_commit_queue_size` | The number of commits queued in the consumer. | + +See [ConsumerMetrics.scala](https://github.com/zio/zio-kafka/blob/master/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala) for the exact details. diff --git a/docs/sidebars.js b/docs/sidebars.js index 557af2bdd..a6d28fb87 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -9,6 +9,7 @@ const sidebars = { "consuming-kafka-topics-using-zio-streams", "example-of-consuming-producing-and-committing-offsets", "partition-assignment-and-offset-retrieval", + "metrics", "consumer-tuning", "preventing-duplicates", "sharing-consumer", diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala index ffe5844df..f43329a4f 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala @@ -193,14 +193,14 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten private val rebalanceCounter: Metric.Counter[Int] = Metric - .counterInt("ziokafka_consumer_rebalances", "The number of rebalances") + .counterInt("ziokafka_consumer_rebalances", "The number of rebalances.") .tagged(metricLabels) private val partitionsCurrentlyAssignedGauge: Metric.Gauge[Int] = Metric .gauge( "ziokafka_consumer_partitions_currently_assigned", - "The number of partitions currently assigned to the consumer" + "The number of partitions currently assigned to the consumer." ) .contramap[Int](_.toDouble) .tagged(metricLabels) @@ -209,7 +209,7 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten Metric .counterInt( s"ziokafka_consumer_partitions_$state", - s"The number of partitions $state to the consumer" + s"The number of partitions $state to the consumer." ) .tagged(metricLabels) @@ -251,7 +251,7 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten Metric .histogram( "ziokafka_consumer_pending_requests", - "The number of partition queues that that ran out of records.", + "The number of partitions that ran out of records (the queue is empty).", streamCountBoundaries ) .contramap[Int](_.toDouble) @@ -271,7 +271,7 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten Metric .histogram( "ziokafka_consumer_queue_size", - "The number of records in a partition queue.", + "The number of records queued for a partition.", streamSizeBoundaries ) .contramap[Int](_.toDouble) @@ -281,7 +281,7 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten Metric .histogram( "ziokafka_consumer_queue_polls", - "The number of polls during which records are idling in a partition queue.", + "The number of polls during which records are idling in a queue.", queuePollSizeBoundaries ) .contramap[Int](_.toDouble) @@ -291,7 +291,7 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten Metric .histogram( "ziokafka_consumer_all_queue_size", - "The total number of records in all partition queues.", + "The total number of records queued for all partitions.", streamSizeBoundaries ) .contramap[Int](_.toDouble) From 7452812c5410c61b03c8cda5614f69067c075ed6 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Tue, 16 Jul 2024 19:15:59 +0200 Subject: [PATCH 03/42] Add alternative fetch strategy for many partitions (#1281) When many hundreds of partitions need to be consumed, an excessive amount of heap can be used for pre-fetching. The `ManyPartitionsQueueSizeBasedFetchStrategy` works similarly as the default `QueueSizeBasedFetchStrategy` but limits total memory usage. --- docs/consumer-tuning.md | 2 +- ...tionsQueueSizeBasedFetchStrategySpec.scala | 117 ++++++++++++++++++ .../kafka/consumer/fetch/FetchStrategy.scala | 2 +- ...artitionsQueueSizeBasedFetchStrategy.scala | 59 +++++++++ 4 files changed, 178 insertions(+), 2 deletions(-) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategySpec.scala create mode 100644 zio-kafka/src/main/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategy.scala diff --git a/docs/consumer-tuning.md b/docs/consumer-tuning.md index 5282ee543..625e5c336 100644 --- a/docs/consumer-tuning.md +++ b/docs/consumer-tuning.md @@ -57,7 +57,7 @@ the partition queues. A very rough estimate for the maximum amount of heap neede The total can be tuned by changing the `partitionPreFetchBufferLimit`, `max.poll.records` settings. Another option is to write a custom `FetchStrategy`. For example the `ManyPartitionsQueueSizeBasedFetchStrategy` in -[draft PR 970](https://github.com/zio/zio-kafka/pull/970) (not yet tested at scale, use at your own risk). Note that the fetch strategy API is marked as +[draft PR 970](https://github.com/zio/zio-kafka/pull/970) (merged into zio-kafka since 2.8.1). Note that the fetch strategy API is marked as experimental and may change without notice in any future zio-kafka version. ## Long processing durations diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategySpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategySpec.scala new file mode 100644 index 000000000..e18a736c4 --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategySpec.scala @@ -0,0 +1,117 @@ +package zio.kafka.consumer.fetch + +import org.apache.kafka.common.TopicPartition +import zio.kafka.ZIOSpecDefaultSlf4j +import zio.kafka.consumer.internal.PartitionStream +import zio.test.{ assertTrue, Spec, TestEnvironment } +import zio.{ Chunk, Scope, UIO, ZIO } + +object ManyPartitionsQueueSizeBasedFetchStrategySpec extends ZIOSpecDefaultSlf4j { + + private val maxPartitionQueueSize = 50 + private val fetchStrategy = ManyPartitionsQueueSizeBasedFetchStrategy( + maxPartitionQueueSize, + maxTotalQueueSize = 80 + ) + + private val tp10 = new TopicPartition("topic1", 0) + private val tp11 = new TopicPartition("topic1", 1) + private val tp20 = new TopicPartition("topic2", 0) + private val tp21 = new TopicPartition("topic2", 1) + private val tp22 = new TopicPartition("topic2", 2) + + override def spec: Spec[TestEnvironment with Scope, Any] = + suite("ManyPartitionsQueueSizeBasedFetchStrategySpec")( + test("stream with queue size above maxSize is paused") { + val streams = Chunk(newStream(tp10, currentQueueSize = 100)) + for { + result <- fetchStrategy.selectPartitionsToFetch(streams) + } yield assertTrue(result.isEmpty) + }, + test("stream with queue size below maxSize may resume when less-equal global max") { + val streams = Chunk(newStream(tp10, currentQueueSize = 10)) + for { + result <- fetchStrategy.selectPartitionsToFetch(streams) + } yield assertTrue(result == Set(tp10)) + }, + test("all streams with queue size less-equal maxSize may resume when total is less-equal global max") { + val streams = Chunk( + newStream(tp10, currentQueueSize = maxPartitionQueueSize), + newStream(tp11, currentQueueSize = 10), + newStream(tp20, currentQueueSize = 10), + newStream(tp21, currentQueueSize = 10) + ) + for { + result <- fetchStrategy.selectPartitionsToFetch(streams) + } yield assertTrue(result == Set(tp10, tp11, tp20, tp21)) + }, + test("not all streams with queue size less-equal maxSize may resume when total is less-equal global max") { + val streams = Chunk( + newStream(tp10, currentQueueSize = 40), + newStream(tp11, currentQueueSize = 40), + newStream(tp20, currentQueueSize = 40), + newStream(tp21, currentQueueSize = 40) + ) + for { + result <- fetchStrategy.selectPartitionsToFetch(streams) + } yield assertTrue(result.size == 2) + }, + test("all streams with queue size less-equal maxSize may resume eventually") { + val streams = Chunk( + newStream(tp10, currentQueueSize = 60), + newStream(tp11, currentQueueSize = 60), + newStream(tp20, currentQueueSize = 40), + newStream(tp21, currentQueueSize = 40), + newStream(tp22, currentQueueSize = 40) + ) + for { + result1 <- fetchStrategy.selectPartitionsToFetch(streams) + result2 <- fetchStrategy.selectPartitionsToFetch(streams) + result3 <- fetchStrategy.selectPartitionsToFetch(streams) + result4 <- fetchStrategy.selectPartitionsToFetch(streams) + result5 <- fetchStrategy.selectPartitionsToFetch(streams) + results = Chunk(result1, result2, result3, result4, result5) + } yield assertTrue( + // Only partitions from topic 2 are selected (since 40 <= 50): + results.forall(_.forall(_.topic() == "topic2")), + // 2 partitions are selected every time (since 2*40 <= 80): + results.forall(_.size == 2), + // All partitions from topic 2 are selected eventually: + results.flatten.toSet == Set(tp20, tp21, tp22) + ) + }, + test("different streams may resume every time") { + val streams = Chunk( + newStream(tp10, currentQueueSize = 25), + newStream(tp11, currentQueueSize = 25), + newStream(tp20, currentQueueSize = 25), + newStream(tp21, currentQueueSize = 25), + newStream(tp22, currentQueueSize = 25) + ) + for { + result1 <- fetchStrategy.selectPartitionsToFetch(streams) + result2 <- fetchStrategy.selectPartitionsToFetch(streams) + result3 <- fetchStrategy.selectPartitionsToFetch(streams) + result4 <- fetchStrategy.selectPartitionsToFetch(streams) + result5 <- fetchStrategy.selectPartitionsToFetch(streams) + results = Chunk(result1, result2, result3, result4, result5) + } yield assertTrue( + // All partitions are selected eventually (since 25 <= 50): + results.flatten.toSet.size == 5, + // 3 partitions are selected every time (since 3*25 <= 80): + results.forall(_.size == 3), + // In at least 3 different combinations: + results.combinations(2).count { + case Chunk(resultA, resultB) => resultA != resultB + case _ => false // can not happen + } >= 3 + ) + } + ) + + private def newStream(topicPartition: TopicPartition, currentQueueSize: Int): PartitionStream = + new PartitionStream { + override def tp: TopicPartition = topicPartition + override def queueSize: UIO[Int] = ZIO.succeed(currentQueueSize) + } +} diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/FetchStrategy.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/FetchStrategy.scala index 1b0f98d82..e8098fac0 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/FetchStrategy.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/FetchStrategy.scala @@ -32,7 +32,7 @@ trait FetchStrategy { * The queue size at or below which more records are fetched and buffered (per partition). This buffer improves * throughput and supports varying downstream message processing time, while maintaining some backpressure. Large * values effectively disable backpressure at the cost of high memory usage, low values will effectively disable - * prefetching in favor of low memory consumption. The number of records that is fetched on every poll is controlled + * prefetching in favor of low memory consumption. The number of records that are fetched on every poll is controlled * by the `max.poll.records` setting, the number of records fetched for every partition is somewhere between 0 and * `max.poll.records`. * diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategy.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategy.scala new file mode 100644 index 000000000..2d2de3a2c --- /dev/null +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/fetch/ManyPartitionsQueueSizeBasedFetchStrategy.scala @@ -0,0 +1,59 @@ +package zio.kafka.consumer.fetch + +import org.apache.kafka.common.TopicPartition +import zio.{ Chunk, ZIO } +import zio.kafka.consumer.internal.PartitionStream + +import scala.collection.mutable + +/** + * A fetch strategy that allows a stream to fetch data when its queue size is at or below `maxPartitionQueueSize`, as + * long as the total queue size is at or below `maxTotalQueueSize`. This strategy is suitable when + * [[QueueSizeBasedFetchStrategy]] requires too much heap space, particularly when a lot of partitions are being + * consumed. + * + * @param maxPartitionQueueSize + * Maximum number of records to be buffered per partition. This buffer improves throughput and supports varying + * downstream message processing time, while maintaining some backpressure. Low values effectively disable prefetching + * in favour of low memory consumption. Large values leave it up to `maxTotalQueueSize` parameter to backpressure only + * over the buffers of all partitions together. + * + * The number of records that are fetched on every poll is controlled by the `max.poll.records` setting, the number of + * records fetched for every partition is somewhere between 0 and `max.poll.records`. + * + * The default value for this parameter is 2 * the default `max.poll.records` of 500, rounded to the nearest power of 2. + * + * @param maxTotalQueueSize + * Maximum number of records to be buffered over all partitions together. This can be used to limit memory usage when + * consuming a large number of partitions. + * + * When multiple streams are eligible for pre-fetching (because their queue size is below `maxPartitionQueueSize`), but + * together they exceed `maxTotalQueueSize`, then every call a random set of eligible streams is selected that stays + * below `maxTotalQueueSize`. The randomization ensures fairness and prevents read-starvation for streams at the end of + * the list. + * + * The default value is 20 * the default for `maxPartitionQueueSize`, allowing approximately 20 partitions to do + * pre-fetching in each poll. + */ +final case class ManyPartitionsQueueSizeBasedFetchStrategy( + maxPartitionQueueSize: Int = 1024, + maxTotalQueueSize: Int = 20480 +) extends FetchStrategy { + override def selectPartitionsToFetch( + streams: Chunk[PartitionStream] + ): ZIO[Any, Nothing, Set[TopicPartition]] = + for { + random <- ZIO.random + shuffledStreams <- random.shuffle(streams) + tps <- ZIO + .foldLeft(shuffledStreams)((mutable.ArrayBuilder.make[TopicPartition], maxTotalQueueSize)) { + case (acc @ (partitions, queueBudget), stream) => + stream.queueSize.map { queueSize => + if (queueSize <= maxPartitionQueueSize && queueSize <= queueBudget) { + (partitions += stream.tp, queueBudget - queueSize) + } else acc + } + } + .map { case (tps, _) => tps.result().toSet } + } yield tps +} From fd40816423b1b0ae81ee5487ab69e6f7a3cd5c27 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Thu, 18 Jul 2024 10:42:50 +0200 Subject: [PATCH 04/42] Alternative producer implementation (#1285) Refactoring of the producer so that it handles errors per record. --- .../scala/zio/kafka/producer/Producer.scala | 92 ++++++++++--------- 1 file changed, 50 insertions(+), 42 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index 939f36d31..e4dc34496 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -8,9 +8,7 @@ import zio.kafka.serde.Serializer import zio.kafka.utils.SslHelper import zio.stream.{ ZPipeline, ZStream } -import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ -import scala.util.control.NonFatal trait Producer { @@ -457,53 +455,63 @@ private[producer] final class ProducerLive( override def metrics: Task[Map[MetricName, Metric]] = ZIO.attemptBlocking(p.metrics().asScala.toMap) /** - * Calls to send may block when updating metadata or when communication with the broker is (temporarily) lost, - * therefore this stream is run on the blocking thread pool. + * Currently sending has the following characteristics: + * - You can submit many chunks, they get buffered in the send queue. + * - A chunk only gets send after the previous chunk completes (completes means that the callbacks for each record + * was invoked). + * - The records in a chunk are send in one go, in order. Records for the same partition have a high chance that + * they land in the same batch (which is good for compression). + * - Record ordering is retained and guaranteed between chunks. + * - Record ordering is retained and guaranteed within a chunk (per partition) unless `retries` has been enabled + * (see https://kafka.apache.org/documentation/#producerconfigs_retries). */ val sendFromQueue: ZIO[Any, Nothing, Any] = - ZIO.blocking { - ZStream - .fromQueueWithShutdown(sendQueue) - .mapZIO { case (serializedRecords, done) => - sendChunk(serializedRecords) - .flatMap(done.succeed(_)) - } - .runDrain + ZIO.runtime[Any].flatMap { runtime => + // Calls to 'send' may block when updating metadata or when communication with the broker is (temporarily) lost, + // therefore this stream is run on the blocking thread pool. + ZIO.blocking { + ZStream + .fromQueueWithShutdown(sendQueue) + .mapZIO { case (serializedRecords, done) => + sendChunk(runtime, serializedRecords) + .flatMap(done.succeed(_)) + } + .runDrain + } } private def sendChunk( + runtime: Runtime[Any], serializedRecords: Chunk[ByteRecord] ): ZIO[Any, Nothing, Chunk[Either[Throwable, RecordMetadata]]] = - ZIO - .async[Any, Nothing, Chunk[Either[Throwable, RecordMetadata]]] { callback => - try { - val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex - val res: Array[Either[Throwable, RecordMetadata]] = - new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) - val count: AtomicLong = new AtomicLong - val length = serializedRecords.length - - while (it.hasNext) { - val (rec, idx): (ByteRecord, Int) = it.next() - - // Since we might be sending to multiple partitions, the callbacks - // are _not_ necessarily called in order. - val _ = p.send( - rec, - (metadata: RecordMetadata, err: Exception) => { - res(idx) = Either.cond(err == null, metadata, err) - - if (count.incrementAndGet == length) { - callback(ZIO.succeed(Chunk.fromArray(res))) - } - } - ) - } - } catch { - case NonFatal(e) => - callback(ZIO.succeed(Chunk.fill(serializedRecords.size)(Left(e)))) - } - } + for { + promises <- ZIO.foreach(serializedRecords)(sendRecord(runtime)) + results <- ZIO.foreach(promises)(_.await.either) + } yield results + + private def sendRecord( + runtime: Runtime[Any] + )(record: ByteRecord): ZIO[Any, Nothing, Promise[Throwable, RecordMetadata]] = { + def unsafeRun(f: => ZIO[Any, Nothing, Any]): Unit = { + val _ = Unsafe.unsafe(implicit u => runtime.unsafe.run(f)) + } + + for { + done <- Promise.make[Throwable, RecordMetadata] + _ <- ZIO + .attempt[Any] { + p.send( + record, + (metadata: RecordMetadata, err: Exception) => + unsafeRun { + if (err == null) done.succeed(metadata) + else done.fail(err) + } + ) + } + .catchAll(err => done.fail(err)) + } yield done + } private def serialize[R, K, V]( r: ProducerRecord[K, V], From 092de5365acd4c7758bd77e5884fe44d223eedae Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Wed, 24 Jul 2024 10:21:44 +0200 Subject: [PATCH 05/42] Prevent users from enabling auto commit (#1290) Zio-kafka applications always pre-fetch data so that user streams can process the data asynchronously. This is not compatible with auto commit. When auto commit is enabled, the consumer will automatically commit batches _before_ they are processed by the user streams. An unaware user might accidentally enable auto commit and lose data during rebalances. Solves #1289. --- .../kafka/consumer/ConsumerSettingsSpec.scala | 24 +++++++++++++++++++ .../scala/zio/kafka/consumer/Consumer.scala | 2 +- .../zio/kafka/consumer/ConsumerSettings.scala | 5 ++++ 3 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSettingsSpec.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSettingsSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSettingsSpec.scala new file mode 100644 index 000000000..eb8ee74a5 --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSettingsSpec.scala @@ -0,0 +1,24 @@ +package zio.kafka.consumer + +import zio._ +import zio.kafka.ZIOSpecDefaultSlf4j +import zio.test._ +import zio.test.Assertion._ + +object ConsumerSettingsSpec extends ZIOSpecDefaultSlf4j { + + override def spec: Spec[TestEnvironment with Scope, Throwable] = + suite("ConsumerSettingsSpec")( + test("accepts no auto.commit") { + ZIO.attempt(ConsumerSettings(List("host"))) *> assertCompletesZIO + }, + test("accepts disabled auto.commit") { + ZIO.attempt(ConsumerSettings(List("host")).withProperty("enable.auto.commit", "false")) *> assertCompletesZIO + }, + test("rejects auto.commit") { + val settings = ZIO.attempt(ConsumerSettings(List("host")).withProperty("enable.auto.commit", "true")).exit + assertZIO(settings)(failsWithA[IllegalArgumentException]) + } + ) + +} diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/Consumer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/Consumer.scala index c1b66da1d..776e403e8 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/Consumer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/Consumer.scala @@ -197,7 +197,7 @@ object Consumer { /** * Create a zio-kafka Consumer from an org.apache.kafka KafkaConsumer * - * You are responsible for creating and closing the KafkaConsumer + * You are responsible for creating and closing the KafkaConsumer. Make sure auto.commit is disabled. */ def fromJavaConsumer( javaConsumer: JConsumer[Array[Byte], Array[Byte]], diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/ConsumerSettings.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/ConsumerSettings.scala index 934454132..7e5fc01b5 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/ConsumerSettings.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/ConsumerSettings.scala @@ -33,6 +33,11 @@ final case class ConsumerSettings( runloopMetricsSchedule: Schedule[Any, Unit, Long] = Schedule.fixed(500.millis), authErrorRetrySchedule: Schedule[Any, Throwable, Any] = Schedule.recurs(5) && Schedule.spaced(500.millis) ) { + // Parse booleans in a way compatible with how Kafka does this in org.apache.kafka.common.config.ConfigDef.parseType: + require( + properties.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).forall(_.toString.trim.equalsIgnoreCase("false")), + "Because zio-kafka does pre-fetching, auto commit is not supported" + ) /** * Tunes the consumer for high throughput. From e4da64b6a398846484571d866af83d7ddf29f2d3 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Fri, 26 Jul 2024 09:15:32 +0200 Subject: [PATCH 06/42] Update scalafmt-core to 3.8.3 (#1291) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [org.scalameta:scalafmt-core](https://github.com/scalameta/scalafmt) from `3.8.2` to `3.8.3` 📜 [GitHub Release Notes](https://github.com/scalameta/scalafmt/releases/tag/v3.8.3) - [Version Diff](https://github.com/scalameta/scalafmt/compare/v3.8.2...v3.8.3) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "org.scalameta", artifactId = "scalafmt-core" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "org.scalameta", artifactId = "scalafmt-core" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- .scalafmt.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.scalafmt.conf b/.scalafmt.conf index 26e9f540a..fe088f70e 100644 --- a/.scalafmt.conf +++ b/.scalafmt.conf @@ -1,4 +1,4 @@ -version=3.8.2 +version=3.8.3 project.git = true maxColumn = 120 align { From 78fe019c97c487f409087585a6efed1099f60e28 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Tue, 13 Aug 2024 21:24:18 +0200 Subject: [PATCH 07/42] Update zio, zio-streams, zio-test, ... to 2.1.7 (#1295) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio](https://github.com/zio/zio) * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.6` to `2.1.7` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.7) - [Version Diff](https://github.com/zio/zio/compare/v2.1.6...v2.1.7) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
🔍 Files still referring to the old version number The following files still refer to the old version number (2.1.6). You might want to review and update them manually. ``` build.sbt ```
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, old-version-remains, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 37fee423e..f2a5c8841 100644 --- a/build.sbt +++ b/build.sbt @@ -179,13 +179,13 @@ lazy val zioKafkaExample = .settings(run / fork := false) .settings( libraryDependencies ++= Seq( - "dev.zio" %% "zio" % "2.1.6", + "dev.zio" %% "zio" % "2.1.7", "dev.zio" %% "zio-kafka" % "2.8.0", "dev.zio" %% "zio-logging-slf4j2" % "2.3.0", "io.github.embeddedkafka" %% "embedded-kafka" % embeddedKafkaVersion, logback, "dev.zio" %% "zio-kafka-testkit" % "2.8.0" % Test, - "dev.zio" %% "zio-test" % "2.1.6" % Test + "dev.zio" %% "zio-test" % "2.1.7" % Test ), // Scala 3 compiling fails with: // [error] Modules were resolved with conflicting cross-version suffixes in ProjectRef(uri("file:/home/runner/work/zio-kafka/zio-kafka/"), "zioKafkaExample"): From f18a00abbe8990d727aa1771617707e5def581d7 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Wed, 14 Aug 2024 08:32:59 +0200 Subject: [PATCH 08/42] Update zio-streams, zio-test-sbt to 2.1.7 (#1297) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.6` to `2.1.7` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.7) - [Version Diff](https://github.com/zio/zio/compare/v2.1.6...v2.1.7) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index f2a5c8841..d29a51f11 100644 --- a/build.sbt +++ b/build.sbt @@ -25,7 +25,7 @@ lazy val _scala3 = "3.3.3" inThisBuild( List( name := "ZIO Kafka", - zioVersion := "2.1.6", + zioVersion := "2.1.7", scalaVersion := _scala213, // zio-sbt defines these 'scala213' and 'scala3' settings, but we need to define them here to override the defaults and better control them scala213 := _scala213, From f2de848cfebc137b38959ff208d261112863e7f0 Mon Sep 17 00:00:00 2001 From: Yurii Talashko Date: Thu, 15 Aug 2024 21:04:07 +0300 Subject: [PATCH 09/42] Fix assign followed by revoke during same rebalance (#1294) When an assign and revoke for the same partition follow each other very quickly, in the same poll, right now we disregard the revoke and start a stream for the partition this is a problem because the partition is no longer assigned. With this change the partition will be ignored. The same change is made for an assign followed by a 'lost'. --- .../kafka/consumer/internal/RunloopSpec.scala | 51 +++++++++++++++++-- .../zio/kafka/consumer/internal/Runloop.scala | 7 ++- 2 files changed, 52 insertions(+), 6 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopSpec.scala index 94213604f..126ed41fa 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopSpec.scala @@ -5,7 +5,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{ AuthenticationException, AuthorizationException } import zio._ import zio.kafka.consumer.{ ConsumerSettings, Subscription } -import zio.kafka.consumer.diagnostics.Diagnostics +import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment import zio.metrics.{ MetricState, Metrics } import zio.stream.{ Take, ZStream } @@ -20,6 +20,7 @@ object RunloopSpec extends ZIOSpecDefault { private type PartitionsHub = Hub[Take[Throwable, PartitionAssignment]] private val tp10 = new TopicPartition("t1", 0) + private val tp11 = new TopicPartition("t1", 1) private val key123 = "123".getBytes private val consumerSettings = ConsumerSettings(List("bootstrap")) @@ -27,7 +28,7 @@ object RunloopSpec extends ZIOSpecDefault { override def spec: Spec[TestEnvironment with Scope, Any] = suite("RunloopSpec")( test("runloop creates a new partition stream and polls for new records") { - withRunloop { (mockConsumer, partitionsHub, runloop) => + withRunloop() { (mockConsumer, partitionsHub, runloop) => mockConsumer.schedulePollTask { () => mockConsumer.updateEndOffsets(Map(tp10 -> Long.box(0L)).asJava) mockConsumer.rebalance(Seq(tp10).asJava) @@ -52,8 +53,48 @@ object RunloopSpec extends ZIOSpecDefault { ) } }, + test( + "runloop does not starts a new stream for partition which being revoked right after assignment within the same RebalanceEvent" + ) { + Diagnostics.SlidingQueue.make(100).flatMap { diagnostics => + withRunloop(diagnostics) { (mockConsumer, partitionsHub, runloop) => + mockConsumer.schedulePollTask { () => + mockConsumer.updateEndOffsets(Map(tp10 -> Long.box(0L), tp11 -> Long.box(0L)).asJava) + mockConsumer.rebalance(Seq(tp10, tp11).asJava) + mockConsumer.rebalance(Seq(tp10).asJava) + mockConsumer.addRecord(makeConsumerRecord(tp10, key123)) + } + for { + streamStream <- ZStream.fromHubScoped(partitionsHub) + _ <- runloop.addSubscription(Subscription.Topics(Set(tp10, tp11).map(_.topic()))) + _ <- streamStream + .map(_.exit) + .flattenExitOption + .flattenChunks + .take(1) + .mapZIO { case (_, stream) => + stream.runHead + } + .runDrain + diagnosticEvents <- diagnostics.queue.takeAll + rebalanceEvents = + diagnosticEvents.collect { case rebalanceEvent: DiagnosticEvent.Rebalance => + rebalanceEvent + } + } yield assertTrue( + rebalanceEvents.length == 1, + rebalanceEvents.head == DiagnosticEvent.Rebalance( + revoked = Set(tp11), + assigned = Set(tp10), + lost = Set.empty, + ended = Set.empty + ) + ) + } + } + }, test("runloop retries poll upon AuthorizationException and AuthenticationException") { - withRunloop { (mockConsumer, partitionsHub, runloop) => + withRunloop() { (mockConsumer, partitionsHub, runloop) => mockConsumer.schedulePollTask { () => mockConsumer.updateEndOffsets(Map(tp10 -> Long.box(0L)).asJava) mockConsumer.rebalance(Seq(tp10).asJava) @@ -90,7 +131,7 @@ object RunloopSpec extends ZIOSpecDefault { } ) @@ withLiveClock - private def withRunloop( + private def withRunloop(diagnostics: Diagnostics = Diagnostics.NoOp)( f: (BinaryMockConsumer, PartitionsHub, Runloop) => ZIO[Scope, Throwable, TestResult] ): ZIO[Scope, Throwable, TestResult] = ZIO.scoped { @@ -105,7 +146,7 @@ object RunloopSpec extends ZIOSpecDefault { consumerSettings, 100.millis, 100.millis, - Diagnostics.NoOp, + diagnostics, consumerAccess, partitionsHub ) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index bec04862a..32d0b162c 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -825,12 +825,17 @@ object Runloop { ): RebalanceEvent = copy( wasInvoked = true, + assignedTps = assignedTps -- revoked, revokedTps = revokedTps ++ revoked, endedStreams = this.endedStreams ++ endedStreams ) def onLost(lost: Set[TopicPartition]): RebalanceEvent = - copy(wasInvoked = true, lostTps = lostTps ++ lost) + copy( + wasInvoked = true, + assignedTps = assignedTps -- lost, + lostTps = lostTps ++ lost + ) } private object RebalanceEvent { From 2f1affef11b3c6df4c9cd0d6ec07e843b0452add Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Fri, 16 Aug 2024 08:27:21 +0200 Subject: [PATCH 10/42] Update logback-classic to 1.5.7 (#1299) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [ch.qos.logback:logback-classic](https://github.com/qos-ch/logback) from `1.5.6` to `1.5.7` ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "ch.qos.logback", artifactId = "logback-classic" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "ch.qos.logback", artifactId = "logback-classic" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index d29a51f11..b77ba9101 100644 --- a/build.sbt +++ b/build.sbt @@ -15,7 +15,7 @@ lazy val kafkaVersion = "3.7.1" lazy val embeddedKafkaVersion = "3.7.0" // Should be the same as kafkaVersion, except for the patch part lazy val kafkaClients = "org.apache.kafka" % "kafka-clients" % kafkaVersion -lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.6" +lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.7" enablePlugins(ZioSbtEcosystemPlugin, ZioSbtCiPlugin) From 8614f3f21d87b45743118a4574d7172b47fa754b Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 17 Aug 2024 09:54:59 +0200 Subject: [PATCH 11/42] Update embedded-kafka to 3.7.1.1 (#1300) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [io.github.embeddedkafka:embedded-kafka](https://github.com/embeddedkafka/embedded-kafka) from `3.7.0` to `3.7.1.1` 📜 [GitHub Release Notes](https://github.com/embeddedkafka/embedded-kafka/releases/tag/v3.7.1.1) - [Version Diff](https://github.com/embeddedkafka/embedded-kafka/compare/v3.7.0...v3.7.1.1) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "io.github.embeddedkafka", artifactId = "embedded-kafka" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "io.github.embeddedkafka", artifactId = "embedded-kafka" } }] ```
labels: library-update, version-scheme:semver-spec, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index b77ba9101..671a71a5f 100644 --- a/build.sbt +++ b/build.sbt @@ -12,7 +12,7 @@ import MimaSettings.mimaSettings lazy val binCompatVersionToCompare = None // Some("2.8.0") lazy val kafkaVersion = "3.7.1" -lazy val embeddedKafkaVersion = "3.7.0" // Should be the same as kafkaVersion, except for the patch part +lazy val embeddedKafkaVersion = "3.7.1.1" // Should be the same as kafkaVersion, except for the patch part lazy val kafkaClients = "org.apache.kafka" % "kafka-clients" % kafkaVersion lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.7" From b842a18a452364b24b93e1418377efdec94049a0 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Tue, 20 Aug 2024 19:39:00 +0200 Subject: [PATCH 12/42] Update sbt-tpolecat to 0.5.2 (#1301) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [org.typelevel:sbt-tpolecat](https://github.com/typelevel/sbt-tpolecat) from `0.5.1` to `0.5.2` 📜 [GitHub Release Notes](https://github.com/typelevel/sbt-tpolecat/releases/tag/v0.5.2) - [Version Diff](https://github.com/typelevel/sbt-tpolecat/compare/v0.5.1...v0.5.2) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "org.typelevel", artifactId = "sbt-tpolecat" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "org.typelevel", artifactId = "sbt-tpolecat" } }] ```
labels: sbt-plugin-update, early-semver-minor, semver-spec-patch, version-scheme:early-semver, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 53aa8e900..0bf23d5ac 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -5,7 +5,7 @@ addSbtPlugin("dev.zio" % "zio-sbt-website" % zioSbtVersion) addSbtPlugin("dev.zio" % "zio-sbt-ci" % zioSbtVersion) addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.12.1") -addSbtPlugin("org.typelevel" % "sbt-tpolecat" % "0.5.1") +addSbtPlugin("org.typelevel" % "sbt-tpolecat" % "0.5.2") addSbtPlugin("com.github.sbt" % "sbt-native-packager" % "1.10.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "1.1.3") From 3393fbf4487bb3cc34962a304edf792dd832deee Mon Sep 17 00:00:00 2001 From: Yurii Talashko Date: Wed, 21 Aug 2024 19:44:17 +0300 Subject: [PATCH 13/42] Revert back to 2.8.0 sendFromQueue implementation in producer (#1304) During running services with the new version of library 2.8.1, I noticed huge increase in messages production time to kafka. Some quick rough tests shoving me around 40x-100x times increase in amount of time taken to `produceChunk` on even 1-10 records chunks comparing to version 2.8.0 (or also to version 2.8.1 with the changes in this MR). Please, note, it is not a proper benchmarks. Also, to note, Im using Scala version 3.3.1. This MR just reverts two MRs updating `ProducersendFromQueue` implementation: - https://github.com/zio/zio-kafka/pull/1272 - https://github.com/zio/zio-kafka/pull/1285 Not sure if it's possible to revert two MRs at a time (with a single one for revert), so created this one. I haven't researched yet which exact change/changes are causing such performance degradation. I would suggest the next steps: - confirm the problem exists - reverting to the previous implementation (the one from 2.8.0/this MR) - release fixed version (to allow users have a nicely working version) - investigate & fix problem from the https://github.com/zio/zio-kafka/pull/1272 and/or https://github.com/zio/zio-kafka/pull/1285 It is only suggestions on the approach, feel free to ignore them. Also, feel free to modify/ignore this MR and treat it as an issue. --- .../scala/zio/kafka/producer/Producer.scala | 105 +++++++++--------- .../producer/TransactionalProducer.scala | 5 +- 2 files changed, 53 insertions(+), 57 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index e4dc34496..645f5f7ae 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -8,7 +8,9 @@ import zio.kafka.serde.Serializer import zio.kafka.utils.SslHelper import zio.stream.{ ZPipeline, ZStream } +import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ +import scala.util.control.NonFatal trait Producer { @@ -218,12 +220,13 @@ object Producer { settings: ProducerSettings ): ZIO[Scope, Throwable, Producer] = for { + runtime <- ZIO.runtime[Any] sendQueue <- Queue.bounded[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])]( settings.sendBufferSize ) - producer = new ProducerLive(javaProducer, sendQueue) - _ <- producer.sendFromQueue.forkScoped + producer = new ProducerLive(javaProducer, runtime, sendQueue) + _ <- ZIO.blocking(producer.sendFromQueue).forkScoped } yield producer /** @@ -359,6 +362,7 @@ object Producer { private[producer] final class ProducerLive( private[producer] val p: JProducer[Array[Byte], Array[Byte]], + runtime: Runtime[Any], sendQueue: Queue[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])] ) extends Producer { @@ -455,63 +459,52 @@ private[producer] final class ProducerLive( override def metrics: Task[Map[MetricName, Metric]] = ZIO.attemptBlocking(p.metrics().asScala.toMap) /** - * Currently sending has the following characteristics: - * - You can submit many chunks, they get buffered in the send queue. - * - A chunk only gets send after the previous chunk completes (completes means that the callbacks for each record - * was invoked). - * - The records in a chunk are send in one go, in order. Records for the same partition have a high chance that - * they land in the same batch (which is good for compression). - * - Record ordering is retained and guaranteed between chunks. - * - Record ordering is retained and guaranteed within a chunk (per partition) unless `retries` has been enabled - * (see https://kafka.apache.org/documentation/#producerconfigs_retries). + * Calls to send may block when updating metadata or when communication with the broker is (temporarily) lost, + * therefore this stream is run on a the blocking thread pool */ val sendFromQueue: ZIO[Any, Nothing, Any] = - ZIO.runtime[Any].flatMap { runtime => - // Calls to 'send' may block when updating metadata or when communication with the broker is (temporarily) lost, - // therefore this stream is run on the blocking thread pool. - ZIO.blocking { - ZStream - .fromQueueWithShutdown(sendQueue) - .mapZIO { case (serializedRecords, done) => - sendChunk(runtime, serializedRecords) - .flatMap(done.succeed(_)) + ZStream + .fromQueueWithShutdown(sendQueue) + .mapZIO { case (serializedRecords, done) => + ZIO.succeed { + try { + val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex + val res: Array[Either[Throwable, RecordMetadata]] = + new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) + val count: AtomicLong = new AtomicLong + val length = serializedRecords.length + + while (it.hasNext) { + val (rec, idx): (ByteRecord, Int) = it.next() + + val _ = p.send( + rec, + (metadata: RecordMetadata, err: Exception) => + Unsafe.unsafe { implicit u => + exec { + if (err != null) res(idx) = Left(err) + else res(idx) = Right(metadata) + + if (count.incrementAndGet == length) { + exec { + runtime.unsafe.run(done.succeed(Chunk.fromArray(res))).getOrThrowFiberFailure() + } + } + } + } + ) + } + } catch { + case NonFatal(e) => + Unsafe.unsafe { implicit u => + exec { + runtime.unsafe.run(done.succeed(Chunk.fill(serializedRecords.size)(Left(e)))).getOrThrowFiberFailure() + } + } } - .runDrain + } } - } - - private def sendChunk( - runtime: Runtime[Any], - serializedRecords: Chunk[ByteRecord] - ): ZIO[Any, Nothing, Chunk[Either[Throwable, RecordMetadata]]] = - for { - promises <- ZIO.foreach(serializedRecords)(sendRecord(runtime)) - results <- ZIO.foreach(promises)(_.await.either) - } yield results - - private def sendRecord( - runtime: Runtime[Any] - )(record: ByteRecord): ZIO[Any, Nothing, Promise[Throwable, RecordMetadata]] = { - def unsafeRun(f: => ZIO[Any, Nothing, Any]): Unit = { - val _ = Unsafe.unsafe(implicit u => runtime.unsafe.run(f)) - } - - for { - done <- Promise.make[Throwable, RecordMetadata] - _ <- ZIO - .attempt[Any] { - p.send( - record, - (metadata: RecordMetadata, err: Exception) => - unsafeRun { - if (err == null) done.succeed(metadata) - else done.fail(err) - } - ) - } - .catchAll(err => done.fail(err)) - } yield done - } + .runDrain private def serialize[R, K, V]( r: ProducerRecord[K, V], @@ -523,4 +516,6 @@ private[producer] final class ProducerLive( value <- valueSerializer.serialize(r.topic, r.headers, r.value()) } yield new ProducerRecord(r.topic, r.partition(), r.timestamp(), key, value, r.headers) + /** Used to prevent warnings about not using the result of an expression. */ + @inline private def exec[A](f: => A): Unit = { val _ = f } } diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala index c27f90633..173f43100 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/TransactionalProducer.scala @@ -96,11 +96,12 @@ object TransactionalProducer { )(p => ZIO.attemptBlocking(p.close(settings.producerSettings.closeTimeout)).orDie) _ <- ZIO.attemptBlocking(rawProducer.initTransactions()) semaphore <- Semaphore.make(1) + runtime <- ZIO.runtime[Any] sendQueue <- Queue.bounded[(Chunk[ByteRecord], Promise[Nothing, Chunk[Either[Throwable, RecordMetadata]]])]( settings.producerSettings.sendBufferSize ) - live = new ProducerLive(rawProducer, sendQueue) - _ <- live.sendFromQueue.forkScoped + live = new ProducerLive(rawProducer, runtime, sendQueue) + _ <- ZIO.blocking(live.sendFromQueue).forkScoped } yield new LiveTransactionalProducer(live, semaphore) } From 194f5ed27d941c92004c0d2f58d8a1fcfac0cbd7 Mon Sep 17 00:00:00 2001 From: "zio-assistant[bot]" <130037499+zio-assistant[bot]@users.noreply.github.com> Date: Wed, 21 Aug 2024 16:46:20 +0000 Subject: [PATCH 14/42] Update README.md (#1305) Autogenerated changes after running the `sbt docs/generateReadme` command of the [zio-sbt-website](https://zio.dev/zio-sbt) plugin. I will automatically update the README.md file whenever there is new change for README.md, e.g. - After each release, I will update the version in the installation section. - After any changes to the "docs/index.md" file, I will update the README.md file accordingly. Co-authored-by: ZIO Assistant --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index a9c998814..0da69b579 100644 --- a/README.md +++ b/README.md @@ -19,8 +19,8 @@ Kafka has a mature Java client for producing and consuming events, but it has a In order to use this library, we need to add the following line in our `build.sbt` file: ```scala -libraryDependencies += "dev.zio" %% "zio-kafka" % "2.8.0" -libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "2.8.0" % Test +libraryDependencies += "dev.zio" %% "zio-kafka" % "2.8.1" +libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "2.8.1" % Test ``` Snapshots are available on Sonatype's snapshot repository https://oss.sonatype.org/content/repositories/snapshots. From c607e669b3936e6397e3bc1af2089449e7357783 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 24 Aug 2024 09:05:58 +0200 Subject: [PATCH 15/42] Update zio-logging-slf4j, ... to 2.3.1 (#1307) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio-logging-slf4j](https://github.com/zio/zio-logging) * [dev.zio:zio-logging-slf4j2](https://github.com/zio/zio-logging) from `2.3.0` to `2.3.1` 📜 [GitHub Release Notes](https://github.com/zio/zio-logging/releases/tag/v2.3.1) - [Version Diff](https://github.com/zio/zio-logging/compare/v2.3.0...v2.3.1) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
🔍 Files still referring to the old version number The following files still refer to the old version number (2.3.0). You might want to review and update them manually. ``` zio-kafka/src/main/scala/zio/kafka/admin/AdminClient.scala ```
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, old-version-remains, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 671a71a5f..7eb4e9416 100644 --- a/build.sbt +++ b/build.sbt @@ -157,7 +157,7 @@ lazy val zioKafkaTest = libraryDependencies ++= Seq( kafkaClients, logback % Test, - "dev.zio" %% "zio-logging-slf4j" % "2.3.0" % Test + "dev.zio" %% "zio-logging-slf4j" % "2.3.1" % Test ) ++ `embedded-kafka`.value ) @@ -181,7 +181,7 @@ lazy val zioKafkaExample = libraryDependencies ++= Seq( "dev.zio" %% "zio" % "2.1.7", "dev.zio" %% "zio-kafka" % "2.8.0", - "dev.zio" %% "zio-logging-slf4j2" % "2.3.0", + "dev.zio" %% "zio-logging-slf4j2" % "2.3.1", "io.github.embeddedkafka" %% "embedded-kafka" % embeddedKafkaVersion, logback, "dev.zio" %% "zio-kafka-testkit" % "2.8.0" % Test, From 6006cac2929d2f8a32e051c5971f0b3f89168ec4 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 24 Aug 2024 09:06:27 +0200 Subject: [PATCH 16/42] Update sbt-native-packager to 1.10.4 (#1293) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [com.github.sbt:sbt-native-packager](https://github.com/sbt/sbt-native-packager) from `1.10.0` to `1.10.4` 📜 [GitHub Release Notes](https://github.com/sbt/sbt-native-packager/releases/tag/v1.10.4) - [Changelog](https://github.com/sbt/sbt-native-packager/blob/master/CHANGELOG.md) - [Version Diff](https://github.com/sbt/sbt-native-packager/compare/v1.10.0...v1.10.4) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "com.github.sbt", artifactId = "sbt-native-packager" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "com.github.sbt", artifactId = "sbt-native-packager" } }] ```
labels: sbt-plugin-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> Co-authored-by: svroonland --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 0bf23d5ac..6fe2994e2 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,7 +6,7 @@ addSbtPlugin("dev.zio" % "zio-sbt-ci" % zioSbtVersion) addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.12.1") addSbtPlugin("org.typelevel" % "sbt-tpolecat" % "0.5.2") -addSbtPlugin("com.github.sbt" % "sbt-native-packager" % "1.10.0") +addSbtPlugin("com.github.sbt" % "sbt-native-packager" % "1.10.4") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "1.1.3") resolvers ++= Resolver.sonatypeOssRepos("public") From 194e06eef7633a9410d196eadc303c38f4efffcf Mon Sep 17 00:00:00 2001 From: "zio-assistant[bot]" <130037499+zio-assistant[bot]@users.noreply.github.com> Date: Sat, 24 Aug 2024 07:08:00 +0000 Subject: [PATCH 17/42] Update README.md (#1308) Autogenerated changes after running the `sbt docs/generateReadme` command of the [zio-sbt-website](https://zio.dev/zio-sbt) plugin. I will automatically update the README.md file whenever there is new change for README.md, e.g. - After each release, I will update the version in the installation section. - After any changes to the "docs/index.md" file, I will update the README.md file accordingly. Co-authored-by: ZIO Assistant --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0da69b579..cc8223623 100644 --- a/README.md +++ b/README.md @@ -19,8 +19,8 @@ Kafka has a mature Java client for producing and consuming events, but it has a In order to use this library, we need to add the following line in our `build.sbt` file: ```scala -libraryDependencies += "dev.zio" %% "zio-kafka" % "2.8.1" -libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "2.8.1" % Test +libraryDependencies += "dev.zio" %% "zio-kafka" % "2.8.2" +libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "2.8.2" % Test ``` Snapshots are available on Sonatype's snapshot repository https://oss.sonatype.org/content/repositories/snapshots. From cef9d383b938179a880652e363995df31949f9cb Mon Sep 17 00:00:00 2001 From: "zio-assistant[bot]" <130037499+zio-assistant[bot]@users.noreply.github.com> Date: Sat, 24 Aug 2024 07:08:45 +0000 Subject: [PATCH 18/42] Update README.md (#1309) Autogenerated changes after running the `sbt docs/generateReadme` command of the [zio-sbt-website](https://zio.dev/zio-sbt) plugin. I will automatically update the README.md file whenever there is new change for README.md, e.g. - After each release, I will update the version in the installation section. - After any changes to the "docs/index.md" file, I will update the README.md file accordingly. Co-authored-by: ZIO Assistant From 5674dbd0e3e87f40e696439bb47dbda64b1e3bf0 Mon Sep 17 00:00:00 2001 From: svroonland Date: Sun, 25 Aug 2024 09:12:33 +0200 Subject: [PATCH 19/42] Simple producer benchmark (#1310) It's not much, but it gives us something to compare against. Relates to #1304 --- .github/workflows/benchs.yml | 2 +- .../zio/kafka/bench/ProducerBenchmark.scala | 47 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) create mode 100644 zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala diff --git a/.github/workflows/benchs.yml b/.github/workflows/benchs.yml index 782f1d8a7..f6439ec0e 100644 --- a/.github/workflows/benchs.yml +++ b/.github/workflows/benchs.yml @@ -58,7 +58,7 @@ jobs: # - "-rf json": Format type for machine-readable results. JSON # - "-foe true": Should JMH fail immediately if any benchmark had experienced an unrecoverable error?. True # - "-to 60": 1 minute timeout per iteration - run: sbt "zioKafkaBench/Jmh/run -wi 5 -i 5 -r 1 -w 1 -t 1 -to 60 -rf json -foe true" + run: sbt "zioKafkaBench/Jmh/run -wi 5 -i 5 -r 1 -w 1 -t 1 -to 120 -rf json -foe true" - name: Download previous benchmark data uses: actions/cache@v3 diff --git a/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala new file mode 100644 index 000000000..e6a26fbc4 --- /dev/null +++ b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala @@ -0,0 +1,47 @@ +package zio.kafka.bench + +import io.github.embeddedkafka.EmbeddedKafka +import org.apache.kafka.clients.producer.ProducerRecord +import org.openjdk.jmh.annotations._ +import zio.kafka.producer.Producer +import zio.kafka.serde.Serde +import zio.kafka.testkit.Kafka +import zio.kafka.testkit.KafkaTestUtils.producer +import zio.{ Chunk, ZIO, ZLayer } + +import java.util.concurrent.TimeUnit + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +class ProducerBenchmark extends ZioBenchmark[Kafka with Producer] { + val topic1 = "topic1" + val nrPartitions = 6 + val nrMessages = 500 + val kvs: List[(String, String)] = List.tabulate(nrMessages)(i => (s"key$i", s"msg$i")) + val records: Chunk[ProducerRecord[String, String]] = Chunk.fromIterable(kvs.map { case (k, v) => + new ProducerRecord(topic1, k, v) + }) + + override protected def bootstrap: ZLayer[Any, Nothing, Kafka with Producer] = + ZLayer.make[Kafka with Producer](Kafka.embedded, producer).orDie + + override def initialize: ZIO[Kafka with Producer, Throwable, Any] = for { + _ <- ZIO.succeed(EmbeddedKafka.deleteTopics(List(topic1))).ignore + _ <- ZIO.succeed(EmbeddedKafka.createCustomTopic(topic1, partitions = nrPartitions)) + } yield () + + @Benchmark + @BenchmarkMode(Array(Mode.AverageTime)) + def produceChunk(): Any = runZIO { + for { + _ <- Producer.produceChunk(records, Serde.string, Serde.string).repeatN(100) + } yield () + } + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def produceSingleRecord(): Any = runZIO { + Producer.produce(topic1, "key", "value", Serde.string, Serde.string) + } +} From 35530e86c7e6b5db89fb781c14217c121e54c6ee Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sun, 25 Aug 2024 09:19:36 +0200 Subject: [PATCH 20/42] Update zio-kafka, zio-kafka-testkit to 2.8.2 (#1306) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio-kafka](https://github.com/zio/zio-kafka) * [dev.zio:zio-kafka-testkit](https://github.com/zio/zio-kafka) from `2.8.0` to `2.8.2` 📜 [GitHub Release Notes](https://github.com/zio/zio-kafka/releases/tag/v2.8.2) - [Version Diff](https://github.com/zio/zio-kafka/compare/v2.8.0...v2.8.2) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
🔍 Files still referring to the old version number The following files still refer to the old version number (2.8.0). You might want to review and update them manually. ``` build.sbt ```
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, old-version-remains, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 7eb4e9416..87044bedf 100644 --- a/build.sbt +++ b/build.sbt @@ -180,11 +180,11 @@ lazy val zioKafkaExample = .settings( libraryDependencies ++= Seq( "dev.zio" %% "zio" % "2.1.7", - "dev.zio" %% "zio-kafka" % "2.8.0", + "dev.zio" %% "zio-kafka" % "2.8.2", "dev.zio" %% "zio-logging-slf4j2" % "2.3.1", "io.github.embeddedkafka" %% "embedded-kafka" % embeddedKafkaVersion, logback, - "dev.zio" %% "zio-kafka-testkit" % "2.8.0" % Test, + "dev.zio" %% "zio-kafka-testkit" % "2.8.2" % Test, "dev.zio" %% "zio-test" % "2.1.7" % Test ), // Scala 3 compiling fails with: From 4983ef9ede40a19449191e54253adee013e9b6a5 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 25 Aug 2024 11:04:31 +0200 Subject: [PATCH 21/42] Update to Kafka 3.8.0 (#1313) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 📦 Updates [org.apache.kafka:kafka-clients](https://kafka.apache.org/) from 3.7.1 to 3.8.0 📦 Updates [io.github.embeddedkafka:embedded-kafka](https://github.com/embeddedkafka/embedded-kafka) from `3.7.1.1` to `3.8.0` 📜 [GitHub Release Notes](https://github.com/embeddedkafka/embedded-kafka/releases/tag/v3.8.0) - [Version Diff](https://github.com/embeddedkafka/embedded-kafka/compare/v3.7.1.1...v3.8.0) --------- Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 4 +-- .../main/scala/zio/kafka/testkit/Kafka.scala | 35 +++++++++---------- .../scala/zio/kafka/admin/AdminClient.scala | 10 ++++++ 3 files changed, 29 insertions(+), 20 deletions(-) diff --git a/build.sbt b/build.sbt index 87044bedf..209791294 100644 --- a/build.sbt +++ b/build.sbt @@ -11,8 +11,8 @@ import MimaSettings.mimaSettings */ lazy val binCompatVersionToCompare = None // Some("2.8.0") -lazy val kafkaVersion = "3.7.1" -lazy val embeddedKafkaVersion = "3.7.1.1" // Should be the same as kafkaVersion, except for the patch part +lazy val kafkaVersion = "3.8.0" +lazy val embeddedKafkaVersion = "3.8.0" // Should be the same as kafkaVersion, except for the patch part lazy val kafkaClients = "org.apache.kafka" % "kafka-clients" % kafkaVersion lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.7" diff --git a/zio-kafka-testkit/src/main/scala/zio/kafka/testkit/Kafka.scala b/zio-kafka-testkit/src/main/scala/zio/kafka/testkit/Kafka.scala index 0524c8804..77e025293 100644 --- a/zio-kafka-testkit/src/main/scala/zio/kafka/testkit/Kafka.scala +++ b/zio-kafka-testkit/src/main/scala/zio/kafka/testkit/Kafka.scala @@ -1,6 +1,5 @@ package zio.kafka.testkit -import _root_.kafka.server.KafkaConfig import io.github.embeddedkafka.{ EmbeddedK, EmbeddedKafka, EmbeddedKafkaConfig } import zio._ @@ -78,23 +77,23 @@ object Kafka { embeddedWithBrokerProps( ports => Map( - "group.min.session.timeout.ms" -> "500", - "group.initial.rebalance.delay.ms" -> "0", - "authorizer.class.name" -> "kafka.security.authorizer.AclAuthorizer", - "super.users" -> "User:ANONYMOUS", - "ssl.client.auth" -> "required", - "ssl.enabled.protocols" -> "TLSv1.2", - "ssl.truststore.type" -> "JKS", - "ssl.keystore.type" -> "JKS", - "ssl.truststore.location" -> KafkaTestUtils.trustStoreFile.getAbsolutePath, - "ssl.truststore.password" -> "123456", - "ssl.keystore.location" -> KafkaTestUtils.keyStoreFile.getAbsolutePath, - "ssl.keystore.password" -> "123456", - "ssl.key.password" -> "123456", - KafkaConfig.InterBrokerListenerNameProp -> "SSL", - KafkaConfig.ListenersProp -> s"SSL://localhost:${ports.kafkaPort}", - KafkaConfig.AdvertisedListenersProp -> s"SSL://localhost:${ports.kafkaPort}", - KafkaConfig.ZkConnectionTimeoutMsProp -> s"${30.second.toMillis}" + "group.min.session.timeout.ms" -> "500", + "group.initial.rebalance.delay.ms" -> "0", + "authorizer.class.name" -> "kafka.security.authorizer.AclAuthorizer", + "super.users" -> "User:ANONYMOUS", + "ssl.client.auth" -> "required", + "ssl.enabled.protocols" -> "TLSv1.2", + "ssl.truststore.type" -> "JKS", + "ssl.keystore.type" -> "JKS", + "ssl.truststore.location" -> KafkaTestUtils.trustStoreFile.getAbsolutePath, + "ssl.truststore.password" -> "123456", + "ssl.keystore.location" -> KafkaTestUtils.keyStoreFile.getAbsolutePath, + "ssl.keystore.password" -> "123456", + "ssl.key.password" -> "123456", + "inter.broker.listener.name" -> "SSL", + "listeners" -> s"SSL://localhost:${ports.kafkaPort}", + "advertised.listeners" -> s"SSL://localhost:${ports.kafkaPort}", + "zookeeper.connection.timeout.ms" -> s"${30.second.toMillis}" ), customBrokerProps ) diff --git a/zio-kafka/src/main/scala/zio/kafka/admin/AdminClient.scala b/zio-kafka/src/main/scala/zio/kafka/admin/AdminClient.scala index eb0caa125..1d408a0ef 100644 --- a/zio-kafka/src/main/scala/zio/kafka/admin/AdminClient.scala +++ b/zio-kafka/src/main/scala/zio/kafka/admin/AdminClient.scala @@ -1046,6 +1046,14 @@ object AdminClient { override def asJava: JConsumerGroupState = JConsumerGroupState.EMPTY } + case object Assigning extends ConsumerGroupState { + override def asJava: JConsumerGroupState = JConsumerGroupState.ASSIGNING + } + + case object Reconciling extends ConsumerGroupState { + override def asJava: JConsumerGroupState = JConsumerGroupState.RECONCILING + } + def apply(state: JConsumerGroupState): ConsumerGroupState = state match { case JConsumerGroupState.UNKNOWN => ConsumerGroupState.Unknown @@ -1054,6 +1062,8 @@ object AdminClient { case JConsumerGroupState.STABLE => ConsumerGroupState.Stable case JConsumerGroupState.DEAD => ConsumerGroupState.Dead case JConsumerGroupState.EMPTY => ConsumerGroupState.Empty + case JConsumerGroupState.ASSIGNING => ConsumerGroupState.Assigning + case JConsumerGroupState.RECONCILING => ConsumerGroupState.Reconciling } } From 55ea0747c8b7bcc092077a7d49c9b2ce13105187 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sun, 25 Aug 2024 11:32:44 +0200 Subject: [PATCH 22/42] Update zio, zio-streams, zio-test, ... to 2.1.8 (#1312) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio](https://github.com/zio/zio) * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.7` to `2.1.8` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.8) - [Version Diff](https://github.com/zio/zio/compare/v2.1.7...v2.1.8) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
🔍 Files still referring to the old version number The following files still refer to the old version number (2.1.7). You might want to review and update them manually. ``` build.sbt ```
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, old-version-remains, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> Co-authored-by: Erik van Oosten --- build.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 209791294..1995190e7 100644 --- a/build.sbt +++ b/build.sbt @@ -179,13 +179,13 @@ lazy val zioKafkaExample = .settings(run / fork := false) .settings( libraryDependencies ++= Seq( - "dev.zio" %% "zio" % "2.1.7", + "dev.zio" %% "zio" % "2.1.8", "dev.zio" %% "zio-kafka" % "2.8.2", "dev.zio" %% "zio-logging-slf4j2" % "2.3.1", "io.github.embeddedkafka" %% "embedded-kafka" % embeddedKafkaVersion, logback, "dev.zio" %% "zio-kafka-testkit" % "2.8.2" % Test, - "dev.zio" %% "zio-test" % "2.1.7" % Test + "dev.zio" %% "zio-test" % "2.1.8" % Test ), // Scala 3 compiling fails with: // [error] Modules were resolved with conflicting cross-version suffixes in ProjectRef(uri("file:/home/runner/work/zio-kafka/zio-kafka/"), "zioKafkaExample"): From 26ff52b5ae45e0881609bea6969a7815cdd38b41 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sun, 25 Aug 2024 12:22:25 +0200 Subject: [PATCH 23/42] Update sbt-mima-plugin to 1.1.4 (#1296) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [com.typesafe:sbt-mima-plugin](https://github.com/lightbend-labs/mima) from `1.1.3` to `1.1.4` 📜 [GitHub Release Notes](https://github.com/lightbend-labs/mima/releases/tag/1.1.4) - [Version Diff](https://github.com/lightbend-labs/mima/compare/1.1.3...1.1.4) - [Version Diff](https://github.com/lightbend-labs/mima/compare/release-1.1.3...release-1.1.4) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "com.typesafe", artifactId = "sbt-mima-plugin" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "com.typesafe", artifactId = "sbt-mima-plugin" } }] ```
labels: sbt-plugin-update, early-semver-patch, semver-spec-patch, version-scheme:early-semver, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> Co-authored-by: Erik van Oosten --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 6fe2994e2..8a944609a 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -7,6 +7,6 @@ addSbtPlugin("dev.zio" % "zio-sbt-ci" % zioSbtVersion) addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.12.1") addSbtPlugin("org.typelevel" % "sbt-tpolecat" % "0.5.2") addSbtPlugin("com.github.sbt" % "sbt-native-packager" % "1.10.4") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "1.1.3") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "1.1.4") resolvers ++= Resolver.sonatypeOssRepos("public") From 41d059d18b76b18875d4434004b614cf4ce514e8 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Mon, 26 Aug 2024 08:05:56 +0200 Subject: [PATCH 24/42] Update zio-streams, zio-test-sbt to 2.1.8 (#1316) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.7` to `2.1.8` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.8) - [Version Diff](https://github.com/zio/zio/compare/v2.1.7...v2.1.8) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 1995190e7..2e477cb6d 100644 --- a/build.sbt +++ b/build.sbt @@ -25,7 +25,7 @@ lazy val _scala3 = "3.3.3" inThisBuild( List( name := "ZIO Kafka", - zioVersion := "2.1.7", + zioVersion := "2.1.8", scalaVersion := _scala213, // zio-sbt defines these 'scala213' and 'scala3' settings, but we need to define them here to override the defaults and better control them scala213 := _scala213, From 0231c5d30f6c52b0c9e8beeafefebaf0da6147eb Mon Sep 17 00:00:00 2001 From: Yurii Talashko Date: Mon, 26 Aug 2024 09:07:12 +0300 Subject: [PATCH 25/42] Use AtomicInteger instead of AtomicLong inside Producer.sendFromQueue implementation (#1315) Minor change (alignment), in case we gonna keep current `Producer.sendFromQueue` implementation (https://github.com/zio/zio-kafka/pull/1311#discussion_r1730449835), otherwise, this PR can be closed. --- .../src/main/scala/zio/kafka/producer/Producer.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index 645f5f7ae..4dcc0a519 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -8,7 +8,7 @@ import zio.kafka.serde.Serializer import zio.kafka.utils.SslHelper import zio.stream.{ ZPipeline, ZStream } -import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal @@ -460,7 +460,7 @@ private[producer] final class ProducerLive( /** * Calls to send may block when updating metadata or when communication with the broker is (temporarily) lost, - * therefore this stream is run on a the blocking thread pool + * therefore this stream is run on the blocking thread pool */ val sendFromQueue: ZIO[Any, Nothing, Any] = ZStream @@ -471,8 +471,8 @@ private[producer] final class ProducerLive( val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex val res: Array[Either[Throwable, RecordMetadata]] = new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) - val count: AtomicLong = new AtomicLong - val length = serializedRecords.length + val count: AtomicInteger = new AtomicInteger + val length = serializedRecords.length while (it.hasNext) { val (rec, idx): (ByteRecord, Int) = it.next() From cbed94ce81961e0d5e1d4eeb4d2c84c17413ce72 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Mon, 26 Aug 2024 09:00:34 +0200 Subject: [PATCH 26/42] Produce sequentially and in parallel in benchmark (#1314) Tests `Producer` in sequential and parallel settings. Also, changes the number of iterations to reduce runtime. Because the results are now no longer comparable to the previous tests, the existing tests have been renamed. --- .../zio/kafka/bench/ProducerBenchmark.scala | 38 ++++++++++++++++--- 1 file changed, 32 insertions(+), 6 deletions(-) diff --git a/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala index e6a26fbc4..a6c66e278 100644 --- a/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala +++ b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ProducerBenchmark.scala @@ -7,6 +7,7 @@ import zio.kafka.producer.Producer import zio.kafka.serde.Serde import zio.kafka.testkit.Kafka import zio.kafka.testkit.KafkaTestUtils.producer +import zio.stream.ZStream import zio.{ Chunk, ZIO, ZLayer } import java.util.concurrent.TimeUnit @@ -32,16 +33,41 @@ class ProducerBenchmark extends ZioBenchmark[Kafka with Producer] { @Benchmark @BenchmarkMode(Array(Mode.AverageTime)) - def produceChunk(): Any = runZIO { - for { - _ <- Producer.produceChunk(records, Serde.string, Serde.string).repeatN(100) - } yield () + def produceChunkSeq(): Any = runZIO { + // Produce 30 chunks sequentially + Producer.produceChunk(records, Serde.string, Serde.string).repeatN(29) + } + + @Benchmark + @BenchmarkMode(Array(Mode.AverageTime)) + def produceChunkPar(): Any = runZIO { + // Produce 30 chunks of which 4 run in parallel + ZStream + .range(0, 30, 1) + .mapZIOParUnordered(4) { _ => + Producer.produceChunk(records, Serde.string, Serde.string) + } + .runDrain + } + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def produceSingleRecordSeq(): Any = runZIO { + // Produce 50 records sequentially + Producer.produce(topic1, "key", "value", Serde.string, Serde.string).repeatN(99) } @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - def produceSingleRecord(): Any = runZIO { - Producer.produce(topic1, "key", "value", Serde.string, Serde.string) + def produceSingleRecordPar(): Any = runZIO { + // Produce 100 records of which 4 run in parallel + ZStream + .range(0, 100, 1) + .mapZIOParUnordered(4) { _ => + Producer.produce(topic1, "key", "value", Serde.string, Serde.string) + } + .runDrain } } From 381b3a0947db4154fd8d722e716a307fde5fbfa7 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Fri, 30 Aug 2024 08:43:35 +0200 Subject: [PATCH 27/42] Update zio, zio-streams, zio-test, ... to 2.1.9 (#1317) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio](https://github.com/zio/zio) * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.8` to `2.1.9` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.9) - [Version Diff](https://github.com/zio/zio/compare/v2.1.8...v2.1.9) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
🔍 Files still referring to the old version number The following files still refer to the old version number (2.1.8). You might want to review and update them manually. ``` build.sbt ```
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, old-version-remains, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 2e477cb6d..c08cbaeec 100644 --- a/build.sbt +++ b/build.sbt @@ -179,13 +179,13 @@ lazy val zioKafkaExample = .settings(run / fork := false) .settings( libraryDependencies ++= Seq( - "dev.zio" %% "zio" % "2.1.8", + "dev.zio" %% "zio" % "2.1.9", "dev.zio" %% "zio-kafka" % "2.8.2", "dev.zio" %% "zio-logging-slf4j2" % "2.3.1", "io.github.embeddedkafka" %% "embedded-kafka" % embeddedKafkaVersion, logback, "dev.zio" %% "zio-kafka-testkit" % "2.8.2" % Test, - "dev.zio" %% "zio-test" % "2.1.8" % Test + "dev.zio" %% "zio-test" % "2.1.9" % Test ), // Scala 3 compiling fails with: // [error] Modules were resolved with conflicting cross-version suffixes in ProjectRef(uri("file:/home/runner/work/zio-kafka/zio-kafka/"), "zioKafkaExample"): From 1ecadba47d5c425247d03edb0f35216f7b6223e7 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 31 Aug 2024 09:21:04 +0200 Subject: [PATCH 28/42] Update zio-streams, zio-test-sbt to 2.1.9 (#1320) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates * [dev.zio:zio-streams](https://github.com/zio/zio) * [dev.zio:zio-test-sbt](https://github.com/zio/zio) from `2.1.8` to `2.1.9` 📜 [GitHub Release Notes](https://github.com/zio/zio/releases/tag/v2.1.9) - [Version Diff](https://github.com/zio/zio/compare/v2.1.8...v2.1.9) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "dev.zio" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "dev.zio" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index c08cbaeec..39746d4ab 100644 --- a/build.sbt +++ b/build.sbt @@ -25,7 +25,7 @@ lazy val _scala3 = "3.3.3" inThisBuild( List( name := "ZIO Kafka", - zioVersion := "2.1.8", + zioVersion := "2.1.9", scalaVersion := _scala213, // zio-sbt defines these 'scala213' and 'scala3' settings, but we need to define them here to override the defaults and better control them scala213 := _scala213, From 99e32003aa48aa686b97a31fdf0235b49fd930d7 Mon Sep 17 00:00:00 2001 From: Ashwin Bhaskar Date: Sat, 31 Aug 2024 12:54:41 +0530 Subject: [PATCH 29/42] Fixes #1318 Adds documentation about auto commit behaviour (#1319) --- docs/example-of-consuming-producing-and-committing-offsets.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/example-of-consuming-producing-and-committing-offsets.md b/docs/example-of-consuming-producing-and-committing-offsets.md index b1a2c4485..30f1f1073 100644 --- a/docs/example-of-consuming-producing-and-committing-offsets.md +++ b/docs/example-of-consuming-producing-and-committing-offsets.md @@ -3,7 +3,7 @@ id: example-of-consuming-producing-and-committing-offsets title: "Example of Consuming, Producing and Committing Offsets" --- -This example shows how to consume messages from topic `topic_a` and produce transformed messages to `topic_b`, after which consumer offsets are committed. Processing is done in chunks using `ZStreamChunk` for more efficiency. +This example shows how to consume messages from topic `topic_a` and produce transformed messages to `topic_b`, after which consumer offsets are committed. Processing is done in chunks using `ZStreamChunk` for more efficiency. Please note: ZIO consumer does not support automatic offset committing. As a result, it ignores the Kafka consumer setting `enable.auto.commit=true`. Developers should manually commit offsets using the provided commit methods, typically after processing messages or at appropriate points in their application logic. ```scala import zio.ZLayer From 03832dae9c415501bace3bbd54469183bcae0404 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 7 Sep 2024 12:35:55 +0200 Subject: [PATCH 30/42] Update logback-classic to 1.5.8 (#1322) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [ch.qos.logback:logback-classic](https://github.com/qos-ch/logback) from `1.5.7` to `1.5.8` ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "ch.qos.logback", artifactId = "logback-classic" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "ch.qos.logback", artifactId = "logback-classic" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 39746d4ab..5b4d699fe 100644 --- a/build.sbt +++ b/build.sbt @@ -15,7 +15,7 @@ lazy val kafkaVersion = "3.8.0" lazy val embeddedKafkaVersion = "3.8.0" // Should be the same as kafkaVersion, except for the patch part lazy val kafkaClients = "org.apache.kafka" % "kafka-clients" % kafkaVersion -lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.7" +lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.8" enablePlugins(ZioSbtEcosystemPlugin, ZioSbtCiPlugin) From 023f11ef52d97abb940866cc211e658199ba2952 Mon Sep 17 00:00:00 2001 From: Yurii Talashko Date: Wed, 11 Sep 2024 11:22:08 +0300 Subject: [PATCH 31/42] More precise batch publish errors (#1321) This increases the precision of the result of producer method `produceChunkAsyncWithFailures` for cases where something went wrong. There are two changes compared to before: 1. Each entry in the result of`produceChunkAsyncWithFailures` now accurately corresponds to each record in the input chunk. Previously, if sending fails directly (*) on any of the given records, the error would be used for _all_ records in the batch, ignoring the send-outcome of the other records. An advantage of this change is that if sending some records failed, but some other records were actually sent, you can now correctly see all of that in the method's response. 2. In addition, if sending fails directly for a record (*), we no longer attempt to send subsequent records from the input. The result contains a `PublishOmittedException` for each record that is not sent. When implementing retries, this change makes it easier to publish records in the original order. In addition, we introduce unit-level tests for the producer. (*) By 'sending' we mean offering a record to the underlying java Kafka producer. Sending can fail directly (when we call the method), or later on (from a callback). --- .../zio/kafka/producer/ProducerSpec.scala | 228 ++++++++++++++++++ .../scala/zio/kafka/producer/Producer.scala | 108 ++++++--- 2 files changed, 305 insertions(+), 31 deletions(-) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/producer/ProducerSpec.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/producer/ProducerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/producer/ProducerSpec.scala new file mode 100644 index 000000000..5fe6fa203 --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/producer/ProducerSpec.scala @@ -0,0 +1,228 @@ +package zio.kafka.producer + +import org.apache.kafka.clients.producer +import org.apache.kafka.clients.producer.{ MockProducer, ProducerRecord, RecordMetadata } +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.AuthenticationException +import org.apache.kafka.common.serialization.ByteArraySerializer +import zio._ +import zio.test.TestAspect.withLiveClock +import zio.test._ + +import java.util.concurrent.Future +import java.util.concurrent.atomic.AtomicBoolean + +object ProducerSpec extends ZIOSpecDefault { + + private object TestKeyValueSerializer extends ByteArraySerializer + + private class BinaryMockProducer(autoComplete: Boolean) + extends MockProducer[Array[Byte], Array[Byte]]( + autoComplete, + TestKeyValueSerializer, + TestKeyValueSerializer + ) { + + private val nextSendAllowed = new AtomicBoolean(autoComplete) + + override def send( + record: ProducerRecord[Array[Byte], Array[Byte]], + callback: producer.Callback + ): Future[RecordMetadata] = { + awaitSendAllowed() + val sendResult = super.send(record, callback) + nextSendAllowed.set(autoComplete) + + sendResult + } + + def allowNextSendAndAwaitSendCompletion(): Unit = { + allowNextSend() + awaitSendCompletion() + } + + def allowNextSend(): Unit = + nextSendAllowed.set(true) + + def awaitSendAllowed(): Unit = + awaitSendCondition(true) + + def awaitSendCompletion(): Unit = + awaitSendCondition(false) + + private def awaitSendCondition(expectedCondition: Boolean): Unit = { + var awaitingSendCondition = true + while (awaitingSendCondition) + awaitingSendCondition = expectedCondition != nextSendAllowed.get() + } + + } + + override def spec: Spec[TestEnvironment with Scope, Any] = + suite("Producer")( + suite("produceChunkAsyncWithFailures")( + test("successfully produces chunk of records") { + withProducer() { (_, producer) => + val recordsToSend = Chunk( + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord() + ) + for { + results <- producer.produceChunkAsyncWithFailures(recordsToSend).flatten + } yield assertTrue( + results.length == recordsToSend.length, + results.forall(_.isRight) + ) + } + }, + test("omits sending further records in chunk in case the first send call fails") { + withProducer() { (mockJavaProducer, producer) => + val recordsToSend = Chunk( + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord() + ) + val testAuthenticationExceptionMessage = "test authentication exception" + mockJavaProducer.sendException = new AuthenticationException(testAuthenticationExceptionMessage) + for { + results <- producer.produceChunkAsyncWithFailures(recordsToSend).flatten + } yield assertTrue( + results.length == recordsToSend.length, + results.head.isLeft, + results.head.left.forall(_.getMessage == testAuthenticationExceptionMessage), + results.tail.forall(_ == Left(Producer.PublishOmittedException)) + ) + } + }, + test("provides correct results in case last send call fails") { + withProducer(autoCompleteProducerRequests = false) { (mockJavaProducer, producer) => + val recordsToSend = Chunk( + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord() + ) + val testAuthenticationExceptionMessage = "test authentication exception" + val mockJavaProducerBehaviour = ZIO.succeed { + // Send calls behaviours + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.sendException = new AuthenticationException(testAuthenticationExceptionMessage) + mockJavaProducer.allowNextSend() + // Send callbacks behaviours + mockJavaProducer.completeNext() + mockJavaProducer.completeNext() + mockJavaProducer.completeNext() + mockJavaProducer.completeNext() + } + for { + _ <- mockJavaProducerBehaviour.forkScoped + results <- producer.produceChunkAsyncWithFailures(recordsToSend).flatten + } yield assertTrue( + results.length == recordsToSend.length, + results.init.forall(_.isRight), + results.last.isLeft, + results.last.left.forall(_.getMessage == testAuthenticationExceptionMessage) + ) + } + }, + test("omits sending further records in chunk and provides correct results in case middle send call fails") { + withProducer(autoCompleteProducerRequests = false) { (mockJavaProducer, producer) => + val recordsToSend = Chunk( + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord() + ) + val testAuthenticationExceptionMessage = "test authentication exception" + val mockJavaProducerBehaviour = ZIO.succeed { + // Send calls behaviours + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.sendException = new AuthenticationException(testAuthenticationExceptionMessage) + mockJavaProducer.allowNextSend() + // Send callbacks behaviours + mockJavaProducer.completeNext() + mockJavaProducer.completeNext() + } + for { + _ <- mockJavaProducerBehaviour.forkScoped + results <- producer.produceChunkAsyncWithFailures(recordsToSend).flatten + } yield assertTrue( + results.length == recordsToSend.length, + results(0).isRight, + results(1).isRight, + results(2).left.forall(_.getMessage == testAuthenticationExceptionMessage), + results(3) == Left(Producer.PublishOmittedException), + results(4) == Left(Producer.PublishOmittedException) + ) + } + }, + test( + "omits sending further records in chunk and provides correct results in case second publication to broker fails along with middle send call failure" + ) { + withProducer(autoCompleteProducerRequests = false) { (mockJavaProducer, producer) => + val recordsToSend = Chunk( + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord(), + makeProducerRecord() + ) + val testAuthenticationExceptionMessage = "test authentication exception" + val testKafkaExceptionMessage = "unexpected broker exception" + val mockJavaProducerBehaviour = ZIO.succeed { + // Send calls behaviours + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.allowNextSendAndAwaitSendCompletion() + mockJavaProducer.sendException = new AuthenticationException(testAuthenticationExceptionMessage) + mockJavaProducer.allowNextSend() + // Send callbacks behaviours + mockJavaProducer.completeNext() + mockJavaProducer.errorNext(new KafkaException(testKafkaExceptionMessage)) + } + for { + _ <- mockJavaProducerBehaviour.forkScoped + results <- producer.produceChunkAsyncWithFailures(recordsToSend).flatten + } yield assertTrue( + results.length == recordsToSend.length, + results(0).isRight, + results(1).isLeft, + results(1).left.forall(_.getMessage == testKafkaExceptionMessage), + results(2).left.forall(_.getMessage == testAuthenticationExceptionMessage), + results(3) == Left(Producer.PublishOmittedException), + results(4) == Left(Producer.PublishOmittedException) + ) + } + } + ) + ) @@ withLiveClock + + private def withProducer(autoCompleteProducerRequests: Boolean = true)( + producerTest: (BinaryMockProducer, Producer) => ZIO[Scope, Throwable, TestResult] + ): ZIO[Scope, Throwable, TestResult] = + ZIO.scoped { + val mockJavaProducer = new BinaryMockProducer(autoCompleteProducerRequests) + + Producer + .fromJavaProducer(mockJavaProducer, ProducerSettings()) + .flatMap(producerTest(mockJavaProducer, _)) + } + + private def makeProducerRecord( + topic: String = "testTopic", + key: String = "key", + value: String = "value" + ): ProducerRecord[Array[Byte], Array[Byte]] = + new ProducerRecord[Array[Byte], Array[Byte]](topic, key.getBytes, value.getBytes) + +} diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index 4dcc0a519..02717e529 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -8,9 +8,8 @@ import zio.kafka.serde.Serializer import zio.kafka.utils.SslHelper import zio.stream.{ ZPipeline, ZStream } -import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ -import scala.util.control.NonFatal +import scala.util.control.{ NoStackTrace, NonFatal } trait Producer { @@ -104,6 +103,9 @@ trait Producer { /** * Produces a chunk of records. See [[produceChunkAsync(records*]] for version that allows to avoid round-trip-time * penalty for each chunk. + * + * When publishing any of the records fails, the whole batch fails even though some records might have been published. + * Use [[produceChunkAsyncWithFailures]] to get results per record. */ def produceChunk( records: Chunk[ProducerRecord[Array[Byte], Array[Byte]]] @@ -112,6 +114,9 @@ trait Producer { /** * Produces a chunk of records. See [[produceChunkAsync(records*]] for version that allows to avoid round-trip-time * penalty for each chunk. + * + * When publishing any of the records fails, the whole batch fails even though some records might have been published. + * Use [[produceChunkAsyncWithFailures]] to get results per record. */ def produceChunk[R, K, V]( records: Chunk[ProducerRecord[K, V]], @@ -128,6 +133,9 @@ trait Producer { * It is possible that for chunks that exceed the producer's internal buffer size, the outer layer will also signal * the transmission of part of the chunk. Regardless, awaiting the inner layer guarantees the transmission of the * entire chunk. + * + * When publishing any of the records fails, the whole batch fails even though some records might have been published. + * Use [[produceChunkAsyncWithFailures]] to get results per record. */ def produceChunkAsync( records: Chunk[ProducerRecord[Array[Byte], Array[Byte]]] @@ -142,6 +150,9 @@ trait Producer { * It is possible that for chunks that exceed the producer's internal buffer size, the outer layer will also signal * the transmission of part of the chunk. Regardless, awaiting the inner layer guarantees the transmission of the * entire chunk. + * + * When publishing any of the records fails, the whole batch fails even though some records might have been published. + * Use [[produceChunkAsyncWithFailures]] to get results per record. */ def produceChunkAsync[R, K, V]( records: Chunk[ProducerRecord[K, V]], @@ -162,6 +173,9 @@ trait Producer { * This variant of `produceChunkAsync` more accurately reflects that individual records within the Chunk can fail to * publish, rather than the failure being at the level of the Chunk. * + * When attempt to send a record into buffer for publication fails, the following records in the chunk are not + * published. This is indicated with a [[Producer.PublishOmittedException]]. + * * This variant does not accept serializers as they may also fail independently of each record and this is not * reflected in the return type. */ @@ -187,6 +201,10 @@ trait Producer { } object Producer { + case object PublishOmittedException + extends RuntimeException("Publish omitted due to a publish error for a previous record in the chunk") + with NoStackTrace + val live: RLayer[ProducerSettings, Producer] = ZLayer.scoped { for { @@ -466,41 +484,69 @@ private[producer] final class ProducerLive( ZStream .fromQueueWithShutdown(sendQueue) .mapZIO { case (serializedRecords, done) => - ZIO.succeed { - try { - val it: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex - val res: Array[Either[Throwable, RecordMetadata]] = - new Array[Either[Throwable, RecordMetadata]](serializedRecords.length) - val count: AtomicInteger = new AtomicInteger - val length = serializedRecords.length - - while (it.hasNext) { - val (rec, idx): (ByteRecord, Int) = it.next() - - val _ = p.send( - rec, - (metadata: RecordMetadata, err: Exception) => - Unsafe.unsafe { implicit u => - exec { - if (err != null) res(idx) = Left(err) - else res(idx) = Right(metadata) - - if (count.incrementAndGet == length) { + ZIO.suspendSucceed { + val recordsLength = serializedRecords.length + val recordsIterator: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex + val sentResults: Array[Either[Throwable, RecordMetadata]] = + new Array[Either[Throwable, RecordMetadata]](recordsLength) + + Ref.make(0).map { sentRecordsCountRef => + @inline def safelyInsertSentResult(resultIndex: Int, sentResult: Either[Throwable, RecordMetadata]): Unit = + Unsafe.unsafe { implicit u => + exec { + runtime.unsafe.run( + sentRecordsCountRef.update { sentRecordsCount => + // Updating sentResults[resultIndex] here is safe, + // cause Ref.update starts with volatile variable read and ends with volatile variable write, + // which guarantees sentResults.update executed on the latest updated version of sentResults + // and currently updated version of sentResults + // will be visible to the next sentResults read or update called within Ref.update + sentResults.update(resultIndex, sentResult) + + val newSentRecordsCount = sentRecordsCount + 1 + if (newSentRecordsCount == recordsLength) { + val sentResultsChunk = Chunk.fromArray(sentResults) + exec { - runtime.unsafe.run(done.succeed(Chunk.fromArray(res))).getOrThrowFiberFailure() + runtime.unsafe.run(done.succeed(sentResultsChunk)) } } + + newSentRecordsCount } - } - ) - } - } catch { - case NonFatal(e) => - Unsafe.unsafe { implicit u => - exec { - runtime.unsafe.run(done.succeed(Chunk.fill(serializedRecords.size)(Left(e)))).getOrThrowFiberFailure() + ) } } + + var previousSendCallsSucceed = true + + recordsIterator.foreach { case (record: ByteRecord, recordIndex: Int) => + if (previousSendCallsSucceed) { + try { + val _ = p.send( + record, + (metadata: RecordMetadata, err: Exception) => + safelyInsertSentResult( + recordIndex, + if (err eq null) Right(metadata) else Left(err) + ) + ) + } catch { + case NonFatal(err) => + previousSendCallsSucceed = false + + safelyInsertSentResult( + recordIndex, + Left(err) + ) + } + } else { + safelyInsertSentResult( + recordIndex, + Left(Producer.PublishOmittedException) + ) + } + } } } } From 956c2f3eee0dfe06b02f4da82f9cbaaa23c956a7 Mon Sep 17 00:00:00 2001 From: Yurii Talashko Date: Tue, 17 Sep 2024 19:49:58 +0300 Subject: [PATCH 32/42] Optimize Producer sendFromQueue implementation (#1326) From java's producer send method doc: `Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or they will delay the sending of messages from other threads`. I believe we should try to keep callback implementation fast, but with recent changes we added to provide proper error responses, we make the callback slower. In general we are talking approximately about 1-3 microseconds (current master) instead of 0,1-0,2 microseconds (version 2.8.2) for single callback execution time on my laptop. I believe it was reasonable, but at the same time wanted to keep implementation fast, and provided the fastest version I come up with at that time. Now, I got with an idea of faster implementation (with keeping the same proper error semantics). With this implementation single callback execution takes approximately 0,05-0,2 microseconds (even a bit faster than version 2.8.2). --- .../scala/zio/kafka/producer/Producer.scala | 90 ++++++++----------- 1 file changed, 38 insertions(+), 52 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala index 02717e529..bc77c6469 100644 --- a/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/producer/Producer.scala @@ -8,6 +8,7 @@ import zio.kafka.serde.Serializer import zio.kafka.utils.SslHelper import zio.stream.{ ZPipeline, ZStream } +import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ import scala.util.control.{ NoStackTrace, NonFatal } @@ -484,68 +485,56 @@ private[producer] final class ProducerLive( ZStream .fromQueueWithShutdown(sendQueue) .mapZIO { case (serializedRecords, done) => - ZIO.suspendSucceed { + ZIO.succeed { val recordsLength = serializedRecords.length + val sentRecordsCounter = new AtomicInteger(0) val recordsIterator: Iterator[(ByteRecord, Int)] = serializedRecords.iterator.zipWithIndex val sentResults: Array[Either[Throwable, RecordMetadata]] = new Array[Either[Throwable, RecordMetadata]](recordsLength) - Ref.make(0).map { sentRecordsCountRef => - @inline def safelyInsertSentResult(resultIndex: Int, sentResult: Either[Throwable, RecordMetadata]): Unit = + @inline def insertSentResult(resultIndex: Int, sentResult: Either[Throwable, RecordMetadata]): Unit = { + // Updating sentResults[resultIndex] here is safe, + // because only a single update for every resultIndex of sentResults is performed + sentResults.update(resultIndex, sentResult) + + // Reading from sentRecordsCounter guarantees fully updated version of sentResults + // will be visible and used to complete done promise + if (sentRecordsCounter.incrementAndGet() == recordsLength) { + val sentResultsChunk = Chunk.fromArray(sentResults) + Unsafe.unsafe { implicit u => - exec { - runtime.unsafe.run( - sentRecordsCountRef.update { sentRecordsCount => - // Updating sentResults[resultIndex] here is safe, - // cause Ref.update starts with volatile variable read and ends with volatile variable write, - // which guarantees sentResults.update executed on the latest updated version of sentResults - // and currently updated version of sentResults - // will be visible to the next sentResults read or update called within Ref.update - sentResults.update(resultIndex, sentResult) - - val newSentRecordsCount = sentRecordsCount + 1 - if (newSentRecordsCount == recordsLength) { - val sentResultsChunk = Chunk.fromArray(sentResults) - - exec { - runtime.unsafe.run(done.succeed(sentResultsChunk)) - } - } - - newSentRecordsCount - } - ) - } + val _ = runtime.unsafe.run(done.succeed(sentResultsChunk)) } + } + } - var previousSendCallsSucceed = true - - recordsIterator.foreach { case (record: ByteRecord, recordIndex: Int) => - if (previousSendCallsSucceed) { - try { - val _ = p.send( - record, - (metadata: RecordMetadata, err: Exception) => - safelyInsertSentResult( - recordIndex, - if (err eq null) Right(metadata) else Left(err) - ) - ) - } catch { - case NonFatal(err) => - previousSendCallsSucceed = false + var previousSendCallsSucceed = true - safelyInsertSentResult( + recordsIterator.foreach { case (record: ByteRecord, recordIndex: Int) => + if (previousSendCallsSucceed) { + try { + val _ = p.send( + record, + (metadata: RecordMetadata, err: Exception) => + insertSentResult( recordIndex, - Left(err) + if (err eq null) Right(metadata) else Left(err) ) - } - } else { - safelyInsertSentResult( - recordIndex, - Left(Producer.PublishOmittedException) ) + } catch { + case NonFatal(err) => + previousSendCallsSucceed = false + + insertSentResult( + recordIndex, + Left(err) + ) } + } else { + insertSentResult( + recordIndex, + Left(Producer.PublishOmittedException) + ) } } } @@ -561,7 +550,4 @@ private[producer] final class ProducerLive( key <- keySerializer.serialize(r.topic, r.headers, r.key()) value <- valueSerializer.serialize(r.topic, r.headers, r.value()) } yield new ProducerRecord(r.topic, r.partition(), r.timestamp(), key, value, r.headers) - - /** Used to prevent warnings about not using the result of an expression. */ - @inline private def exec[A](f: => A): Unit = { val _ = f } } From 8c2d98cf6d73d0dea340bc497d0049f9659bfdbe Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:50:22 +0200 Subject: [PATCH 33/42] Update sbt to 1.10.2 (#1327) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [org.scala-sbt:sbt](https://github.com/sbt/sbt) from `1.10.1` to `1.10.2` ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "org.scala-sbt", artifactId = "sbt" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "org.scala-sbt", artifactId = "sbt" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, version-scheme:early-semver, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- project/build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index ee4c672cd..0b699c305 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.10.1 +sbt.version=1.10.2 From 26118a81d31c9834d24201236979766be40bd6e5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:50:57 +0200 Subject: [PATCH 34/42] Bump scala-steward-org/scala-steward-action from 2.65.0 to 2.69.0 (#1328) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [scala-steward-org/scala-steward-action](https://github.com/scala-steward-org/scala-steward-action) from 2.65.0 to 2.69.0.
Release notes

Sourced from scala-steward-org/scala-steward-action's releases.

v2.69.0

What's Changed

Other Changes

Full Changelog: https://github.com/scala-steward-org/scala-steward-action/compare/v2.68.0...v2.69.0

v2.68.0

What's Changed

Other Changes

Full Changelog: https://github.com/scala-steward-org/scala-steward-action/compare/v2.67.0...v2.68.0

v2.67.0

What's Changed

Other Changes

New Contributors

Full Changelog: https://github.com/scala-steward-org/scala-steward-action/compare/v2.66.0...v2.67.0

v2.66.0

What's Changed

Other Changes

New Contributors

Full Changelog: https://github.com/scala-steward-org/scala-steward-action/compare/v2.65.0...v2.66.0

Commits
  • 7d71624 Release v2.69.0
  • 7292c0b Merge pull request #635 from EnviousSwan/cancel-token-refresh-at-the-end
  • ba67ff8 Cancel GitHub token refresh at the end
  • 74a9a67 Merge pull request #633 from EnviousSwan/use-installation-octokit
  • e2d6f01 Refresh token every 50 minutes
  • d9619bf Merge pull request #631 from EnviousSwan/save-workspace-cache-when-failed
  • f84b59a Save workspace cache when run failed
  • 2763313 Bump actions/upload-artifact from 4.3.4 to 4.4.0 (#627)
  • 7ac1543 Bump @​types/node from 20.14.9 to 22.5.1 (#629)
  • 880a26d Bump ts-pattern from 5.2.0 to 5.3.1 (#628)
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=scala-steward-org/scala-steward-action&package-manager=github_actions&previous-version=2.65.0&new-version=2.69.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/scala-steward.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/scala-steward.yml b/.github/workflows/scala-steward.yml index b5475716e..bfa793e01 100644 --- a/.github/workflows/scala-steward.yml +++ b/.github/workflows/scala-steward.yml @@ -13,7 +13,7 @@ jobs: name: Scala Steward steps: - name: Scala Steward - uses: scala-steward-org/scala-steward-action@v2.65.0 + uses: scala-steward-org/scala-steward-action@v2.69.0 with: github-app-id: ${{ secrets.SCALA_STEWARD_GITHUB_APP_ID }} github-app-installation-id: ${{ secrets.SCALA_STEWARD_GITHUB_APP_INSTALLATION_ID }} From c22ed31c7e762f036b5a498b2ab6394920702d31 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Wed, 18 Sep 2024 11:12:54 +0200 Subject: [PATCH 35/42] Document scala 3 plus `zio-kafka-testkit` (#1329) This solves #1193. --- docs/index.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/index.md b/docs/index.md index 0dc59fdf2..1c8e97d3a 100644 --- a/docs/index.md +++ b/docs/index.md @@ -26,6 +26,12 @@ libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "@VERSION@" % Test Snapshots are available on Sonatype's snapshot repository https://oss.sonatype.org/content/repositories/snapshots. [Browse here](https://oss.sonatype.org/content/repositories/snapshots/dev/zio/zio-kafka_3/) to find available versions. +For `zio-kafka-testkit` together with Scala 3, you also need to add the following to your `build.sbt` file: + +```scala +excludeDependencies += "org.scala-lang.modules" % "scala-collection-compat_2.13" +``` + ## Example Let's write a simple Kafka producer and consumer using ZIO Kafka with ZIO Streams. Before everything, we need a running instance of Kafka. We can do that by saving the following docker-compose script in the `docker-compose.yml` file and run `docker-compose up`: From eafb9ad100930c5396a635e0492e4eccd755f2d6 Mon Sep 17 00:00:00 2001 From: "zio-assistant[bot]" <130037499+zio-assistant[bot]@users.noreply.github.com> Date: Wed, 18 Sep 2024 09:14:58 +0000 Subject: [PATCH 36/42] Update README.md (#1330) Autogenerated changes after running the `sbt docs/generateReadme` command of the [zio-sbt-website](https://zio.dev/zio-sbt) plugin. I will automatically update the README.md file whenever there is new change for README.md, e.g. - After each release, I will update the version in the installation section. - After any changes to the "docs/index.md" file, I will update the README.md file accordingly. Co-authored-by: ZIO Assistant --- README.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/README.md b/README.md index cc8223623..656234a7c 100644 --- a/README.md +++ b/README.md @@ -26,6 +26,12 @@ libraryDependencies += "dev.zio" %% "zio-kafka-testkit" % "2.8.2" % Test Snapshots are available on Sonatype's snapshot repository https://oss.sonatype.org/content/repositories/snapshots. [Browse here](https://oss.sonatype.org/content/repositories/snapshots/dev/zio/zio-kafka_3/) to find available versions. +For `zio-kafka-testkit` together with Scala 3, you also need to add the following to your `build.sbt` file: + +```scala +excludeDependencies += "org.scala-lang.modules" % "scala-collection-compat_2.13" +``` + ## Example Let's write a simple Kafka producer and consumer using ZIO Kafka with ZIO Streams. Before everything, we need a running instance of Kafka. We can do that by saving the following docker-compose script in the `docker-compose.yml` file and run `docker-compose up`: From b3440ed9cc9a411ce927c0372c7a7561c0a42acd Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Thu, 26 Sep 2024 08:53:58 +0200 Subject: [PATCH 37/42] Update scala-library to 2.13.15 (#1334) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [org.scala-lang:scala-library](https://github.com/scala/scala) from `2.13.14` to `2.13.15` 📜 [GitHub Release Notes](https://github.com/scala/scala/releases/tag/v2.13.15) - [Version Diff](https://github.com/scala/scala/compare/v2.13.14...v2.13.15) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "org.scala-lang", artifactId = "scala-library" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "org.scala-lang", artifactId = "scala-library" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 5b4d699fe..c553d6321 100644 --- a/build.sbt +++ b/build.sbt @@ -19,7 +19,7 @@ lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.8" enablePlugins(ZioSbtEcosystemPlugin, ZioSbtCiPlugin) -lazy val _scala213 = "2.13.14" +lazy val _scala213 = "2.13.15" lazy val _scala3 = "3.3.3" inThisBuild( From b6200b39ebe599f32f68cb46c302ab54fc0c7d1f Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 28 Sep 2024 10:56:35 +0200 Subject: [PATCH 38/42] Update sbt-scalafix to 0.13.0 (#1335) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [ch.epfl.scala:sbt-scalafix](https://github.com/scalacenter/sbt-scalafix) from `0.12.1` to `0.13.0` 📜 [GitHub Release Notes](https://github.com/scalacenter/sbt-scalafix/releases/tag/v0.13.0) - [Version Diff](https://github.com/scalacenter/sbt-scalafix/compare/v0.12.1...v0.13.0) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "ch.epfl.scala", artifactId = "sbt-scalafix" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "ch.epfl.scala", artifactId = "sbt-scalafix" } }] ```
labels: sbt-plugin-update, early-semver-major, semver-spec-minor, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 8a944609a..a2a45ec4b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,7 +4,7 @@ addSbtPlugin("dev.zio" % "zio-sbt-ecosystem" % zioSbtVersion) addSbtPlugin("dev.zio" % "zio-sbt-website" % zioSbtVersion) addSbtPlugin("dev.zio" % "zio-sbt-ci" % zioSbtVersion) -addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.12.1") +addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.13.0") addSbtPlugin("org.typelevel" % "sbt-tpolecat" % "0.5.2") addSbtPlugin("com.github.sbt" % "sbt-native-packager" % "1.10.4") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "1.1.4") From 3b0cce0000967d1fa02d86ce922d5e8ad0a455d1 Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Sat, 28 Sep 2024 12:23:53 +0200 Subject: [PATCH 39/42] Update scala3-library to 3.3.4 (#1336) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [org.scala-lang:scala3-library](https://github.com/scala/scala3) from `3.3.3` to `3.3.4` 📜 [GitHub Release Notes](https://github.com/scala/scala3/releases/tag/3.3.4) - [Version Diff](https://github.com/scala/scala3/compare/3.3.3...3.3.4) - [Version Diff](https://github.com/scala/scala3/compare/release-3.3.3...release-3.3.4) ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "org.scala-lang", artifactId = "scala3-library" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "org.scala-lang", artifactId = "scala3-library" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, version-scheme:semver-spec, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> Co-authored-by: Erik van Oosten --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index c553d6321..fd3423377 100644 --- a/build.sbt +++ b/build.sbt @@ -20,7 +20,7 @@ lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.8" enablePlugins(ZioSbtEcosystemPlugin, ZioSbtCiPlugin) lazy val _scala213 = "2.13.15" -lazy val _scala3 = "3.3.3" +lazy val _scala3 = "3.3.4" inThisBuild( List( From da2ec468d44a361aa508d66480114114d83ee516 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 7 Oct 2024 08:46:28 +0200 Subject: [PATCH 40/42] Bump scala-steward-org/scala-steward-action from 2.69.0 to 2.70.0 (#1338) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [scala-steward-org/scala-steward-action](https://github.com/scala-steward-org/scala-steward-action) from 2.69.0 to 2.70.0.
Release notes

Sourced from scala-steward-org/scala-steward-action's releases.

v2.70.0

What's Changed

Other Changes

Full Changelog: https://github.com/scala-steward-org/scala-steward-action/compare/v2.69.0...v2.70.0

Commits
  • 6244c04 Release v2.70.0
  • 147c4b1 Merge pull request #644 from yokra9/dev-fix-usage-1
  • 91a1e5d Fix README.md to install sbt with setup-sbt action
  • 4531590 Fix README.md to run on ubuntu-latest and install sbt
  • 41920c3 Bump actions/checkout from 4.1.7 to 4.2.0 (#643)
  • 0101bd8 Bump ts-pattern from 5.3.1 to 5.4.0 (#638)
  • c2e210f Bump typescript from 5.5.4 to 5.6.2 (#640)
  • b069c56 Bump sinon from 18.0.0 to 19.0.2 (#637)
  • f3d5c3a Bump @​vercel/ncc from 0.38.1 to 0.38.2 (#639)
  • a86a956 Bump @​types/node from 22.5.1 to 22.7.4 (#636)
  • See full diff in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=scala-steward-org/scala-steward-action&package-manager=github_actions&previous-version=2.69.0&new-version=2.70.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/scala-steward.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/scala-steward.yml b/.github/workflows/scala-steward.yml index bfa793e01..26dab1611 100644 --- a/.github/workflows/scala-steward.yml +++ b/.github/workflows/scala-steward.yml @@ -13,7 +13,7 @@ jobs: name: Scala Steward steps: - name: Scala Steward - uses: scala-steward-org/scala-steward-action@v2.69.0 + uses: scala-steward-org/scala-steward-action@v2.70.0 with: github-app-id: ${{ secrets.SCALA_STEWARD_GITHUB_APP_ID }} github-app-installation-id: ${{ secrets.SCALA_STEWARD_GITHUB_APP_INSTALLATION_ID }} From 9cc273a4e0b9d31a117547da203360ab0dfecfcf Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Tue, 8 Oct 2024 19:31:53 +0200 Subject: [PATCH 41/42] Document Serdes in more depth (#1337) Fixes #1331. --- docs/serialization-and-deserialization.md | 131 ++++++++++++++++++++-- 1 file changed, 123 insertions(+), 8 deletions(-) diff --git a/docs/serialization-and-deserialization.md b/docs/serialization-and-deserialization.md index e3d29308b..a789352d3 100644 --- a/docs/serialization-and-deserialization.md +++ b/docs/serialization-and-deserialization.md @@ -1,24 +1,63 @@ --- id: serialization-and-deserialization -title: "Serialization And Deserialization" +title: "Serialization and Deserialization" --- +Zio-kafka deserializes incoming data, and deserializes outgoing data (both keys and values) from byte arrays to any +other type and back. This works by providing a key and value `Deserializer` while constructing a `Consumer`, +and a key and value `Serializer` while constructing the `Producer`. + +A `Serde` combines a `Deserializer` and a `Serializer`. Common serdes are provided in the `Serdes` object, e.g. +`Serdes.byteArray`, `Serdes.string` and `Serdes.long`. A serde can be converted to other serdes, or you can create a +custom serde by implementing the `Serde` trait directly. + +This document contains: + +- Handling failures in a serde +- How to create a custom serde +- How to create and use a custom serde that wraps invalid data +- How to do deserialization in the consumer stream +- A warning about using `mapZIO` + +## Handling failures in a serde + +Ideally, a serde can not fail serializing and deserializing. This is for example the case with the provided +`Serdes.byteArray` and `Serdes.string`. This is not the case for any serde that needs to handle invalid input, +(for example `Serdes.long`), or a serde that needs to do a remote lookup. + +By default, a consumer stream will fail if it encounters a deserialization error in the serde. Unfortunately, the +resulting failure might not clearly indicate that the cause is in the serde. + +There are 2 solutions for improving this: + +- Wrap the result of the serde in a `Try` with the `Serde.asTry` method. +- Use `Serdes.byteArray`, put the deserialization code in the consumer stream, or do serialization before handing the + data to zio-kafka. This way you can handle failures any way you want. + +Both approaches are discussed further below. + ## Custom Data Type Serdes -Serializers and deserializers (serdes) for custom data types can be constructed from scratch or by converting existing serdes. For example, to create a serde for an `Instant`: +Serializers and deserializers for custom data types can be created from scratch, or by converting existing +serdes. For example, to create a serde for an `Instant` from a serde for a `Long`: ```scala import java.time.Instant import zio.kafka.serde._ -val instantSerde: Serde[Any, Instant] = Serde.long.inmap(java.time.Instant.ofEpochMilli)(_.toEpochMilli) +val instantSerde: Serde[Any, Instant] = + Serdes.long.inmap(java.time.Instant.ofEpochMilli)(_.toEpochMilli) ``` -## Handling deserialization failures +To handle missing data (an empty key or value), you can use the `Serde.asOption` transformer. For example: +`Serdes.string.asOption`. This results in a `None` if the key or value is empty, and in a `Some(string)` otherwise. -The default behavior for a consumer stream when encountering a deserialization failure is to fail the stream. In many cases you may want to handle this situation differently, e.g. by skipping the message that failed to deserialize or by executing an alternative effect. For this purpose, any `Deserializer[T]` for some type `T` can be easily converted into a `Deserializer[Try[T]]` where deserialization failures are converted to a `Failure` using the `asTry` method. +## Custom serdes that wraps invalid data -Below is an example of skipping messages that fail to deserialize. The offset is passed downstream to be committed. +Any `Deserializer[A]` for a given type `A` can be converted into a `Deserializer[Try[A]]` where deserialization +failures are converted to a `Failure` using the `asTry` method. (Method `asTry` is also available on `Serde`.) + +Below is an example of skipping records that fail to deserialize. The offset is passed downstream to be committed. ```scala import zio._, stream._ @@ -28,11 +67,13 @@ import scala.util.{Try, Success, Failure} val consumer = ZLayer.scoped(Consumer.make(consumerSettings)) +val keySerde = Serdes.string +val valueSerde = Serdes.string.asTry // <-- using `.asTry` val stream = Consumer - .plainStream(Subscription.topics("topic150"), Serde.string, Serde.string.asTry) + .plainStream(Subscription.topics("topic150"), keySerde, valueSerde) stream - .mapZIO { record => + .mapZIO { record => // ⚠️ see section about `mapZIO` below! val tryValue: Try[String] = record.record.value() val offset: Offset = record.offset @@ -50,3 +91,77 @@ stream .runDrain .provideSomeLayer(consumer) ``` + +## Deserialization in the consumer stream + +In this approach we provide zio-kafka with the `Serdes.byteArray` serde (which is a pass-through serde) and do the +deserialization in the consumer stream. The deserialization can be done with regular ZIO operators. + +This approach provides more freedom at the cost of having to write more code. It also allows for optimizations such as +operating on chunks of records (see next section), and more contextual failure handling. + +Here is an example: + +```scala +import zio._, stream._ +import zio.kafka.consumer._ + +val consumer = ZLayer.scoped(Consumer.make(consumerSettings)) + +val stream = Consumer + .plainStream(Subscription.topics("topic150"), Serde.byteArray, Serde.byteArray) + +def deserialize(value: Array[Byte]): ZIO[Any, Throwable, Message] = ??? + +stream + .mapZIO { record => // ⚠️ see section about `mapZIO` below! + val value: Array[Byte] = record.record.value() + val offset: Offset = record.offset + + deserialize(value) + // possible action to take if deserialization fails: + .recoverWith(_ => someEffect(value)) + .flatMap(processMessage) + .as(offset) + } + .aggregateAsync(Consumer.offsetBatches) + .mapZIO(_.commit) + .runDrain + .provideSomeLayer(consumer) +``` + +## A warning about `mapZIO` + +Be careful with using `mapZIO` as it breaks the chunking structure of the stream (or more precisely, the resulting +stream has chunks with a single element). Throughput can be considerably lower than with the chunking structure intact. + +If your application requirements allow all elements of a chunk to be processed in one go, then you can use one of these +techniques to preserve the chunking structure: + +### Use `chunksWith` + +Use `chunksWith` when you have a single processing step that needs to work on a chunk. + +```scala +def f(a: A): ZIO[R, E, B] + +stream // ZStream[R, E, A] + .chunksWith { stream => stream.mapZIO(f) } // ZStream[R, E, B] +``` + +### Expose chunking structure with `chunks` + +Use `chunks` when you have multiple processing steps that can all work on a chunk at a time. Since `chunks` exposes the +chunking structure explicitly, the program can no longer accidentally break the chunking structure (unless +`flattenChunks` is also used). + +```scala +def f(a: A): ZIO[R, E, B] +def g(b: B): ZIO[R, E, C] + +stream // ZStream[R, E, A] + .chunks // ZStream[R, E, Chunk[A]] + .mapZIO { chunk => ZIO.foreach(chunk)(f) } // ZStream[R, E, Chunk[B]] + .mapZIO { chunk => ZIO.foreach(chunk)(g) } // ZStream[R, E, Chunk[C]] + .flattenChunks // ZStream[R, E, C] +``` From a0f45cacc3b2f4513866f9e9c572ac7126adb47c Mon Sep 17 00:00:00 2001 From: "zio-scala-steward[bot]" <145262613+zio-scala-steward[bot]@users.noreply.github.com> Date: Wed, 9 Oct 2024 09:23:09 +0200 Subject: [PATCH 42/42] Update logback-classic to 1.5.9 (#1340) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## About this PR 📦 Updates [ch.qos.logback:logback-classic](https://github.com/qos-ch/logback) from `1.5.8` to `1.5.9` ## Usage ✅ **Please merge!** I'll automatically update this PR to resolve conflicts as long as you don't change it yourself. If you'd like to skip this version, you can just close this PR. If you have any feedback, just mention me in the comments below. Configure Scala Steward for your repository with a [`.scala-steward.conf`](https://github.com/scala-steward-org/scala-steward/blob/767fcfecbfd53c507152f6cf15c846176bae561d/docs/repo-specific-configuration.md) file. _Have a fantastic day writing Scala!_
⚙ Adjust future updates Add this to your `.scala-steward.conf` file to ignore future updates of this dependency: ``` updates.ignore = [ { groupId = "ch.qos.logback", artifactId = "logback-classic" } ] ``` Or, add this to slow down future updates of this dependency: ``` dependencyOverrides = [{ pullRequests = { frequency = "30 days" }, dependency = { groupId = "ch.qos.logback", artifactId = "logback-classic" } }] ```
labels: library-update, early-semver-patch, semver-spec-patch, commit-count:1 Co-authored-by: zio-scala-steward[bot] <145262613+zio-scala-steward[bot]@users.noreply.github.com> --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index fd3423377..913e044ef 100644 --- a/build.sbt +++ b/build.sbt @@ -15,7 +15,7 @@ lazy val kafkaVersion = "3.8.0" lazy val embeddedKafkaVersion = "3.8.0" // Should be the same as kafkaVersion, except for the patch part lazy val kafkaClients = "org.apache.kafka" % "kafka-clients" % kafkaVersion -lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.8" +lazy val logback = "ch.qos.logback" % "logback-classic" % "1.5.9" enablePlugins(ZioSbtEcosystemPlugin, ZioSbtCiPlugin)