From 0ad10cb137ea191c04e2c5b0dba25bed980ae42b Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 17 Jan 2021 12:43:58 +0100 Subject: [PATCH 01/61] Async commit completed with callback (monix#94) Add consumer polling (monix#94) AutoCommit polling logic (monix#94) Remove nested task loop Remove redundant atomic refs Add test for (monix#101) Apply changes to older versions --- .../main/resources/monix/kafka/default.conf | 1 + .../src/main/scala/monix/kafka/Commit.scala | 7 +- .../scala/monix/kafka/CommittableOffset.scala | 6 -- .../monix/kafka/KafkaConsumerConfig.scala | 9 ++- .../monix/kafka/KafkaConsumerObservable.scala | 43 ++++++++++-- .../KafkaConsumerObservableAutoCommit.scala | 6 ++ .../KafkaConsumerObservableManualCommit.scala | 53 +++++++++++---- .../kafka/MergeByCommitCallbackTest.scala | 4 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 63 ++++++++++++++++++ .../main/resources/monix/kafka/default.conf | 1 + .../src/main/scala/monix/kafka/Commit.scala | 7 +- .../scala/monix/kafka/CommittableOffset.scala | 6 -- .../monix/kafka/KafkaConsumerConfig.scala | 9 ++- .../monix/kafka/KafkaConsumerObservable.scala | 43 ++++++++++-- .../KafkaConsumerObservableAutoCommit.scala | 6 ++ .../KafkaConsumerObservableManualCommit.scala | 45 +++++++++++++ .../kafka/MergeByCommitCallbackTest.scala | 4 ++ .../monix/kafka/MonixKafkaTopicListTest.scala | 66 +++++++++++++++++++ .../main/resources/monix/kafka/default.conf | 1 + .../src/main/scala/monix/kafka/Commit.scala | 7 +- .../scala/monix/kafka/CommittableOffset.scala | 3 + .../monix/kafka/KafkaConsumerConfig.scala | 13 ++++ .../monix/kafka/KafkaConsumerObservable.scala | 47 +++++++++++-- .../KafkaConsumerObservableAutoCommit.scala | 6 ++ .../KafkaConsumerObservableManualCommit.scala | 45 +++++++++++++ .../scala/monix/kafka/MonixKafkaTest.scala | 59 +++++++++++++++++ .../main/resources/monix/kafka/default.conf | 1 + .../src/main/scala/monix/kafka/Commit.scala | 8 +-- .../scala/monix/kafka/CommittableOffset.scala | 9 +-- .../monix/kafka/CommittableOffsetBatch.scala | 13 ++-- .../monix/kafka/KafkaConsumerConfig.scala | 9 ++- .../monix/kafka/KafkaConsumerObservable.scala | 43 ++++++++++-- .../KafkaConsumerObservableAutoCommit.scala | 6 ++ .../KafkaConsumerObservableManualCommit.scala | 53 +++++++++++---- .../kafka/MergeByCommitCallbackTest.scala | 3 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 66 ++++++++++++++++++- 36 files changed, 665 insertions(+), 106 deletions(-) diff --git a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf index 182c8300..740abcd3 100644 --- a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf @@ -77,4 +77,5 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" + monix.observable.poll.interval.ms = 100 } \ No newline at end of file diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala index ba56a332..6009d5ab 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala @@ -24,16 +24,13 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback */ trait Commit { def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] - def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] - final def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = commitBatchAsync(batch, null) + def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] } private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit } } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala index 71b7da86..1c8831af 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -46,12 +46,6 @@ final class CommittableOffset private[kafka] ( * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) - - /** Asynchronously commits [[offset]] to Kafka. It is recommended - * to use batched commit with [[CommittableOffsetBatch]] class. - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = - commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) } object CommittableOffset { diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 82178bcc..a17afcda 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -198,6 +198,9 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * + * @param pollInterval is the `monix.observable.poll.interval.ms` setting. + * Specifies time between KafkaConsumer#poll call attempts. + * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -243,7 +246,8 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekOnStart: ObservableSeekOnStart, + observableSeekToEndOnStart: Boolean, + pollInterval: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -428,7 +432,8 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), + observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), + pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 04d1e409..149e8150 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,7 @@ package monix.kafka -import monix.eval.Task +import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -41,7 +41,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig protected def consumer: Task[Consumer[K, V]] - /** Creates a task that polls the source, then feeds the downstream + @volatile + protected var isAcked = true + + /** + * Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -67,8 +71,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - val onCancel = cancelTask(c) - runLoop(c, out).guarantee(onCancel) + pollConsumer(c).loopForever.start.flatMap { pollFiber => + val onCancel = cancelTask(c, pollFiber) + runLoop(c, out).guarantee(onCancel) + } } feedTask.runAsync(cb) } @@ -89,10 +95,12 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { /* Returns a `Task` that triggers the closing of the * Kafka Consumer connection. */ - private def cancelTask(consumer: Consumer[K, V]): Task[Unit] = { + private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { // Forced asynchronous boundary - val cancelTask = Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) + val cancelTask = pollFiber.cancel.flatMap { _ => + Task.evalAsync { + consumer.synchronized(blocking(consumer.close())) + } } // By applying memoization, we are turning this @@ -101,6 +109,27 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // at most once cancelTask.memoize } + + /* Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + */ + private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + Task + .sleep(config.pollInterval) + .flatMap { _ => + if (!isAcked) { + Task.evalAsync { + consumer.synchronized { + blocking(consumer.poll(0)) + } + } + } else { + Task.unit + } + } + } } object KafkaConsumerObservable { diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 6406c224..09e0e200 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -68,12 +68,16 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen + isAcked = false Observer.feed(out, next.asScala)(out.scheduler) } } catch { @@ -83,6 +87,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -106,6 +111,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 58d7302b..b8b48097 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,6 +16,8 @@ package monix.kafka +import java.util + import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -48,17 +50,40 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task { - blocking( - consumer.synchronized( - consumer.commitAsync( - batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava, - callback))) - } + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { + Task + .async0[Unit] { (s, cb) => + val asyncCb = Callback.forked(cb)(s) + s.executeAsync { () => + consumer.synchronized { + try { + val offsets = batch.map { + case (k, v) => k -> new OffsetAndMetadata(v) + }.asJava + consumer.commitAsync( + offsets, + new OffsetCommitCallback { + override def onComplete( + offsets: util.Map[TopicPartition, OffsetAndMetadata], + exception: Exception): Unit = { + if (exception != null) { + if (!asyncCb.tryOnError(exception)) + s.reportFailure(exception) + } else { + asyncCb.tryOnSuccess(()) + } + } + } + ) + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) + s.reportFailure(ex) + } + } + } + } + } } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = @@ -73,9 +98,12 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen val result = next.asScala.map { record => @@ -86,6 +114,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + isAcked = false Observer.feed(out, result)(out.scheduler) } } catch { @@ -95,6 +124,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -117,6 +147,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 6a03f5b4..f0107b63 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -18,9 +18,7 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit }) val committableOffsetsGen: Gen[CommittableOffset] = for { diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index d1810ffe..5c1c67a0 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -138,6 +138,38 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } + test("manual async commit consumer test when subscribed to topics list") { + withRunningKafka { + + val count = 10000 + val topicName = "monix-kafka-manual-commit-tests" + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .bufferTumbling(count) + .map { messages => + messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) + } + .mapEval { case (values, batch) => Task.shift *> batch.commitAsync().map(_ => values -> batch.offsets) } + .headL + + val ((result, offsets), _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + + val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + } + } + test("publish to closed producer when subscribed to topics list") { withRunningKafka { val producer = KafkaProducer[String, String](producerCfg, io) @@ -155,4 +187,35 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } + + test("slow batches processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + pollInterval = 100.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } } diff --git a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf index 7c9ea91b..3af5e1c3 100644 --- a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf @@ -79,4 +79,5 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" + monix.observable.poll.interval.ms = 100 } \ No newline at end of file diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala index ba56a332..6009d5ab 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala @@ -24,16 +24,13 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback */ trait Commit { def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] - def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] - final def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = commitBatchAsync(batch, null) + def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] } private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit } } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala index 71b7da86..1c8831af 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -46,12 +46,6 @@ final class CommittableOffset private[kafka] ( * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) - - /** Asynchronously commits [[offset]] to Kafka. It is recommended - * to use batched commit with [[CommittableOffsetBatch]] class. - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = - commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) } object CommittableOffset { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index e1229c12..21c5714b 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -202,6 +202,9 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * + * @param pollInterval is the `monix.observable.poll.interval.ms` setting. + * Specifies time between KafkaConsumer#poll call attempts. + * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -248,7 +251,8 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekOnStart: ObservableSeekOnStart, + observableSeekToEndOnStart: Boolean, + pollInterval: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -435,7 +439,8 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), + observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), + pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index bf457267..aee0c8ad 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,7 @@ package monix.kafka -import monix.eval.Task +import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -40,7 +40,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig protected def consumer: Task[Consumer[K, V]] - /** Creates a task that polls the source, then feeds the downstream + @volatile + protected var isAcked = true + + /** + * Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -66,8 +70,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - val onCancel = cancelTask(c) - runLoop(c, out).guarantee(onCancel) + pollConsumer(c).loopForever.start.flatMap { pollFiber => + val onCancel = cancelTask(c, pollFiber) + runLoop(c, out).guarantee(onCancel) + } } feedTask.runAsync(cb) } @@ -88,10 +94,12 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { /* Returns a `Task` that triggers the closing of the * Kafka Consumer connection. */ - private def cancelTask(consumer: Consumer[K, V]): Task[Unit] = { + private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { // Forced asynchronous boundary - val cancelTask = Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) + val cancelTask = pollFiber.cancel.flatMap { _ => + Task.evalAsync { + consumer.synchronized(blocking(consumer.close())) + } } // By applying memoization, we are turning this @@ -100,6 +108,27 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // at most once cancelTask.memoize } + + /* Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + */ + private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + Task + .sleep(config.pollInterval) + .flatMap { _ => + if (!isAcked) { + Task.evalAsync { + consumer.synchronized { + blocking(consumer.poll(0)) + } + } + } else { + Task.unit + } + } + } } object KafkaConsumerObservable { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 6406c224..09e0e200 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -68,12 +68,16 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen + isAcked = false Observer.feed(out, next.asScala)(out.scheduler) } } catch { @@ -83,6 +87,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -106,6 +111,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 58d7302b..049c33ec 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,6 +16,8 @@ package monix.kafka +import java.util + import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -48,6 +50,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) +<<<<<<< refs/remotes/monix/master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task { @@ -59,6 +62,42 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( }.asJava, callback))) } +======= + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { + Task + .async0[Unit] { (s, cb) => + val asyncCb = Callback.forked(cb)(s) + s.executeAsync { () => + consumer.synchronized { + try { + val offsets = batch.map { + case (k, v) => k -> new OffsetAndMetadata(v) + }.asJava + consumer.commitAsync( + offsets, + new OffsetCommitCallback { + override def onComplete( + offsets: util.Map[TopicPartition, OffsetAndMetadata], + exception: Exception): Unit = { + if (exception != null) { + if (!asyncCb.tryOnError(exception)) + s.reportFailure(exception) + } else { + asyncCb.tryOnSuccess(()) + } + } + } + ) + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) + s.reportFailure(ex) + } + } + } + } + } +>>>>>>> Apply changes to older versions } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = @@ -73,9 +112,12 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen val result = next.asScala.map { record => @@ -86,6 +128,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + isAcked = false Observer.feed(out, result)(out.scheduler) } } catch { @@ -95,6 +138,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -117,6 +161,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 6a03f5b4..97c6b603 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -18,9 +18,13 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit +<<<<<<< refs/remotes/monix/master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task.unit +======= + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit +>>>>>>> Apply changes to older versions }) val committableOffsetsGen: Gen[CommittableOffset] = for { diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 46006cce..940768c0 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -135,6 +135,38 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } + test("manual async commit consumer test when subscribed to topics list") { + withRunningKafka { + + val count = 10000 + val topicName = "monix-kafka-manual-commit-tests" + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .bufferTumbling(count) + .map { messages => + messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) + } + .mapEval { case (values, batch) => Task.shift *> batch.commitAsync().map(_ => values -> batch.offsets) } + .headL + + val ((result, offsets), _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + + val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + } + } + test("publish to closed producer when subscribed to topics list") { withRunningKafka { val producer = KafkaProducer[String, String](producerCfg, io) @@ -152,4 +184,38 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } + + test("slow batches processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + pollInterval = 100.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay { + println(s) + s + }) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } } diff --git a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf index b995b27d..4a6bddfe 100644 --- a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf @@ -72,4 +72,5 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" + monix.observable.poll.interval.ms = 100 } \ No newline at end of file diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala index ba56a332..2afdf073 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala @@ -24,16 +24,19 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback */ trait Commit { def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] - def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] - final def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = commitBatchAsync(batch, null) + def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] } private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit +<<<<<<< refs/remotes/monix/master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task.unit +======= + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit +>>>>>>> Apply changes to older versions } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala index 71b7da86..ceec7e38 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -46,12 +46,15 @@ final class CommittableOffset private[kafka] ( * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) +<<<<<<< refs/remotes/monix/master /** Asynchronously commits [[offset]] to Kafka. It is recommended * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(callback: OffsetCommitCallback): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) +======= +>>>>>>> Apply changes to older versions } object CommittableOffset { diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index d90b1473..58bdfc17 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -194,6 +194,9 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * + * @param pollInterval is the `monix.observable.poll.interval.ms` setting. + * Specifies time between KafkaConsumer#poll call attempts. + * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -237,7 +240,12 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, +<<<<<<< refs/remotes/monix/master observableSeekOnStart: ObservableSeekOnStart, +======= + observableSeekToEndOnStart: Boolean, + pollInterval: FiniteDuration, +>>>>>>> Apply changes to older versions properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -418,7 +426,12 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), +<<<<<<< refs/remotes/monix/master observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), +======= + observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), + pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, +>>>>>>> Apply changes to older versions properties = Map.empty ) } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index c9a44a0c..2d32a311 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,7 @@ package monix.kafka -import monix.eval.Task +import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -38,7 +38,15 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig protected def consumer: Task[Consumer[K, V]] +<<<<<<< refs/remotes/monix/master /** Creates a task that polls the source, then feeds the downstream +======= + @volatile + protected var isAcked = true + + /** + * Creates a task that polls the source, then feeds the downstream +>>>>>>> Apply changes to older versions * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -64,8 +72,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd() else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning() // A task to execute on both cancellation and normal termination - val onCancel = cancelTask(c) - runLoop(c, out).guarantee(onCancel) + pollConsumer(c).loopForever.start.flatMap { pollFiber => + val onCancel = cancelTask(c, pollFiber) + runLoop(c, out).guarantee(onCancel) + } } feedTask.runAsync(cb) } @@ -86,10 +96,16 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { /* Returns a `Task` that triggers the closing of the * Kafka Consumer connection. */ +<<<<<<< refs/remotes/monix/master private def cancelTask(consumer: Consumer[K, V]): Task[Unit] = { +======= + private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { +>>>>>>> Apply changes to older versions // Forced asynchronous boundary - val cancelTask = Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) + val cancelTask = pollFiber.cancel.flatMap { _ => + Task.evalAsync { + consumer.synchronized(blocking(consumer.close())) + } } // By applying memoization, we are turning this @@ -98,6 +114,27 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // at most once cancelTask.memoize } + + /* Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + */ + private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + Task + .sleep(config.pollInterval) + .flatMap { _ => + if (!isAcked) { + Task.evalAsync { + consumer.synchronized { + blocking(consumer.poll(0)) + } + } + } else { + Task.unit + } + } + } } object KafkaConsumerObservable { diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 6406c224..068e62b9 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -68,12 +68,16 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment().asScala.toArray if (cancelable.isCanceled) Stop else { + consumer.resume(assignment: _ *) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment: _ *) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen + isAcked = false Observer.feed(out, next.asScala)(out.scheduler) } } catch { @@ -83,6 +87,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -106,6 +111,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 58d7302b..56d74fc1 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,6 +16,8 @@ package monix.kafka +import java.util + import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -48,6 +50,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) +<<<<<<< refs/remotes/monix/master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task { @@ -59,6 +62,42 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( }.asJava, callback))) } +======= + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { + Task + .async0[Unit] { (s, cb) => + val asyncCb = Callback.forked(cb)(s) + s.executeAsync { () => + consumer.synchronized { + try { + val offsets = batch.map { + case (k, v) => k -> new OffsetAndMetadata(v) + }.asJava + consumer.commitAsync( + offsets, + new OffsetCommitCallback { + override def onComplete( + offsets: util.Map[TopicPartition, OffsetAndMetadata], + exception: Exception): Unit = { + if (exception != null) { + if (!asyncCb.tryOnError(exception)) + s.reportFailure(exception) + } else { + asyncCb.tryOnSuccess(()) + } + } + } + ) + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) + s.reportFailure(ex) + } + } + } + } + } +>>>>>>> Apply changes to older versions } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = @@ -73,9 +112,12 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment().asScala.toArray if (cancelable.isCanceled) Stop else { + consumer.resume(assignment: _ *) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment: _ *) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen val result = next.asScala.map { record => @@ -86,6 +128,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + isAcked = false Observer.feed(out, result)(out.scheduler) } } catch { @@ -95,6 +138,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -117,6 +161,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala index 604bd196..7cfba264 100644 --- a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala +++ b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala @@ -127,6 +127,36 @@ class MonixKafkaTest extends FunSuite { assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) } + test("manual async commit consumer test when subscribed to topics list") { + + val count = 10000 + val topicName = "monix-kafka-manual-commit-tests" + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .bufferTumbling(count) + .map { messages => + messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) + } + .mapEval { case (values, batch) => Task.shift *> batch.commitAsync().map(_ => values -> batch.offsets) } + .headL + + val ((result, offsets), _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + + val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + } + test("publish to closed producer when subscribed to topics list") { val producer = KafkaProducer[String, String](producerCfg, io) val sendTask = producer.send(topicName, "test-message") @@ -185,4 +215,33 @@ class MonixKafkaTest extends FunSuite { .headL Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) } + + test("slow batches processing doesn't cause rebalancing") { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + sessionTimeout = 200.millis, + pollInterval = 100.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } } diff --git a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf index 91aefa9c..e9f477aa 100644 --- a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf @@ -78,4 +78,5 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" + monix.observable.poll.interval.ms = 100 } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/Commit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/Commit.scala index ba56a332..b174ef76 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/Commit.scala @@ -18,22 +18,18 @@ package monix.kafka import monix.eval.Task import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.OffsetCommitCallback /** Callback for batched commit realized as closure in [[KafkaConsumerObservable]] context. */ trait Commit { def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] - def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] - final def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = commitBatchAsync(batch, null) + def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] } private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit } } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala index 71b7da86..deea3756 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Represents offset for specified topic and partition that can be @@ -44,14 +43,8 @@ final class CommittableOffset private[kafka] ( /** Asynchronously commits [[offset]] to Kafka. It is recommended * to use batched commit with [[CommittableOffsetBatch]] class. - */ + * */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) - - /** Asynchronously commits [[offset]] to Kafka. It is recommended - * to use batched commit with [[CommittableOffsetBatch]] class. - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = - commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) } object CommittableOffset { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala index 4bfee4ad..c32292cc 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Batch of Kafka offsets which can be committed together. @@ -41,15 +40,13 @@ final class CommittableOffsetBatch private[kafka] (val offsets: Map[TopicPartiti */ def commitSync(): Task[Unit] = commitCallback.commitBatchSync(offsets) - /** Asynchronously commits [[offsets]] to Kafka - */ + /** + * Asynchronously commits [[offsets]] to Kafka + * */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(offsets) - /** Asynchronously commits [[offsets]] to Kafka - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = commitCallback.commitBatchAsync(offsets) - - /** Adds new [[CommittableOffset]] to batch. Added offset replaces previous one specified + /** + * Adds new [[CommittableOffset]] to batch. Added offset replaces previous one specified * for same topic and partition. */ def updated(committableOffset: CommittableOffset): CommittableOffsetBatch = diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 8fd10421..e3f95d83 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -207,6 +207,9 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * + * @param pollInterval is the `monix.observable.poll.interval.ms` setting. + * Specifies time between KafkaConsumer#poll call attempts. + * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -254,7 +257,8 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekOnStart: ObservableSeekOnStart, + observableSeekToEndOnStart: Boolean, + pollInterval: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -443,7 +447,8 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), + observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), + pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, properties = Map.empty ) } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index a65eee76..51be2ebb 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,7 @@ package monix.kafka -import monix.eval.Task +import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -39,7 +39,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig protected def consumer: Task[Consumer[K, V]] - /** Creates a task that polls the source, then feeds the downstream + @volatile + protected var isAcked = true + + /** + * Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -65,8 +69,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - val onCancel = cancelTask(c) - runLoop(c, out).guarantee(onCancel) + pollConsumer(c).loopForever.start.flatMap { pollFiber => + val onCancel = cancelTask(c, pollFiber) + runLoop(c, out).guarantee(onCancel) + } } feedTask.runAsync(cb) } @@ -87,10 +93,12 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { /* Returns a `Task` that triggers the closing of the * Kafka Consumer connection. */ - private def cancelTask(consumer: Consumer[K, V]): Task[Unit] = { + private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { // Forced asynchronous boundary - val cancelTask = Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) + val cancelTask = pollFiber.cancel.flatMap { _ => + Task.evalAsync { + consumer.synchronized(blocking(consumer.close())) + } } // By applying memoization, we are turning this @@ -99,6 +107,27 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // at most once cancelTask.memoize } + + /* Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + */ + private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + Task + .sleep(config.pollInterval) + .flatMap { _ => + if (!isAcked) { + Task.evalAsync { + consumer.synchronized { + blocking(consumer.poll(0)) + } + } + } else { + Task.unit + } + } + } } object KafkaConsumerObservable { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 6406c224..09e0e200 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -68,12 +68,16 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen + isAcked = false Observer.feed(out, next.asScala)(out.scheduler) } } catch { @@ -83,6 +87,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -106,6 +111,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index cb3dc5cf..3890e7bf 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,6 +16,8 @@ package monix.kafka +import java.util + import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -48,17 +50,40 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task { - blocking( - consumer.synchronized( - consumer.commitAsync( - batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava, - callback))) - } + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { + Task + .async0[Unit] { (s, cb) => + val asyncCb = Callback.forked(cb)(s) + s.executeAsync { () => + consumer.synchronized { + try { + val offsets = batch.map { + case (k, v) => k -> new OffsetAndMetadata(v) + }.asJava + consumer.commitAsync( + offsets, + new OffsetCommitCallback { + override def onComplete( + offsets: util.Map[TopicPartition, OffsetAndMetadata], + exception: Exception): Unit = { + if (exception != null) { + if (!asyncCb.tryOnError(exception)) + s.reportFailure(exception) + } else { + asyncCb.tryOnSuccess(()) + } + } + } + ) + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) + s.reportFailure(ex) + } + } + } + } + } } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = @@ -73,9 +98,12 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => val ackFuture = try consumer.synchronized { + val assignment = consumer.assignment() if (cancelable.isCanceled) Stop else { + consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) + consumer.pause(assignment) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen val result = next.asScala.map { record => @@ -86,6 +114,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + isAcked = false Observer.feed(out, result)(out.scheduler) } } catch { @@ -95,6 +124,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( ackFuture.syncOnComplete { case Success(ack) => + isAcked = true // The `streamError` flag protects against contract violations // (i.e. onSuccess/onError should happen only once). // Not really required, but we don't want to depend on the @@ -117,6 +147,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } case Failure(ex) => + isAcked = true asyncCb.onError(ex) } } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 6a03f5b4..2d289309 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -18,8 +18,7 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit }) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 314c5a86..ee8cdc07 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -17,12 +17,13 @@ package monix.kafka +import cats.syntax.apply._ import monix.eval.Task import monix.execution.Scheduler.Implicits.global import monix.kafka.config.AutoOffsetReset import monix.reactive.Observable -import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.TopicPartition import org.scalatest.FunSuite import scala.collection.JavaConverters._ @@ -137,6 +138,38 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } + test("manual async commit consumer test when subscribed to topics list") { + withRunningKafka { + + val count = 10000 + val topicName = "monix-kafka-manual-commit-tests" + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .bufferTumbling(count) + .map { messages => + messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) + } + .mapEval { case (values, batch) => Task.shift *> batch.commitAsync().map(_ => values -> batch.offsets) } + .headL + + val ((result, offsets), _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + + val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + } + } + test("publish to closed producer when subscribed to topics list") { withRunningKafka { val producer = KafkaProducer[String, String](producerCfg, io) @@ -154,4 +187,35 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } + + test("slow batches processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + pollInterval = 100.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } } From d56095d58563085fd796f61a20493aa531660d02 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 17 Jan 2021 13:32:32 +0100 Subject: [PATCH 02/61] Tests passing --- .gitignore | 2 ++ .../KafkaConsumerObservableManualCommit.scala | 12 ------------ .../scala/monix/kafka/KafkaConsumerConfig.scala | 6 ------ .../KafkaConsumerObservableManualCommit.scala | 14 +------------- .../scala/monix/kafka/KafkaConsumerConfig.scala | 4 ++-- .../monix/kafka/KafkaConsumerObservable.scala | 4 ++-- 6 files changed, 7 insertions(+), 35 deletions(-) diff --git a/.gitignore b/.gitignore index 1310ac33..5b5bab1b 100644 --- a/.gitignore +++ b/.gitignore @@ -16,3 +16,5 @@ project/plugins/project/ .scala_dependencies .worksheet .idea + +.bsp/sbt.json diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 049c33ec..96df2a4b 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -50,19 +50,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) -<<<<<<< refs/remotes/monix/master - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task { - blocking( - consumer.synchronized( - consumer.commitAsync( - batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava, - callback))) - } -======= override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { Task .async0[Unit] { (s, cb) => diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 58bdfc17..8767367c 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -240,12 +240,9 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, -<<<<<<< refs/remotes/monix/master observableSeekOnStart: ObservableSeekOnStart, -======= observableSeekToEndOnStart: Boolean, pollInterval: FiniteDuration, ->>>>>>> Apply changes to older versions properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -426,12 +423,9 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), -<<<<<<< refs/remotes/monix/master observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), -======= observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, ->>>>>>> Apply changes to older versions properties = Map.empty ) } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 56d74fc1..b7e10fc8 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -50,19 +50,8 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) -<<<<<<< refs/remotes/monix/master - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task { - blocking( - consumer.synchronized( - consumer.commitAsync( - batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava, - callback))) - } -======= + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { Task .async0[Unit] { (s, cb) => @@ -97,7 +86,6 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } } } ->>>>>>> Apply changes to older versions } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index e3f95d83..b6c4f445 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -257,7 +257,7 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekToEndOnStart: Boolean, + observableSeekOnStart: ObservableSeekOnStart, pollInterval: FiniteDuration, properties: Map[String, String]) { @@ -447,7 +447,7 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), + observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, properties = Map.empty ) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 51be2ebb..ed5c9ed2 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -93,7 +93,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { /* Returns a `Task` that triggers the closing of the * Kafka Consumer connection. */ - private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { + private def cancelTask(consumer: Consumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { // Forced asynchronous boundary val cancelTask = pollFiber.cancel.flatMap { _ => Task.evalAsync { @@ -113,7 +113,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * This allows producer process commit calls and also keeps consumer alive even * with long batch processing. */ - private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + private def pollConsumer(consumer: Consumer[K, V]): Task[Unit] = { Task .sleep(config.pollInterval) .flatMap { _ => From bc18d910626becb8935b28e4fb59be1eb8b8a0fa Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 23 Jan 2021 11:26:02 +0100 Subject: [PATCH 03/61] Observable poll heartbeat rate --- .../main/resources/monix/kafka/default.conf | 3 +- .../monix/kafka/KafkaConsumerConfig.scala | 10 +- .../monix/kafka/KafkaConsumerObservable.scala | 86 ++++++++--------- .../KafkaConsumerObservableManualCommit.scala | 5 +- .../monix/kafka/KafkaProducerConfig.scala | 2 +- .../main/resources/monix/kafka/default.conf | 3 +- .../monix/kafka/KafkaConsumerConfig.scala | 10 +- .../monix/kafka/KafkaConsumerObservable.scala | 85 ++++++++--------- .../KafkaConsumerObservableManualCommit.scala | 6 +- .../monix/kafka/KafkaProducerConfig.scala | 2 +- .../kafka/MergeByCommitCallbackTest.scala | 6 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 13 +-- .../src/main/scala/monix/kafka/Commit.scala | 6 +- .../scala/monix/kafka/CommittableOffset.scala | 6 +- .../monix/kafka/KafkaConsumerConfig.scala | 8 +- .../monix/kafka/KafkaConsumerObservable.scala | 95 ++++++++----------- .../KafkaConsumerObservableAutoCommit.scala | 4 +- .../KafkaConsumerObservableManualCommit.scala | 9 +- .../monix/kafka/KafkaProducerConfig.scala | 2 +- .../main/resources/monix/kafka/default.conf | 3 +- .../scala/monix/kafka/CommittableOffset.scala | 2 +- .../monix/kafka/CommittableOffsetBatch.scala | 8 +- .../monix/kafka/KafkaConsumerConfig.scala | 8 +- .../monix/kafka/KafkaConsumerObservable.scala | 89 +++++++++-------- .../KafkaConsumerObservableManualCommit.scala | 5 +- .../monix/kafka/KafkaProducerConfig.scala | 2 +- .../kafka/MergeByCommitCallbackTest.scala | 2 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 3 +- 28 files changed, 229 insertions(+), 254 deletions(-) diff --git a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf index 740abcd3..dbbfebd3 100644 --- a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf @@ -77,5 +77,6 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - monix.observable.poll.interval.ms = 100 + # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. + monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index a17afcda..c987958f 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -246,8 +246,8 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekToEndOnStart: Boolean, - pollInterval: FiniteDuration, + observableSeekOnStart: ObservableSeekOnStart, + observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -291,7 +291,7 @@ final case class KafkaConsumerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() @@ -432,8 +432,8 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), - pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, + observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), + observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 149e8150..f5e762d8 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,6 +16,7 @@ package monix.kafka +import cats.effect.Resource import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} @@ -25,7 +26,6 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer} - import scala.jdk.CollectionConverters._ import scala.concurrent.blocking import scala.util.matching.Regex @@ -44,8 +44,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { @volatile protected var isAcked = true - /** - * Creates a task that polls the source, then feeds the downstream + /** Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -66,25 +65,35 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { private def feedTask(out: Subscriber[Out]): Task[Unit] = { Task.create { (scheduler, cb) => implicit val s = scheduler - val feedTask = consumer.flatMap { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollConsumer(c).loopForever.start.flatMap { pollFiber => - val onCancel = cancelTask(c, pollFiber) - runLoop(c, out).guarantee(onCancel) - } - } - feedTask.runAsync(cb) + val startConsuming = + Resource + .make(consumer) { c => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess + } + .use { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + // A task to execute on both cancellation and normal termination + heartbeat(c) + // If polling fails the error is reported to the subscriber and + // wait 1sec as a rule of thumb leaving enough time for the consumer + // to recover and reassign partitions + .onErrorHandleWith(ex => Task(out.onError(ex))) + .loopForever + .startAndForget + .flatMap(_ => runLoop(c, out)) + } + startConsuming.runAsync(cb) } } - /* Returns a task that continuously polls the `KafkaConsumer` for - * new messages and feeds the given subscriber. - * - * Creates an asynchronous boundary on every poll. - */ + /** Returns a task that continuously polls the `KafkaConsumer` for + * new messages and feeds the given subscriber. + * + * Creates an asynchronous boundary on every poll. + */ private def runLoop(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Unit] = { ackTask(consumer, out).flatMap { case Stop => Task.unit @@ -92,37 +101,22 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } - /* Returns a `Task` that triggers the closing of the - * Kafka Consumer connection. - */ - private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { - // Forced asynchronous boundary - val cancelTask = pollFiber.cancel.flatMap { _ => - Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) - } - } - - // By applying memoization, we are turning this - // into an idempotent action, such that we are - // guaranteed that consumer.close() happens - // at most once - cancelTask.memoize - } - - /* Returns task that constantly polls the `KafkaConsumer` in case subscriber - * is still processing last fed batch. - * This allows producer process commit calls and also keeps consumer alive even - * with long batch processing. - */ - private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + /** Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + * + * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] + */ + private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { Task - .sleep(config.pollInterval) + .sleep(config.observablePollHeartbeatRate) .flatMap { _ => if (!isAcked) { Task.evalAsync { - consumer.synchronized { - blocking(consumer.poll(0)) + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } } } else { diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index b8b48097..e5999fc5 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -50,6 +50,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { Task .async0[Unit] { (s, cb) => @@ -57,8 +58,8 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => consumer.synchronized { try { - val offsets = batch.map { - case (k, v) => k -> new OffsetAndMetadata(v) + val offsets = batch.map { case (k, v) => + k -> new OffsetAndMetadata(v) }.asJava consumer.commitAsync( offsets, diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala index 90656598..de704730 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala @@ -272,7 +272,7 @@ case class KafkaProducerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() diff --git a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf index 3af5e1c3..ea06b1b9 100644 --- a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf @@ -79,5 +79,6 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - monix.observable.poll.interval.ms = 100 + # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. + monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 21c5714b..12e71563 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -251,8 +251,8 @@ final case class KafkaConsumerConfig( retryBackoffTime: FiniteDuration, observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, - observableSeekToEndOnStart: Boolean, - pollInterval: FiniteDuration, + observableSeekOnStart: ObservableSeekOnStart, + observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -297,7 +297,7 @@ final case class KafkaConsumerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() @@ -439,8 +439,8 @@ object KafkaConsumerConfig { retryBackoffTime = config.getInt("retry.backoff.ms").millis, observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), - observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), - pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, + observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), + observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index aee0c8ad..f5e762d8 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,6 +16,7 @@ package monix.kafka +import cats.effect.Resource import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} @@ -43,8 +44,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { @volatile protected var isAcked = true - /** - * Creates a task that polls the source, then feeds the downstream + /** Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -65,25 +65,35 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { private def feedTask(out: Subscriber[Out]): Task[Unit] = { Task.create { (scheduler, cb) => implicit val s = scheduler - val feedTask = consumer.flatMap { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollConsumer(c).loopForever.start.flatMap { pollFiber => - val onCancel = cancelTask(c, pollFiber) - runLoop(c, out).guarantee(onCancel) - } - } - feedTask.runAsync(cb) + val startConsuming = + Resource + .make(consumer) { c => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess + } + .use { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + // A task to execute on both cancellation and normal termination + heartbeat(c) + // If polling fails the error is reported to the subscriber and + // wait 1sec as a rule of thumb leaving enough time for the consumer + // to recover and reassign partitions + .onErrorHandleWith(ex => Task(out.onError(ex))) + .loopForever + .startAndForget + .flatMap(_ => runLoop(c, out)) + } + startConsuming.runAsync(cb) } } - /* Returns a task that continuously polls the `KafkaConsumer` for - * new messages and feeds the given subscriber. - * - * Creates an asynchronous boundary on every poll. - */ + /** Returns a task that continuously polls the `KafkaConsumer` for + * new messages and feeds the given subscriber. + * + * Creates an asynchronous boundary on every poll. + */ private def runLoop(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Unit] = { ackTask(consumer, out).flatMap { case Stop => Task.unit @@ -91,37 +101,22 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } - /* Returns a `Task` that triggers the closing of the - * Kafka Consumer connection. - */ - private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { - // Forced asynchronous boundary - val cancelTask = pollFiber.cancel.flatMap { _ => - Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) - } - } - - // By applying memoization, we are turning this - // into an idempotent action, such that we are - // guaranteed that consumer.close() happens - // at most once - cancelTask.memoize - } - - /* Returns task that constantly polls the `KafkaConsumer` in case subscriber - * is still processing last fed batch. - * This allows producer process commit calls and also keeps consumer alive even - * with long batch processing. - */ - private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + /** Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + * + * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] + */ + private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { Task - .sleep(config.pollInterval) + .sleep(config.observablePollHeartbeatRate) .flatMap { _ => if (!isAcked) { Task.evalAsync { - consumer.synchronized { - blocking(consumer.poll(0)) + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } } } else { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 96df2a4b..14c45d14 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -58,8 +58,8 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => consumer.synchronized { try { - val offsets = batch.map { - case (k, v) => k -> new OffsetAndMetadata(v) + val offsets = batch.map { case (k, v) => + k -> new OffsetAndMetadata(v) }.asJava consumer.commitAsync( offsets, @@ -85,7 +85,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } } } ->>>>>>> Apply changes to older versions + >>>>>>> Apply changes to older versions } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala index 3afea389..efdea9f9 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala @@ -272,7 +272,7 @@ case class KafkaProducerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 97c6b603..69381117 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -18,13 +18,13 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit -<<<<<<< refs/remotes/monix/master + <<<<<<< refs / remotes / monix / master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task.unit -======= + ======= override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit ->>>>>>> Apply changes to older versions + >>>>>>> Apply changes to older versions }) val committableOffsetsGen: Gen[CommittableOffset] = for { diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 940768c0..63d5a584 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -162,7 +162,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val ((result, offsets), _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) - val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + val properOffsets = Map(new TopicPartition(topicName, 0) -> count) assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) } } @@ -191,7 +191,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - pollInterval = 100.millis + observablePollHeartbeatRate = 100.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -207,10 +207,11 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .take(count) .map(_.value()) .bufferTumbling(count / 4) - .mapEval(s => Task.sleep(2.second) >> Task.delay { - println(s) - s - }) + .mapEval(s => + Task.sleep(2.second) >> Task.delay { + println(s) + s + }) .flatMap(Observable.fromIterable) .toListL diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala index 2afdf073..2f4db548 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala @@ -31,12 +31,12 @@ private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit -<<<<<<< refs/remotes/monix/master + <<<<<<< refs / remotes / monix / master override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = Task.unit -======= + ======= override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit ->>>>>>> Apply changes to older versions + >>>>>>> Apply changes to older versions } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala index ceec7e38..3914139a 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -46,15 +46,15 @@ final class CommittableOffset private[kafka] ( * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) -<<<<<<< refs/remotes/monix/master + <<<<<<< refs / remotes / monix / master /** Asynchronously commits [[offset]] to Kafka. It is recommended * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(callback: OffsetCommitCallback): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) -======= ->>>>>>> Apply changes to older versions + ======= + >>>>>>> Apply changes to older versions } object CommittableOffset { diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 8767367c..20905eca 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -241,8 +241,7 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - observableSeekToEndOnStart: Boolean, - pollInterval: FiniteDuration, + observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -284,7 +283,7 @@ final case class KafkaConsumerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() @@ -424,8 +423,7 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - observableSeekToEndOnStart = config.getBoolean("monix.observable.seekEnd.onStart"), - pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, + observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 2d32a311..094bc949 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,6 +16,7 @@ package monix.kafka +import cats.effect.Resource import monix.eval.{Fiber, Task} import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} @@ -38,15 +39,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig protected def consumer: Task[Consumer[K, V]] -<<<<<<< refs/remotes/monix/master - /** Creates a task that polls the source, then feeds the downstream -======= + /** Creates a task that polls the source, then feeds the downstream */ @volatile protected var isAcked = true - /** - * Creates a task that polls the source, then feeds the downstream ->>>>>>> Apply changes to older versions + /** Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -67,25 +64,35 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { private def feedTask(out: Subscriber[Out]): Task[Unit] = { Task.create { (scheduler, cb) => implicit val s = scheduler - val feedTask = consumer.flatMap { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd() - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning() - // A task to execute on both cancellation and normal termination - pollConsumer(c).loopForever.start.flatMap { pollFiber => - val onCancel = cancelTask(c, pollFiber) - runLoop(c, out).guarantee(onCancel) - } - } - feedTask.runAsync(cb) + val startConsuming = + Resource + .make(consumer) { c => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess + } + .use { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd() + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning() + // A task to execute on both cancellation and normal termination + heartbeat(c) + // If polling fails the error is reported to the subscriber and + // wait 1sec as a rule of thumb leaving enough time for the consumer + // to recover and reassign partitions + .onErrorHandleWith(ex => Task(out.onError(ex))) + .loopForever + .startAndForget + .flatMap(_ => runLoop(c, out)) + } + startConsuming.runAsync(cb) } } - /* Returns a task that continuously polls the `KafkaConsumer` for - * new messages and feeds the given subscriber. - * - * Creates an asynchronous boundary on every poll. - */ + /** Returns a task that continuously polls the `KafkaConsumer` for + * new messages and feeds the given subscriber. + * + * Creates an asynchronous boundary on every poll. + */ private def runLoop(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Unit] = { ackTask(consumer, out).flatMap { case Stop => Task.unit @@ -93,41 +100,22 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } - /* Returns a `Task` that triggers the closing of the - * Kafka Consumer connection. - */ -<<<<<<< refs/remotes/monix/master - private def cancelTask(consumer: Consumer[K, V]): Task[Unit] = { -======= - private def cancelTask(consumer: KafkaConsumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { ->>>>>>> Apply changes to older versions - // Forced asynchronous boundary - val cancelTask = pollFiber.cancel.flatMap { _ => - Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) - } - } - - // By applying memoization, we are turning this - // into an idempotent action, such that we are - // guaranteed that consumer.close() happens - // at most once - cancelTask.memoize - } - - /* Returns task that constantly polls the `KafkaConsumer` in case subscriber - * is still processing last fed batch. - * This allows producer process commit calls and also keeps consumer alive even - * with long batch processing. - */ - private def pollConsumer(consumer: KafkaConsumer[K, V]): Task[Unit] = { + /** Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + * + * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] + */ + private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { Task - .sleep(config.pollInterval) + .sleep(config.observablePollHeartbeatRate) .flatMap { _ => if (!isAcked) { Task.evalAsync { - consumer.synchronized { - blocking(consumer.poll(0)) + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } } } else { @@ -135,6 +123,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } } + } object KafkaConsumerObservable { diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 068e62b9..8828a083 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -71,9 +71,9 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( val assignment = consumer.assignment().asScala.toArray if (cancelable.isCanceled) Stop else { - consumer.resume(assignment: _ *) + consumer.resume(assignment: _*) val next = blocking(consumer.poll(pollTimeoutMillis)) - consumer.pause(assignment: _ *) + consumer.pause(assignment: _*) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index b7e10fc8..cfe0b6d7 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -51,7 +51,6 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( k -> new OffsetAndMetadata(v) }.asJava)))) - override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { Task .async0[Unit] { (s, cb) => @@ -59,8 +58,8 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => consumer.synchronized { try { - val offsets = batch.map { - case (k, v) => k -> new OffsetAndMetadata(v) + val offsets = batch.map { case (k, v) => + k -> new OffsetAndMetadata(v) }.asJava consumer.commitAsync( offsets, @@ -103,9 +102,9 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( val assignment = consumer.assignment().asScala.toArray if (cancelable.isCanceled) Stop else { - consumer.resume(assignment: _ *) + consumer.resume(assignment: _*) val next = blocking(consumer.poll(pollTimeoutMillis)) - consumer.pause(assignment: _ *) + consumer.pause(assignment: _*) // Feeding the observer happens on the Subscriber's scheduler // if any asynchronous boundaries happen val result = next.asScala.map { record => diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala index 2d221c12..9ecfe392 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala @@ -266,7 +266,7 @@ case class KafkaProducerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() diff --git a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf index e9f477aa..59c2d7b9 100644 --- a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf @@ -78,5 +78,6 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - monix.observable.poll.interval.ms = 100 + # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. + monix.observable.poll.heartbeat.rate.ms = 100 } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala index deea3756..9bf00254 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -43,7 +43,7 @@ final class CommittableOffset private[kafka] ( /** Asynchronously commits [[offset]] to Kafka. It is recommended * to use batched commit with [[CommittableOffsetBatch]] class. - * */ + */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala index c32292cc..7d7d2bad 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala @@ -40,13 +40,11 @@ final class CommittableOffsetBatch private[kafka] (val offsets: Map[TopicPartiti */ def commitSync(): Task[Unit] = commitCallback.commitBatchSync(offsets) - /** - * Asynchronously commits [[offsets]] to Kafka - * */ + /** Asynchronously commits [[offsets]] to Kafka + */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(offsets) - /** - * Adds new [[CommittableOffset]] to batch. Added offset replaces previous one specified + /** Adds new [[CommittableOffset]] to batch. Added offset replaces previous one specified * for same topic and partition. */ def updated(committableOffset: CommittableOffset): CommittableOffsetBatch = diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index b6c4f445..3b79e63f 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -207,7 +207,7 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param pollInterval is the `monix.observable.poll.interval.ms` setting. + * @param heartbeatInterval is the `monix.observable.poll.interval.ms` setting. * Specifies time between KafkaConsumer#poll call attempts. * * @param properties map of other properties that will be passed to @@ -258,7 +258,7 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - pollInterval: FiniteDuration, + observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -304,7 +304,7 @@ final case class KafkaConsumerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() @@ -448,7 +448,7 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - pollInterval = config.getInt("monix.observable.poll.interval.ms").millis, + observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index ed5c9ed2..475a01c2 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,8 @@ package monix.kafka -import monix.eval.{Fiber, Task} +import cats.effect.Resource +import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -37,13 +38,13 @@ import scala.util.matching.Regex */ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig + protected def consumer: Task[Consumer[K, V]] @volatile protected var isAcked = true - /** - * Creates a task that polls the source, then feeds the downstream + /** Creates a task that polls the source, then feeds the downstream * subscriber, returning the resulting acknowledgement */ protected def ackTask(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Ack] @@ -54,6 +55,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { val callback = new Callback[Throwable, Unit] { def onSuccess(value: Unit): Unit = out.onComplete() + def onError(ex: Throwable): Unit = out.onError(ex) } @@ -64,25 +66,35 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { private def feedTask(out: Subscriber[Out]): Task[Unit] = { Task.create { (scheduler, cb) => implicit val s = scheduler - val feedTask = consumer.flatMap { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollConsumer(c).loopForever.start.flatMap { pollFiber => - val onCancel = cancelTask(c, pollFiber) - runLoop(c, out).guarantee(onCancel) - } - } - feedTask.runAsync(cb) + val startConsuming = + Resource + .make(consumer) { c => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess + } + .use { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + // A task to execute on both cancellation and normal termination + heartbeat(c) + // If polling fails the error is reported to the subscriber and + // wait 1sec as a rule of thumb leaving enough time for the consumer + // to recover and reassign partitions + .onErrorHandleWith(ex => Task(out.onError(ex))) + .loopForever + .startAndForget + .flatMap(_ => runLoop(c, out)) + } + startConsuming.runAsync(cb) } } - /* Returns a task that continuously polls the `KafkaConsumer` for - * new messages and feeds the given subscriber. - * - * Creates an asynchronous boundary on every poll. - */ + /** Returns a task that continuously polls the `KafkaConsumer` for + * new messages and feeds the given subscriber. + * + * Creates an asynchronous boundary on every poll. + */ private def runLoop(consumer: Consumer[K, V], out: Subscriber[Out]): Task[Unit] = { ackTask(consumer, out).flatMap { case Stop => Task.unit @@ -90,37 +102,22 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } - /* Returns a `Task` that triggers the closing of the - * Kafka Consumer connection. - */ - private def cancelTask(consumer: Consumer[K, V], pollFiber: Fiber[Nothing]): Task[Unit] = { - // Forced asynchronous boundary - val cancelTask = pollFiber.cancel.flatMap { _ => - Task.evalAsync { - consumer.synchronized(blocking(consumer.close())) - } - } - - // By applying memoization, we are turning this - // into an idempotent action, such that we are - // guaranteed that consumer.close() happens - // at most once - cancelTask.memoize - } - - /* Returns task that constantly polls the `KafkaConsumer` in case subscriber - * is still processing last fed batch. - * This allows producer process commit calls and also keeps consumer alive even - * with long batch processing. - */ - private def pollConsumer(consumer: Consumer[K, V]): Task[Unit] = { + /** Returns task that constantly polls the `KafkaConsumer` in case subscriber + * is still processing last fed batch. + * This allows producer process commit calls and also keeps consumer alive even + * with long batch processing. + * + * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] + */ + private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { Task - .sleep(config.pollInterval) + .sleep(config.observablePollHeartbeatRate) .flatMap { _ => if (!isAcked) { Task.evalAsync { - consumer.synchronized { - blocking(consumer.poll(0)) + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } } } else { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 3890e7bf..2cb5634f 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -50,6 +50,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( Task(blocking(consumer.synchronized(consumer.commitSync(batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava)))) + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = { Task .async0[Unit] { (s, cb) => @@ -57,8 +58,8 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( s.executeAsync { () => consumer.synchronized { try { - val offsets = batch.map { - case (k, v) => k -> new OffsetAndMetadata(v) + val offsets = batch.map { case (k, v) => + k -> new OffsetAndMetadata(v) }.asJava consumer.commitAsync( offsets, diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala index 90656598..de704730 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala @@ -272,7 +272,7 @@ case class KafkaProducerConfig( ) def toJavaMap: java.util.Map[String, Object] = - toMap.filter(_._2 != null).map{case (a, b) =>(a, b.asInstanceOf[AnyRef])}.asJava + toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava def toProperties: Properties = { val props = new Properties() diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 2d289309..56cdf168 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -9,7 +9,6 @@ import org.scalatest.{FunSuite, Matchers} import scala.concurrent.duration._ import scala.concurrent.Await import monix.execution.Scheduler.Implicits.global -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition import org.scalacheck.Gen import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks @@ -18,6 +17,7 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit + override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit }) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index ee8cdc07..3d91c0aa 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -17,7 +17,6 @@ package monix.kafka -import cats.syntax.apply._ import monix.eval.Task import monix.execution.Scheduler.Implicits.global import monix.kafka.config.AutoOffsetReset @@ -194,7 +193,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - pollInterval = 100.millis + heartbeatInterval = 10.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) From fa380f6f524de448d80c04ea33c5a9422106b4e2 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 23 Jan 2021 22:59:14 +0100 Subject: [PATCH 04/61] Resolves merge conflicts --- kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala | 6 ------ .../src/main/scala/monix/kafka/CommittableOffset.scala | 9 --------- .../main/scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- .../src/test/scala/monix/kafka/MonixKafkaTest.scala | 2 +- 4 files changed, 2 insertions(+), 17 deletions(-) diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala index 2f4db548..8a9a4539 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/Commit.scala @@ -18,7 +18,6 @@ package monix.kafka import monix.eval.Task import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.OffsetCommitCallback /** Callback for batched commit realized as closure in [[KafkaConsumerObservable]] context. */ @@ -31,12 +30,7 @@ private[kafka] object Commit { val empty: Commit = new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - <<<<<<< refs / remotes / monix / master - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit - ======= override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - >>>>>>> Apply changes to older versions } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala index 3914139a..741d6c0e 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Represents offset for specified topic and partition that can be @@ -46,15 +45,7 @@ final class CommittableOffset private[kafka] ( * to use batched commit with [[CommittableOffsetBatch]] class. */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(Map(topicPartition -> offset)) - <<<<<<< refs / remotes / monix / master - /** Asynchronously commits [[offset]] to Kafka. It is recommended - * to use batched commit with [[CommittableOffsetBatch]] class. - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = - commitCallback.commitBatchAsync(Map(topicPartition -> offset), callback) - ======= - >>>>>>> Apply changes to older versions } object CommittableOffset { diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 094bc949..fae224d8 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -17,7 +17,7 @@ package monix.kafka import cats.effect.Resource -import monix.eval.{Fiber, Task} +import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder diff --git a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala index 7cfba264..469d938f 100644 --- a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala +++ b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala @@ -221,7 +221,7 @@ class MonixKafkaTest extends FunSuite { val consumerConfig = consumerCfg.copy( sessionTimeout = 200.millis, - pollInterval = 100.millis + observablePollHeartbeatRate = 100.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) From 7232568e758d2743bc39dc0c4bf585322d0077aa Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 23 Jan 2021 23:42:09 +0100 Subject: [PATCH 05/61] Unused imports --- kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala | 1 - .../src/main/scala/monix/kafka/CommittableOffset.scala | 1 - .../main/scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- .../kafka/KafkaConsumerObservableManualCommit.scala | 9 ++++----- .../scala/monix/kafka/MergeByCommitCallbackTest.scala | 7 ------- 5 files changed, 5 insertions(+), 15 deletions(-) diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala index 6009d5ab..b174ef76 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/Commit.scala @@ -18,7 +18,6 @@ package monix.kafka import monix.eval.Task import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.OffsetCommitCallback /** Callback for batched commit realized as closure in [[KafkaConsumerObservable]] context. */ diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala index 1c8831af..9bf00254 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Represents offset for specified topic and partition that can be diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index f5e762d8..e5a9ff89 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -17,7 +17,7 @@ package monix.kafka import cats.effect.Resource -import monix.eval.{Fiber, Task} +import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 14c45d14..3b50a789 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -67,10 +67,10 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override def onComplete( offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = { - if (exception != null) { - if (!asyncCb.tryOnError(exception)) - s.reportFailure(exception) - } else { + if (exception != null && !asyncCb.tryOnError(exception)) { + s.reportFailure(exception) + } + else { asyncCb.tryOnSuccess(()) } } @@ -85,7 +85,6 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( } } } - >>>>>>> Apply changes to older versions } override protected def ackTask(consumer: Consumer[K, V], out: Subscriber[CommittableMessage[K, V]]): Task[Ack] = diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index 69381117..7d854a66 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -9,7 +9,6 @@ import org.scalatest.{FunSuite, Matchers} import scala.concurrent.duration._ import scala.concurrent.Await import monix.execution.Scheduler.Implicits.global -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition import org.scalacheck.Gen import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks @@ -18,13 +17,7 @@ class MergeByCommitCallbackTest extends FunSuite with KafkaTestKit with ScalaChe val commitCallbacks: List[Commit] = List.fill(4)(new Commit { override def commitBatchSync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - <<<<<<< refs / remotes / monix / master - - override def commitBatchAsync(batch: Map[TopicPartition, Long], callback: OffsetCommitCallback): Task[Unit] = - Task.unit - ======= override def commitBatchAsync(batch: Map[TopicPartition, Long]): Task[Unit] = Task.unit - >>>>>>> Apply changes to older versions }) val committableOffsetsGen: Gen[CommittableOffset] = for { From 32d238cb6fb48501aea8e81d5fa29e4a626d1dde Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 24 Jan 2021 13:57:34 +0100 Subject: [PATCH 06/61] Multiple threads test failing --- .../src/main/scala/monix/kafka/Commit.scala | 1 - .../scala/monix/kafka/CommittableOffset.scala | 1 - .../monix/kafka/CommittableOffsetBatch.scala | 5 -- .../monix/kafka/KafkaConsumerObservable.scala | 25 ++++---- .../KafkaConsumerObservableManualCommit.scala | 57 ++++++------------ .../monix/kafka/MonixKafkaTopicListTest.scala | 45 +++++++++++++- .../monix/kafka/KafkaConsumerObservable.scala | 25 ++++---- .../KafkaConsumerObservableManualCommit.scala | 54 ++++++----------- .../main/resources/monix/kafka/default.conf | 3 +- .../monix/kafka/KafkaConsumerObservable.scala | 17 ++---- .../KafkaConsumerObservableManualCommit.scala | 54 ++++++----------- .../monix/kafka/KafkaConsumerObservable.scala | 24 ++++---- .../KafkaConsumerObservableManualCommit.scala | 60 +++++++------------ .../monix/kafka/MonixKafkaTopicListTest.scala | 41 ++++++++++++- 14 files changed, 196 insertions(+), 216 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala index 6009d5ab..b174ef76 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/Commit.scala @@ -18,7 +18,6 @@ package monix.kafka import monix.eval.Task import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.OffsetCommitCallback /** Callback for batched commit realized as closure in [[KafkaConsumerObservable]] context. */ diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala index 1c8831af..9bf00254 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffset.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Represents offset for specified topic and partition that can be diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala index 4bfee4ad..7d7d2bad 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/CommittableOffsetBatch.scala @@ -17,7 +17,6 @@ package monix.kafka import monix.eval.Task -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition /** Batch of Kafka offsets which can be committed together. @@ -45,10 +44,6 @@ final class CommittableOffsetBatch private[kafka] (val offsets: Map[TopicPartiti */ def commitAsync(): Task[Unit] = commitCallback.commitBatchAsync(offsets) - /** Asynchronously commits [[offsets]] to Kafka - */ - def commitAsync(callback: OffsetCommitCallback): Task[Unit] = commitCallback.commitBatchAsync(offsets) - /** Adds new [[CommittableOffset]] to batch. Added offset replaces previous one specified * for same topic and partition. */ diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index f5e762d8..540b77e8 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -17,7 +17,7 @@ package monix.kafka import cats.effect.Resource -import monix.eval.{Fiber, Task} +import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder @@ -76,7 +76,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - heartbeat(c) + pollHeartbeat(c) // If polling fails the error is reported to the subscriber and // wait 1sec as a rule of thumb leaving enough time for the consumer // to recover and reassign partitions @@ -108,21 +108,16 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task - .sleep(config.observablePollHeartbeatRate) - .flatMap { _ => + private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { + Task.sleep(config.observablePollHeartbeatRate) *> + Task.evalAsync( if (!isAcked) { - Task.evalAsync { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") - } + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } - } else { - Task.unit - } - } + } else () + ) } } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index e5999fc5..60c2e676 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,8 +16,6 @@ package monix.kafka -import java.util - import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -56,31 +54,15 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - consumer.synchronized { - try { - val offsets = batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava - consumer.commitAsync( - offsets, - new OffsetCommitCallback { - override def onComplete( - offsets: util.Map[TopicPartition, OffsetAndMetadata], - exception: Exception): Unit = { - if (exception != null) { - if (!asyncCb.tryOnError(exception)) - s.reportFailure(exception) - } else { - asyncCb.tryOnSuccess(()) - } - } - } - ) - } catch { - case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) - s.reportFailure(ex) - } + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } + } + try { + consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) + } catch { case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } @@ -92,21 +74,18 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( implicit val s = scheduler val asyncCb = Callback.forked(cb) val cancelable = BooleanCancelable() - val commit: Commit = CommitWithConsumer(consumer) // Forced asynchronous boundary (on the I/O scheduler) s.executeAsync { () => - val ackFuture = - try consumer.synchronized { - val assignment = consumer.assignment() - if (cancelable.isCanceled) Stop - else { + val ackFuture: Future[Ack] = + if (cancelable.isCanceled) Stop + else { + try consumer.synchronized { + val assignment = consumer.assignment() consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment) - // Feeding the observer happens on the Subscriber's scheduler - // if any asynchronous boundaries happen val result = next.asScala.map { record => CommittableMessage( record, @@ -115,12 +94,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + // Feeding the observer happens on the Subscriber's scheduler + // if any asynchronous boundaries happen isAcked = false Observer.feed(out, result)(out.scheduler) + } catch { + case NonFatal(ex) => + Future.failed(ex) } - } catch { - case NonFatal(ex) => - Future.failed(ex) } ackFuture.syncOnComplete { diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 5c1c67a0..3a13cf88 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -24,6 +24,7 @@ import monix.reactive.Observable import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.producer.ProducerRecord import org.scalatest.FunSuite + import scala.concurrent.Await import scala.concurrent.duration._ import scala.collection.JavaConverters._ @@ -43,6 +44,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { autoOffsetReset = AutoOffsetReset.Earliest ) + /* test("publish one message") { withRunningKafka { @@ -187,14 +189,14 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } - - test("slow batches processing doesn't cause rebalancing") { + */ + test("slow batches autocommit processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - pollInterval = 100.millis + observablePollHeartbeatRate = 10.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -218,4 +220,41 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } + + test("slow manual async commit doesn't cause rebalancing") { + withRunningKafka { + + val count = 4 + val topicName = "monix-kafka-manual-commit-tests" + val fastPollHeartbeatConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + observablePollHeartbeatRate = 10.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .mapEval { committableMessage => + Task.sleep(100.millis) *> + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().as(committableMessage) + } + .toListL + + val (commitableMessages, _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = commitableMessages.last + assert((0 until count).sum === commitableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(10000 === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) //still in partition 0 + } + } } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index e5a9ff89..4ac88be6 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -76,7 +76,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - heartbeat(c) + pollHeartbeat(c) // If polling fails the error is reported to the subscriber and // wait 1sec as a rule of thumb leaving enough time for the consumer // to recover and reassign partitions @@ -108,22 +108,17 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task - .sleep(config.observablePollHeartbeatRate) - .flatMap { _ => + private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { + Task.sleep(config.observablePollHeartbeatRate) *> + Task.evalAsync( if (!isAcked) { - Task.evalAsync { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") - } + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } - } else { - Task.unit - } - } - } + } else () + ) + } } object KafkaConsumerObservable { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 3b50a789..5c098d9c 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -56,31 +56,15 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - consumer.synchronized { - try { - val offsets = batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava - consumer.commitAsync( - offsets, - new OffsetCommitCallback { - override def onComplete( - offsets: util.Map[TopicPartition, OffsetAndMetadata], - exception: Exception): Unit = { - if (exception != null && !asyncCb.tryOnError(exception)) { - s.reportFailure(exception) - } - else { - asyncCb.tryOnSuccess(()) - } - } - } - ) - } catch { - case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) - s.reportFailure(ex) - } + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } + } + try { + consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) + } catch { case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } @@ -97,16 +81,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( // Forced asynchronous boundary (on the I/O scheduler) s.executeAsync { () => - val ackFuture = - try consumer.synchronized { - val assignment = consumer.assignment() - if (cancelable.isCanceled) Stop - else { + val ackFuture: Future[Ack] = + if (cancelable.isCanceled) Stop + else { + try consumer.synchronized { + val assignment = consumer.assignment() consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment) - // Feeding the observer happens on the Subscriber's scheduler - // if any asynchronous boundaries happen val result = next.asScala.map { record => CommittableMessage( record, @@ -115,12 +97,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + // Feeding the observer happens on the Subscriber's scheduler + // if any asynchronous boundaries happen isAcked = false Observer.feed(out, result)(out.scheduler) + } catch { + case NonFatal(ex) => + Future.failed(ex) } - } catch { - case NonFatal(ex) => - Future.failed(ex) } ackFuture.syncOnComplete { diff --git a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf index 4a6bddfe..27a6f7bb 100644 --- a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf @@ -72,5 +72,6 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - monix.observable.poll.interval.ms = 100 + # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. + monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index fae224d8..83ff3e0b 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -75,7 +75,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd() else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning() // A task to execute on both cancellation and normal termination - heartbeat(c) + pollHeartbeat(c) // If polling fails the error is reported to the subscriber and // wait 1sec as a rule of thumb leaving enough time for the consumer // to recover and reassign partitions @@ -107,21 +107,16 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task - .sleep(config.observablePollHeartbeatRate) - .flatMap { _ => + private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { + Task.sleep(config.observablePollHeartbeatRate) *> + Task.evalAsync( if (!isAcked) { - Task.evalAsync { var records = blocking(consumer.poll(0)) if (!records.isEmpty) { throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } - } - } else { - Task.unit - } - } + } else () + ) } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index cfe0b6d7..5b3e1c5e 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -56,31 +56,15 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - consumer.synchronized { - try { - val offsets = batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava - consumer.commitAsync( - offsets, - new OffsetCommitCallback { - override def onComplete( - offsets: util.Map[TopicPartition, OffsetAndMetadata], - exception: Exception): Unit = { - if (exception != null) { - if (!asyncCb.tryOnError(exception)) - s.reportFailure(exception) - } else { - asyncCb.tryOnSuccess(()) - } - } - } - ) - } catch { - case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) - s.reportFailure(ex) - } + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } + } + try { + consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) + } catch { case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } @@ -97,16 +81,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( // Forced asynchronous boundary (on the I/O scheduler) s.executeAsync { () => - val ackFuture = - try consumer.synchronized { - val assignment = consumer.assignment().asScala.toArray - if (cancelable.isCanceled) Stop - else { + val ackFuture: Future[Ack] = + if (cancelable.isCanceled) Stop + else { + try consumer.synchronized { + val assignment = consumer.assignment().asScala.toList consumer.resume(assignment: _*) val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment: _*) - // Feeding the observer happens on the Subscriber's scheduler - // if any asynchronous boundaries happen val result = next.asScala.map { record => CommittableMessage( record, @@ -115,12 +97,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + // Feeding the observer happens on the Subscriber's scheduler + // if any asynchronous boundaries happen isAcked = false Observer.feed(out, result)(out.scheduler) + } catch { + case NonFatal(ex) => + Future.failed(ex) } - } catch { - case NonFatal(ex) => - Future.failed(ex) } ackFuture.syncOnComplete { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 475a01c2..dfd3b907 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -77,7 +77,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) // A task to execute on both cancellation and normal termination - heartbeat(c) + pollHeartbeat(c) // If polling fails the error is reported to the subscriber and // wait 1sec as a rule of thumb leaving enough time for the consumer // to recover and reassign partitions @@ -109,22 +109,18 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def heartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task - .sleep(config.observablePollHeartbeatRate) - .flatMap { _ => + private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { + Task.sleep(config.observablePollHeartbeatRate) *> + Task.evalAsync( if (!isAcked) { - Task.evalAsync { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") - } + var records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages") } - } else { - Task.unit - } - } + } else () + ) } + } object KafkaConsumerObservable { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 2cb5634f..951fda67 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,8 +16,6 @@ package monix.kafka -import java.util - import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -27,7 +25,7 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition -import scala.concurrent.{blocking, Future} +import scala.concurrent.{Future, blocking} import scala.util.control.NonFatal import scala.util.{Failure, Success} import scala.jdk.CollectionConverters._ @@ -56,31 +54,15 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - consumer.synchronized { - try { - val offsets = batch.map { case (k, v) => - k -> new OffsetAndMetadata(v) - }.asJava - consumer.commitAsync( - offsets, - new OffsetCommitCallback { - override def onComplete( - offsets: util.Map[TopicPartition, OffsetAndMetadata], - exception: Exception): Unit = { - if (exception != null) { - if (!asyncCb.tryOnError(exception)) - s.reportFailure(exception) - } else { - asyncCb.tryOnSuccess(()) - } - } - } - ) - } catch { - case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) - s.reportFailure(ex) - } + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } + } + try { + consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) + } catch { case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } @@ -97,16 +79,14 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( // Forced asynchronous boundary (on the I/O scheduler) s.executeAsync { () => - val ackFuture = - try consumer.synchronized { - val assignment = consumer.assignment() - if (cancelable.isCanceled) Stop - else { + val ackFuture: Future[Ack] = + if (cancelable.isCanceled) Stop + else { + try consumer.synchronized { + val assignment = consumer.assignment() consumer.resume(assignment) val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment) - // Feeding the observer happens on the Subscriber's scheduler - // if any asynchronous boundaries happen val result = next.asScala.map { record => CommittableMessage( record, @@ -115,13 +95,15 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( record.offset() + 1, commit)) } + // Feeding the observer happens on the Subscriber's scheduler + // if any asynchronous boundaries happen isAcked = false Observer.feed(out, result)(out.scheduler) - } } catch { - case NonFatal(ex) => - Future.failed(ex) - } + case NonFatal(ex) => + Future.failed(ex) + } + } ackFuture.syncOnComplete { case Success(ack) => diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 3d91c0aa..66205981 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -25,7 +25,6 @@ import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition import org.scalatest.FunSuite -import scala.collection.JavaConverters._ import scala.concurrent.Await import scala.concurrent.duration._ @@ -43,7 +42,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { clientId = "monix-kafka-1-0-consumer-test", autoOffsetReset = AutoOffsetReset.Earliest ) - +/* test("publish one message when subscribed to topics list") { withRunningKafka { @@ -186,7 +185,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } - + */ test("slow batches processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 @@ -217,4 +216,40 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } + + test("slow manual async commit doesn't cause rebalancing") { + withRunningKafka { + + val count = 4 + val topicName = "monix-kafka-manual-commit-tests" + val fastPollHeartbeatConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + observablePollHeartbeatRate = 10.millis + ) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEval { committableMessage => + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().as(committableMessage) + } + .toListL + + val (committableMessages, _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert((0 until count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(10000 === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) //still in partition 0 + } + } } From 622b1804bce98a9818bdd51086192efdc1583191 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 24 Jan 2021 18:51:00 +0100 Subject: [PATCH 07/61] Test small poll heartbeat and slow upstream --- .../monix/kafka/KafkaConsumerObservable.scala | 8 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 76 ++++++++++--------- .../monix/kafka/KafkaConsumerObservable.scala | 10 ++- .../KafkaConsumerObservableManualCommit.scala | 2 - .../monix/kafka/MonixKafkaTopicListTest.scala | 41 ++++++++++ .../monix/kafka/KafkaConsumerObservable.scala | 6 +- .../monix/kafka/KafkaConsumerObservable.scala | 8 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 31 ++++---- 8 files changed, 118 insertions(+), 64 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 540b77e8..4903131f 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -112,9 +112,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.sleep(config.observablePollHeartbeatRate) *> Task.evalAsync( if (!isAcked) { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") + consumer.synchronized { + val records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + } } } else () ) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 3a13cf88..9776b309 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -44,7 +44,6 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { autoOffsetReset = AutoOffsetReset.Earliest ) - /* test("publish one message") { withRunningKafka { @@ -110,6 +109,24 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } + test("publish to closed producer when subscribed to topics list") { + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val sendTask = producer.send(topicName, "test-message") + + val result = for { + //Force creation of producer + s1 <- producer.send(topicName, "test-message-1") + res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) + (_, s2) = res + s3 <- sendTask + } yield (s1, s2, s3) + + val (first, second, third) = Await.result(result.runToFuture, 60.seconds) + assert(first.isDefined && second.isRight && third.isEmpty) + } + } + test("manual commit consumer test when subscribed to topics list") { withRunningKafka { @@ -165,31 +182,13 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .headL val ((result, offsets), _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) } } - test("publish to closed producer when subscribed to topics list") { - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val sendTask = producer.send(topicName, "test-message") - - val result = for { - //Force creation of producer - s1 <- producer.send(topicName, "test-message-1") - res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) - (_, s2) = res - s3 <- sendTask - } yield (s1, s2, s3) - - val (first, second, third) = Await.result(result.runToFuture, 60.seconds) - assert(first.isDefined && second.isRight && third.isEmpty) - } - } - */ test("slow batches autocommit processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 @@ -221,40 +220,43 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow manual async commit doesn't cause rebalancing") { + test("slow upstream with small poll heartbeat and manual async commit keeps the consumer assignment") { withRunningKafka { - val count = 4 + val count = 250 val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis val fastPollHeartbeatConfig = consumerCfg.copy( maxPollInterval = 200.millis, - observablePollHeartbeatRate = 10.millis - ) + observablePollHeartbeatRate = pollHeartbeat) - val producer = KafkaProducerSink[String, String](producerCfg, io) + val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) val pushT = Observable - .range(0, count) + .fromIterable(1 to count) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) + .mapEval(producer.send) + .lastL val listT = consumer .executeOn(io) - .mapEval { committableMessage => - Task.sleep(100.millis) *> - CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().as(committableMessage) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync } + .take(count) .toListL - val (commitableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) - val CommittableMessage(lastRecord, lastCommittableOffset) = commitableMessages.last - assert((0 until count).sum === commitableMessages.map(_.record.value().toInt).sum) + val (committableMessages, _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) - assert(10000 === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) //still in partition 0 + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) } } } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 4ac88be6..4903131f 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -112,13 +112,15 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.sleep(config.observablePollHeartbeatRate) *> Task.evalAsync( if (!isAcked) { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") + consumer.synchronized { + val records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + } } } else () ) - } + } } object KafkaConsumerObservable { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 5c098d9c..12e0a3df 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,8 +16,6 @@ package monix.kafka -import java.util - import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 63d5a584..bbfb3617 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -219,4 +219,45 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } + + test("slow upstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + withRunningKafka { + + val count = 250 + val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis + val fastPollHeartbeatConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + } + .take(count) + .toListL + + val (committableMessages, _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) + } + } + } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 83ff3e0b..1dda2737 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -111,10 +111,12 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.sleep(config.observablePollHeartbeatRate) *> Task.evalAsync( if (!isAcked) { - var records = blocking(consumer.poll(0)) + consumer.synchronized { + val records = blocking(consumer.poll(0)) if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") + throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") } + } } else () ) } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index dfd3b907..3f8c8a0c 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -113,9 +113,11 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.sleep(config.observablePollHeartbeatRate) *> Task.evalAsync( if (!isAcked) { - var records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages") + consumer.synchronized { + val records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + } } } else () ) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 66205981..0f257052 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -27,6 +27,7 @@ import org.scalatest.FunSuite import scala.concurrent.Await import scala.concurrent.duration._ +import scala.jdk.CollectionConverters._ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val topicName = "monix-kafka-tests" @@ -42,7 +43,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { clientId = "monix-kafka-1-0-consumer-test", autoOffsetReset = AutoOffsetReset.Earliest ) -/* + test("publish one message when subscribed to topics list") { withRunningKafka { @@ -185,7 +186,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } - */ + test("slow batches processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 @@ -217,39 +218,43 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow manual async commit doesn't cause rebalancing") { + test("slow upstream with manual async commit commit doesn't cause rebalancing") { withRunningKafka { - val count = 4 + val count = 200 val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis val fastPollHeartbeatConfig = consumerCfg.copy( maxPollInterval = 200.millis, - observablePollHeartbeatRate = 10.millis - ) + observablePollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) val pushT = Observable - .range(0, count) + .fromIterable(1 to count) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) .mapEval(producer.send) .lastL val listT = consumer .executeOn(io) - .mapEval { committableMessage => - CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().as(committableMessage) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync } + .take(count) .toListL val (committableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert((0 until count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) - assert(10000 === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) //still in partition 0 + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) } } } From 1365c7cfcb0324d3b7d4abbec0d4f43f00e246e8 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 24 Jan 2021 19:16:03 +0100 Subject: [PATCH 08/61] ScalafmtAll --- .../KafkaConsumerObservableManualCommit.scala | 7 +-- .../monix/kafka/MonixKafkaTopicListTest.scala | 44 +++++++++---------- .../KafkaConsumerObservableManualCommit.scala | 7 +-- .../monix/kafka/MonixKafkaTopicListTest.scala | 7 ++- .../KafkaConsumerObservableManualCommit.scala | 7 +-- .../KafkaConsumerObservableManualCommit.scala | 15 ++++--- .../monix/kafka/MonixKafkaTopicListTest.scala | 9 ++-- 7 files changed, 49 insertions(+), 47 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 60c2e676..6742f716 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -54,15 +54,16 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) - } catch { case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 9776b309..93a99425 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -109,24 +109,6 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("publish to closed producer when subscribed to topics list") { - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val sendTask = producer.send(topicName, "test-message") - - val result = for { - //Force creation of producer - s1 <- producer.send(topicName, "test-message-1") - res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) - (_, s2) = res - s3 <- sendTask - } yield (s1, s2, s3) - - val (first, second, third) = Await.result(result.runToFuture, 60.seconds) - assert(first.isDefined && second.isRight && third.isEmpty) - } - } - test("manual commit consumer test when subscribed to topics list") { withRunningKafka { @@ -157,6 +139,24 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } + test("publish to closed producer when subscribed to topics list") { + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val sendTask = producer.send(topicName, "test-message") + + val result = for { + //Force creation of producer + s1 <- producer.send(topicName, "test-message-1") + res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) + (_, s2) = res + s3 <- sendTask + } yield (s1, s2, s3) + + val (first, second, third) = Await.result(result.runToFuture, 60.seconds) + assert(first.isDefined && second.isRight && third.isEmpty) + } + } + test("manual async commit consumer test when subscribed to topics list") { withRunningKafka { @@ -227,9 +227,8 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val topicName = "monix-kafka-manual-commit-tests" val delay = 200.millis val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - observablePollHeartbeatRate = pollHeartbeat) + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -256,7 +255,8 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) } } + } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 12e0a3df..9b38dc70 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -54,15 +54,16 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) - } catch { case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index bbfb3617..4ecb63d6 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -227,9 +227,8 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val topicName = "monix-kafka-manual-commit-tests" val delay = 200.millis val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - observablePollHeartbeatRate = pollHeartbeat) + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -256,7 +255,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) } } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 5b3e1c5e..672ab6eb 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -56,15 +56,16 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) - } catch { case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 951fda67..494e1c1c 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -25,7 +25,7 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition -import scala.concurrent.{Future, blocking} +import scala.concurrent.{blocking, Future} import scala.util.control.NonFatal import scala.util.{Failure, Success} import scala.jdk.CollectionConverters._ @@ -54,15 +54,16 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( .async0[Unit] { (s, cb) => val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => - val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v)}.asJava + val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) - } catch { case NonFatal(ex) => - if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } + } catch { + case NonFatal(ex) => + if (!asyncCb.tryOnError(ex)) { s.reportFailure(ex) } } } } @@ -99,11 +100,11 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( // if any asynchronous boundaries happen isAcked = false Observer.feed(out, result)(out.scheduler) - } catch { + } catch { case NonFatal(ex) => Future.failed(ex) - } - } + } + } ackFuture.syncOnComplete { case Success(ack) => diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 0f257052..d8a08919 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -225,9 +225,8 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val topicName = "monix-kafka-manual-commit-tests" val delay = 200.millis val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - observablePollHeartbeatRate = pollHeartbeat) + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -242,7 +241,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .executeOn(io) .doOnNextF { committableMessage => Task.sleep(delay) *> - Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync } .take(count) .toListL @@ -254,7 +253,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition ) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) } } } From c6c1c326b0a0e6efd22542381437018704f3b828 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 24 Jan 2021 20:10:49 +0100 Subject: [PATCH 09/61] Removes lambda anonymous class for scala 11 compatibility --- .../kafka/KafkaConsumerObservableManualCommit.scala | 10 ++++++---- .../kafka/KafkaConsumerObservableManualCommit.scala | 10 ++++++---- .../kafka/KafkaConsumerObservableManualCommit.scala | 12 ++++++------ .../kafka/KafkaConsumerObservableManualCommit.scala | 10 ++++++---- 4 files changed, 24 insertions(+), 18 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 6742f716..36e0f985 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -25,8 +25,9 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition +import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{blocking, Future} +import scala.concurrent.{Future, blocking} import scala.util.control.NonFatal import scala.util.{Failure, Success} @@ -55,9 +56,10 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava - val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => - if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } - else { cb.tryOnSuccess(()) } + val offsetCommitCallback = new OffsetCommitCallback { + def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], ex: Exception): Unit = + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 9b38dc70..4407a927 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -25,8 +25,9 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition +import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{blocking, Future} +import scala.concurrent.{Future, blocking} import scala.util.control.NonFatal import scala.util.{Failure, Success} @@ -55,9 +56,10 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava - val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => - if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } - else { cb.tryOnSuccess(()) } + val offsetCommitCallback = new OffsetCommitCallback { + def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], ex: Exception): Unit = + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 672ab6eb..981c6d87 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -16,8 +16,6 @@ package monix.kafka -import java.util - import monix.eval.Task import monix.execution.Ack.Stop import monix.execution.cancelables.BooleanCancelable @@ -27,8 +25,9 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition +import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{blocking, Future} +import scala.concurrent.{Future, blocking} import scala.util.control.NonFatal import scala.util.{Failure, Success} @@ -57,9 +56,10 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava - val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => - if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } - else { cb.tryOnSuccess(()) } + val offsetCommitCallback = new OffsetCommitCallback { + def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], ex: Exception): Unit = + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 494e1c1c..92017a07 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -25,7 +25,8 @@ import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition -import scala.concurrent.{blocking, Future} +import java.util +import scala.concurrent.{Future, blocking} import scala.util.control.NonFatal import scala.util.{Failure, Success} import scala.jdk.CollectionConverters._ @@ -55,9 +56,10 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( val asyncCb = Callback.forked(cb)(s) s.executeAsync { () => val offsets = batch.map { case (k, v) => k -> new OffsetAndMetadata(v) }.asJava - val offsetCommitCallback: OffsetCommitCallback = { (_, ex) => - if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } - else { cb.tryOnSuccess(()) } + val offsetCommitCallback = new OffsetCommitCallback { + def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], ex: Exception): Unit = + if (ex != null && !cb.tryOnError(ex)) { s.reportFailure(ex) } + else { cb.tryOnSuccess(()) } } try { consumer.synchronized(consumer.commitAsync(offsets, offsetCommitCallback)) From 512f56c77030b88c81f4974a49f0c2d7f3abbc01 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 24 Jan 2021 20:38:39 +0100 Subject: [PATCH 10/61] Scaladoc fix --- .../src/main/scala/monix/kafka/KafkaConsumerConfig.scala | 4 ++-- .../src/main/scala/monix/kafka/KafkaConsumerConfig.scala | 4 ++-- .../src/main/scala/monix/kafka/KafkaConsumerConfig.scala | 4 ++-- .../src/main/scala/monix/kafka/KafkaConsumerConfig.scala | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index c987958f..ad60d84c 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -198,8 +198,8 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param pollInterval is the `monix.observable.poll.interval.ms` setting. - * Specifies time between KafkaConsumer#poll call attempts. + * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. + * Specifies heartbeat time between KafkaConsumer#poll attempts. * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 12e71563..f9e42e61 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -202,8 +202,8 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param pollInterval is the `monix.observable.poll.interval.ms` setting. - * Specifies time between KafkaConsumer#poll call attempts. + * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. + * Specifies heartbeat time between KafkaConsumer#poll attempts. * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 20905eca..7eeee574 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -194,8 +194,8 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param pollInterval is the `monix.observable.poll.interval.ms` setting. - * Specifies time between KafkaConsumer#poll call attempts. + * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. + * Specifies heartbeat time between KafkaConsumer#poll attempts. * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 3b79e63f..5680fb65 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -207,8 +207,8 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param heartbeatInterval is the `monix.observable.poll.interval.ms` setting. - * Specifies time between KafkaConsumer#poll call attempts. + * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. + * Specifies heartbeat time between KafkaConsumer#poll attempts. * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled From 2651712fa13c026be31e9bcc83313065498f4f3e Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 21 Mar 2021 10:16:49 +0100 Subject: [PATCH 11/61] MR Feedback --- .../monix/kafka/MonixKafkaTopicListTest.scala | 4 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 4 +- .../monix/kafka/KafkaConsumerObservable.scala | 134 ++++++++---------- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 4 +- 6 files changed, 69 insertions(+), 81 deletions(-) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 93a99425..303fdeb6 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -220,7 +220,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow upstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { withRunningKafka { val count = 250 @@ -243,7 +243,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .executeOn(io) .doOnNextF { committableMessage => Task.sleep(delay) *> - Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync } .take(count) .toListL diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 4ecb63d6..d9f30775 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -220,7 +220,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow upstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { withRunningKafka { val count = 250 @@ -243,7 +243,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .executeOn(io) .doOnNextF { committableMessage => Task.sleep(delay) *> - Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync } .take(count) .toListL diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 3f8c8a0c..5ebe210e 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,10 +16,10 @@ package monix.kafka -import cats.effect.Resource +import cats.effect.{Bracket, Resource} import monix.eval.Task import monix.execution.Ack.{Continue, Stop} -import monix.execution.{Ack, Callback, Cancelable} +import monix.execution.{Ack, Callback, Cancelable, Scheduler} import monix.kafka.config.ObservableCommitOrder import monix.reactive.Observable import monix.reactive.observers.Subscriber @@ -39,7 +39,7 @@ import scala.util.matching.Regex trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumer: Task[Consumer[K, V]] + protected def consumerTask: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -67,25 +67,16 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - Resource - .make(consumer) { c => - // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess - } - .use { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollHeartbeat(c) - // If polling fails the error is reported to the subscriber and - // wait 1sec as a rule of thumb leaving enough time for the consumer - // to recover and reassign partitions - .onErrorHandleWith(ex => Task(out.onError(ex))) - .loopForever - .startAndForget - .flatMap(_ => runLoop(c, out)) - } + consumerTask.bracket { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + Task.race(runLoop(c, out), pollHeartbeat(c).loopForever) + .void + } { consumer => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + } startConsuming.runAsync(cb) } } @@ -107,20 +98,23 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * This allows producer process commit calls and also keeps consumer alive even * with long batch processing. * + * If polling fails the error is reported to the subscriber through the scheduler. + * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { + private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { Task.sleep(config.observablePollHeartbeatRate) *> - Task.evalAsync( + Task.eval( if (!isAcked) { consumer.synchronized { val records = blocking(consumer.poll(0)) if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) } } } else () - ) + ).onErrorHandle(ex => scheduler.reportFailure(ex)) } } @@ -129,30 +123,28 @@ object KafkaConsumerObservable { /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param consumer is a factory for the - * `org.apache.kafka.clients.consumer.KafkaConsumer` - * instance to use for consuming from Kafka + * `org.apache.kafka.clients.consumer.KafkaConsumer` + * instance to use for consuming from Kafka */ def apply[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = new KafkaConsumerObservableAutoCommit[K, V](cfg, consumer) /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param topics is the list of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) apply(cfg, consumer) @@ -160,15 +152,14 @@ object KafkaConsumerObservable { /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) apply(cfg, consumer) @@ -188,18 +179,17 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param consumer is a factory for the - * `org.apache.kafka.clients.consumer.KafkaConsumer` - * instance to use for consuming from Kafka + * `org.apache.kafka.clients.consumer.KafkaConsumer` + * instance to use for consuming from Kafka */ def manualCommit[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val manualCommitConfig = cfg.copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) new KafkaConsumerObservableManualCommit[K, V](manualCommitConfig, consumer) @@ -219,16 +209,15 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param topics is the list of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) manualCommit(cfg, consumer) @@ -248,16 +237,15 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) manualCommit(cfg, consumer) @@ -265,8 +253,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given list of topics. */ def createConsumer[K, V](config: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap @@ -280,8 +268,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given topics regex. */ def createConsumer[K, V](config: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap blocking { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 09e0e200..d74de333 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 92017a07..922ac987 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.jdk.CollectionConverters._ */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index d8a08919..24575a44 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -218,7 +218,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow upstream with manual async commit commit doesn't cause rebalancing") { + test("slow downstream with manual async commit commit doesn't cause rebalancing") { withRunningKafka { val count = 200 @@ -241,7 +241,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { .executeOn(io) .doOnNextF { committableMessage => Task.sleep(delay) *> - Task.shift >> CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync() } .take(count) .toListL From 336a9f2e98e71709cb70270368bcd48f2b85b9b6 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 21 Mar 2021 13:44:26 +0100 Subject: [PATCH 12/61] Unused import --- .../src/main/scala/monix/kafka/KafkaConsumerObservable.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 5ebe210e..98d5d034 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,6 @@ package monix.kafka -import cats.effect.{Bracket, Resource} import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable, Scheduler} From b83531003c3ec7c0f7dc8e0d32b5237a3ddfc224 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 21 Mar 2021 20:22:23 +0100 Subject: [PATCH 13/61] Adding poll heartbeat tests --- .../monix/kafka/KafkaConsumerObservable.scala | 8 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 69 ------ .../scala/monix/kafka/PollHeartBeatTest.scala | 228 ++++++++++++++++++ 3 files changed, 233 insertions(+), 72 deletions(-) create mode 100644 kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 98d5d034..0d610a2c 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -70,8 +70,8 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // Skipping all available messages on all partitions if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - Task.race(runLoop(c, out), pollHeartbeat(c).loopForever) - .void + Task.race(runLoop(c, out), pollHeartbeat(c)).void + //runLoop(c, out).void } { consumer => // Forced asynchronous boundary Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess @@ -113,7 +113,9 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } } else () - ).onErrorHandle(ex => scheduler.reportFailure(ex)) + ).onErrorHandle(ex => scheduler.reportFailure(ex)) >> + pollHeartbeat(consumer).delayExecution(config.observablePollHeartbeatRate) + } } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 24575a44..635035a6 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -187,73 +187,4 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow batches processing doesn't cause rebalancing") { - withRunningKafka { - val count = 10000 - - val consumerConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - heartbeatInterval = 10.millis - ) - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .take(count) - .map(_.value()) - .bufferTumbling(count / 4) - .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) - .flatMap(Observable.fromIterable) - .toListL - - val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } - - test("slow downstream with manual async commit commit doesn't cause rebalancing") { - withRunningKafka { - - val count = 200 - val topicName = "monix-kafka-manual-commit-tests" - val delay = 200.millis - val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .doOnNextF { committableMessage => - Task.sleep(delay) *> - CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync() - } - .take(count) - .toListL - - val (committableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat * 10 < delay) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala new file mode 100644 index 00000000..ba994f05 --- /dev/null +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -0,0 +1,228 @@ +package monix.kafka + +import monix.eval.Task +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.TopicPartition +import monix.execution.Scheduler.Implicits.global +import org.scalatest.FunSuite +import org.scalatest.concurrent.ScalaFutures + +import scala.concurrent.Await +import scala.concurrent.duration._ + +class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { + + val topicName = "monix-kafka-tests" + override implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + + val producerCfg = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + test("slow batches processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + heartbeatInterval = 10.millis + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } + + test("slow committable downstream with small poll heartbeat does not cause rebalancing") { + withRunningKafka { + + val count = 200 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 200.millis + val pollHeartbeat = 1.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(count) + .toListL + + val (committableMessages, _) = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < downstreamLatency) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + + //unhappy scenario + test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalancing") { + withRunningKafka { + val count = 200 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 2.seconds + val pollHeartbeat = 15.seconds + val maxPollInterval = 100.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(count) + .toListL + + assert(pollHeartbeat > downstreamLatency) + assert(maxPollInterval < downstreamLatency) + + val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) + whenReady(t.runToFuture.failed) { ex => + assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) + } + + } + } + + //java.lang.IllegalStateException: Received 50 unexpected messages. + test("slow committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { + withRunningKafka { + val count = 50 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 100.seconds + val pollHeartbeat = 15.seconds + val maxPollInterval = 10.seconds + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(count) + .toListL + + val committableMessages = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runToFuture, 100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < downstreamLatency) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + + } + } + + test("slow downstream with long poll heart beat and smaller pollInterval causes rebalancing") { + withRunningKafka { + + val fastMessages = 5 + val slowMessages = 3 + val totalMessages = fastMessages + slowMessages + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 500.millis + // the downstream latency of `slowMessages` is higher than the + // `maxPollInterval` but smaller than `pollHeartBeat`, + // kafka will trigger rebalancing and the consumer + // will be kicked out of the consumer group. + val pollHeartbeat = 800.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[Integer, Integer](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalMessages) + .map(Integer.valueOf) + .map(msg => new ProducerRecord(topicName, msg, msg)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) + if(committableMessage.record.value() <= fastMessages) manualCommit + else Task.sleep(downstreamLatency) *> manualCommit + } + .take(7) + .toListL + + val committableMessages = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runSyncUnsafe() + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat > downstreamLatency) + assert(committableMessages.map(_.record.value().toInt).sum === (1 to fastMessages).sum) + assert(lastRecord.value().toInt === fastMessages) + assert(lastCommittableOffset.offset === fastMessages) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } +} From e612d647a6b3b37c31b0366ea4329410b1aabf73 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 21 Mar 2021 20:44:51 +0100 Subject: [PATCH 14/61] l --- .../scala/monix/kafka/PollHeartBeatTest.scala | 47 ++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index ba994f05..863d73a6 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -63,10 +63,11 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { test("slow committable downstream with small poll heartbeat does not cause rebalancing") { withRunningKafka { - val count = 200 + val count = 100 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 200.millis val pollHeartbeat = 1.millis + val maxPollInterval = 100.millis val fastPollHeartbeatConfig = consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) @@ -92,6 +93,8 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) + assert(pollHeartbeat < maxPollInterval) + assert(maxPollInterval < downstreamLatency) assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) @@ -140,6 +143,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } //java.lang.IllegalStateException: Received 50 unexpected messages. + //at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) test("slow committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { withRunningKafka { val count = 50 @@ -180,6 +184,47 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } } + + test("2slows committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { + withRunningKafka { + val count = 10 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 100.millis + val pollHeartbeat = 10.millis + val maxPollInterval = 5.seconds + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(count) + .toListL + + val committableMessages = + Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runToFuture, 25.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat > downstreamLatency) + assert(maxPollInterval > downstreamLatency) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + + } + } test("slow downstream with long poll heart beat and smaller pollInterval causes rebalancing") { withRunningKafka { From ca5e98ae8a9716fa2ff05c700597090766105339 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 21 Mar 2021 21:32:16 +0100 Subject: [PATCH 15/61] Improved rebalancing test scenarios --- .../scala/monix/kafka/PollHeartBeatTest.scala | 47 ++++++++++--------- 1 file changed, 25 insertions(+), 22 deletions(-) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index 863d73a6..fd15d0b1 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -55,7 +55,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .flatMap(Observable.fromIterable) .toListL - val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + val (result, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() assert(result.map(_.toInt).sum === (0 until count).sum) } } @@ -63,7 +63,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { test("slow committable downstream with small poll heartbeat does not cause rebalancing") { withRunningKafka { - val count = 100 + val totalRecords = 100 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 200.millis val pollHeartbeat = 1.millis @@ -75,29 +75,29 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) val pushT = Observable - .fromIterable(1 to count) + .fromIterable(1 to totalRecords) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) .mapEval(producer.send) .lastL val listT = consumer .executeOn(io) - .doOnNextF { committableMessage => - val manualCommit = Task.defer(CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync()) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } - .take(count) + .take(totalRecords) .toListL - val (committableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert(pollHeartbeat < maxPollInterval) assert(maxPollInterval < downstreamLatency) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) + assert((1 to totalRecords).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === totalRecords) + assert(totalRecords === lastCommittableOffset.offset) assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) } } @@ -105,7 +105,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { //unhappy scenario test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalancing") { withRunningKafka { - val count = 200 + val totalRecords = 200 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 2.seconds val pollHeartbeat = 15.seconds @@ -117,18 +117,19 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) val pushT = Observable - .fromIterable(1 to count) + .fromIterable(1 to totalRecords) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) .mapEval(producer.send) .lastL val listT = consumer .executeOn(io) - .doOnNextF { committableMessage => + .mapEvalF { committableMessage => val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } - .take(count) + .take(totalRecords) .toListL assert(pollHeartbeat > downstreamLatency) @@ -206,8 +207,9 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) - .doOnNextF { committableMessage => + .mapEvalF { committableMessage => val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) + .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } .take(count) @@ -232,14 +234,15 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val slowMessages = 3 val totalMessages = fastMessages + slowMessages val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 500.millis + val downstreamLatency = 4.seconds // the downstream latency of `slowMessages` is higher than the // `maxPollInterval` but smaller than `pollHeartBeat`, // kafka will trigger rebalancing and the consumer // will be kicked out of the consumer group. - val pollHeartbeat = 800.millis + val pollHeartbeat = 1.seconds + val maxPollInterval = 200.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[Integer, Integer](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](fastPollHeartbeatConfig, List(topicName)) @@ -253,12 +256,12 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) - .doOnNextF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().as(committableMessage)) if(committableMessage.record.value() <= fastMessages) manualCommit else Task.sleep(downstreamLatency) *> manualCommit } - .take(7) + .take(totalMessages) .toListL val committableMessages = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runSyncUnsafe() From 4bcf70c032387be8bd0d9d35b2a3381934029f9b Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 22 Mar 2021 00:04:43 +0100 Subject: [PATCH 16/61] a --- .../monix/kafka/KafkaConsumerObservable.scala | 29 ++++++++++--------- .../scala/monix/kafka/PollHeartBeatTest.scala | 16 ++++------ 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 0d610a2c..daac6991 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -70,7 +70,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { // Skipping all available messages on all partitions if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - Task.race(runLoop(c, out), pollHeartbeat(c)).void + Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void //runLoop(c, out).void } { consumer => // Forced asynchronous boundary @@ -102,19 +102,20 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { - Task.sleep(config.observablePollHeartbeatRate) *> - Task.eval( - if (!isAcked) { - consumer.synchronized { - val records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - val errorMsg = s"Received ${records.count()} unexpected messages." - throw new IllegalStateException(errorMsg) - } - } - } else () - ).onErrorHandle(ex => scheduler.reportFailure(ex)) >> - pollHeartbeat(consumer).delayExecution(config.observablePollHeartbeatRate) + Task.sleep(config.observablePollHeartbeatRate) *> + Task.defer( + Task.evalAsync { + if (!isAcked) { + consumer.synchronized { + val records = blocking(consumer.poll(0)) + if (!records.isEmpty) { + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) + } + } + } else () + }.onErrorHandle(ex => scheduler.reportFailure(ex)) + ) } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index fd15d0b1..6f2003de 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -9,7 +9,6 @@ import monix.execution.Scheduler.Implicits.global import org.scalatest.FunSuite import org.scalatest.concurrent.ScalaFutures -import scala.concurrent.Await import scala.concurrent.duration._ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { @@ -143,6 +142,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } } + //todo //java.lang.IllegalStateException: Received 50 unexpected messages. //at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) test("slow committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { @@ -173,15 +173,13 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .take(count) .toListL - val committableMessages = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runToFuture, 100.seconds) + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } } @@ -191,8 +189,8 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val count = 10 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 100.millis - val pollHeartbeat = 10.millis - val maxPollInterval = 5.seconds + val pollHeartbeat = 3000.millis + val maxPollInterval = 50000.millis val fastPollHeartbeatConfig = consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) @@ -215,8 +213,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .take(count) .toListL - val committableMessages = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runToFuture, 25.seconds) + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat > downstreamLatency) assert(maxPollInterval > downstreamLatency) @@ -224,7 +221,6 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } } test("slow downstream with long poll heart beat and smaller pollInterval causes rebalancing") { @@ -264,7 +260,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .take(totalMessages) .toListL - val committableMessages = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).runSyncUnsafe() + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat > downstreamLatency) assert(committableMessages.map(_.record.value().toInt).sum === (1 to fastMessages).sum) From f93ae9adc72ecf6c2146fa9f8ac0f74a379d836a Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 22 Mar 2021 11:20:00 +0100 Subject: [PATCH 17/61] Improved test --- .../monix/kafka/MonixKafkaTopicListTest.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 635035a6..dacfddc2 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -140,32 +140,32 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { test("manual async commit consumer test when subscribed to topics list") { withRunningKafka { - val count = 10000 + val totalRecords = 10000 val topicName = "monix-kafka-manual-commit-tests" - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + val producer = KafkaProducerSink[Integer, Integer](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](consumerCfg, List(topicName)) val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .range(0, totalRecords) + .map(id => new ProducerRecord(topicName, Integer.valueOf(id.toInt), Integer.valueOf(id.toInt))) .bufferIntrospective(1024) .consumeWith(producer) val listT = consumer .executeOn(io) - .bufferTumbling(count) - .map { messages => - messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) + .bufferTumbling(totalRecords) + .mapEvalF { committableMessages => + CommittableOffsetBatch(committableMessages.map(_.committableOffset)).commitAsync() + .as(committableMessages) } - .mapEval { case (values, batch) => Task.shift *> batch.commitAsync().map(_ => values -> batch.offsets) } .headL - val ((result, offsets), _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) + val (commitableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) - assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + assert(commitableMessages.map(_.record.value().toInt).sum === (0 until totalRecords).sum) + assert(commitableMessages.map(_.committableOffset.offset).last === totalRecords) } } From 1c8c5c1f5ce40897dfeeeeef6eab5ade8179a3fa Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 27 Sep 2020 12:02:31 +0200 Subject: [PATCH 18/61] Monix Kafka benchmarks for consumer, single and sink producer Block Fix Binded connection First benchmark for kafka producer Added kafka benchmark strategy plan Added sink and consumer benchmarks Producer results Akka Removed references to akka a Final --- benchmarks/docker-compose.yml | 39 ++++ benchmarks/project/build.properties | 1 + benchmarks/readme.md | 35 +++ benchmarks/results/consumer.txt | 208 ++++++++++++++++++ benchmarks/results/producer.txt | 75 +++++++ benchmarks/results/sink.txt | 141 ++++++++++++ benchmarks/src/main/resources/commands | 10 + benchmarks/src/main/resources/log4j2.xml | 16 ++ .../monix/kafka/benchmarks/BaseFixture.scala | 30 +++ .../kafka/benchmarks/ConsumerBenchmark.scala | 85 +++++++ .../monix/kafka/benchmarks/MonixFixture.scala | 54 +++++ .../kafka/benchmarks/ProducerBenchmark.scala | 43 ++++ .../benchmarks/ProducerSinkBenchmark.scala | 51 +++++ .../scala/monix/kafka/benchmarks/kafka.scala | 27 +++ .../test/scala/benchmarks/ConsumerSpec.scala | 38 ++++ .../test/scala/benchmarks/ProducerSpec.scala | 17 ++ benchmarks/start-kafka-cluster.sh | 26 +++ build.sbt | 21 +- project/plugins.sbt | 1 + 19 files changed, 917 insertions(+), 1 deletion(-) create mode 100644 benchmarks/docker-compose.yml create mode 100644 benchmarks/project/build.properties create mode 100644 benchmarks/readme.md create mode 100644 benchmarks/results/consumer.txt create mode 100644 benchmarks/results/producer.txt create mode 100644 benchmarks/results/sink.txt create mode 100644 benchmarks/src/main/resources/commands create mode 100644 benchmarks/src/main/resources/log4j2.xml create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala create mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala create mode 100644 benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala create mode 100644 benchmarks/src/test/scala/benchmarks/ProducerSpec.scala create mode 100644 benchmarks/start-kafka-cluster.sh diff --git a/benchmarks/docker-compose.yml b/benchmarks/docker-compose.yml new file mode 100644 index 00000000..de7d1c73 --- /dev/null +++ b/benchmarks/docker-compose.yml @@ -0,0 +1,39 @@ +--- +version: '2' +services: + + zookeeper: + image: confluentinc/cp-zookeeper:4.1.1 + hostname: zookeeper + container_name: zookeeper + ports: + - "2181:2181" + environment: + ZOOKEEPER_CLIENT_PORT: 2181 + ZOOKEEPER_TICK_TIME: 2000 + + broker: + image: confluentinc/cp-server:5.5.0 + hostname: broker + container_name: broker + depends_on: + - zookeeper + ports: + - "9092:9092" + environment: + KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper:2181' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://broker:29092,PLAINTEXT_HOST://localhost:9092 + KAFKA_METRIC_REPORTERS: io.confluent.metrics.reporter.ConfluentMetricsReporter + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 + KAFKA_CONFLUENT_LICENSE_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + CONFLUENT_METRICS_REPORTER_BOOTSTRAP_SERVERS: broker:29092 + CONFLUENT_METRICS_REPORTER_ZOOKEEPER_CONNECT: zookeeper:2181 + CONFLUENT_METRICS_REPORTER_TOPIC_REPLICAS: 1 + CONFLUENT_METRICS_ENABLE: 'true' + CONFLUENT_SUPPORT_CUSTOMER_ID: 'anonymous' + diff --git a/benchmarks/project/build.properties b/benchmarks/project/build.properties new file mode 100644 index 00000000..6adcdc75 --- /dev/null +++ b/benchmarks/project/build.properties @@ -0,0 +1 @@ +sbt.version=1.3.3 diff --git a/benchmarks/readme.md b/benchmarks/readme.md new file mode 100644 index 00000000..5bc318f6 --- /dev/null +++ b/benchmarks/readme.md @@ -0,0 +1,35 @@ +###Monix Kafka Benchmarks + +This document explains the approach followed to benchmark monix-kafka. + +Although it would be nice to summit monix-kafka onto soak, stress or load tests, we have hardware limitations that makes no possible to run those. + +Therefore we will stick to run performance testing on the different producer and consumer implementations that monix-kafka is exposing. + +Being `KafkaProducer`, `KafkaProducerSink` and `KafkaConsumerObservable` (AutoCommit and Committable) and finally doing both producer and consumer. + +Kafka is a distributed publisher/subscriber paradigm that have lots of configurables whose can directly impact on the performance of the + application running kafka (such as the number of partitions, replication factor, poll interval, records, buffer sizes and more...). + + For these performance tests we won't focus that much on these specific kafka configurations since these will run inside + a local virtual environment, which would not really reflect the real latency spent on the communication within the kafka cluster (network latency, lost packages, failures...), thus, we + do focus more on specific monix configuration (parallelism, async vs sync and commit order). + +Single `KafkaProducer` and `KafkaProducerSink` benchmark (they are different benchmarks but configuration is shared): + + | _Parallelism / Partitions, Replication Factor_ | __1P__, __1RF__ | __1P__, __2RF__ | __2P__, __2RF__ | + | :---: | :---: | :---: | :---: | + | __1__| ...| ... | ... | + | __2__| ... | ...| ... | + | __5__ | ... | ... | ... | + | __10__ | ... | ... | ... | + +Single `KafkaConsumer` (Auto commit and Manual commit) (consider including commit order) + + | _Type / Partitions, Replication Factor_ | __1P__, __1RF__ | __1P__, __2RF__ | __2P__, __2RF__ | + | :---: | :---: | :---: | :---: | + | __async__| ...| ... | ... | + | __sync__| ... | ...| ... | + +Pair `KafkaProducerSink` and `KafkaConsumer`: This test will consist in an observable being consumed by a kafka producer sink that produces +to an specific topic and then it gets consumed by a kafka consumer one. \ No newline at end of file diff --git a/benchmarks/results/consumer.txt b/benchmarks/results/consumer.txt new file mode 100644 index 00000000..5f5589fc --- /dev/null +++ b/benchmarks/results/consumer.txt @@ -0,0 +1,208 @@ + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_1P_1RF + +# Run progress: 2.78% complete, ETA 00:05:06 +# Fork: 1 of 1 +# Warmup Iteration 1: 39.905 ops/s +Iteration 1: 57.642 ops/s +Iteration 2: 57.671 ops/s +Iteration 3: 44.793 ops/s +Iteration 4: 64.277 ops/s +Iteration 5: 63.957 ops/s +Iteration 6: 43.770 ops/s +Iteration 7: 56.371 ops/s +Iteration 8: 62.515 ops/s +Iteration 9: 49.956 ops/s +Iteration 10: 31.907 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_1P_1RF": + 53.286 ±(99.9%) 15.924 ops/s [Average] + (min, avg, max) = (31.907, 53.286, 64.277), stdev = 10.533 + CI (99.9%): [37.362, 69.210] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_2P_1RF + +# Run progress: 18.06% complete, ETA 00:09:43 +# Fork: 1 of 1 +# Warmup Iteration 1: 44.691 ops/s +Iteration 1: 39.595 ops/s +Iteration 2: 51.434 ops/s +Iteration 3: 43.905 ops/s +Iteration 4: 26.324 ops/s +Iteration 5: 28.480 ops/s +Iteration 6: 45.607 ops/s +Iteration 7: 56.213 ops/s +Iteration 8: 56.639 ops/s +Iteration 9: 57.454 ops/s +Iteration 10: 54.110 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_2P_1RF": + 45.976 ±(99.9%) 17.338 ops/s [Average] + (min, avg, max) = (26.324, 45.976, 57.454), stdev = 11.468 + CI (99.9%): [28.638, 63.314] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_1P_1RF + +# Run progress: 33.33% complete, ETA 00:08:16 +# Fork: 1 of 1 +# Warmup Iteration 1: 34.182 ops/s +Iteration 1: 48.052 ops/s +Iteration 2: 47.124 ops/s +Iteration 3: 44.219 ops/s +Iteration 4: 58.554 ops/s +Iteration 5: 66.543 ops/s +Iteration 6: 60.585 ops/s +Iteration 7: 46.291 ops/s +Iteration 8: 41.104 ops/s +Iteration 9: 57.524 ops/s +Iteration 10: 52.359 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_1P_1RF": + 52.235 ±(99.9%) 12.441 ops/s [Average] + (min, avg, max) = (41.104, 52.235, 66.543), stdev = 8.229 + CI (99.9%): [39.794, 64.677] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_2P_1RF + +# Run progress: 48.61% complete, ETA 00:06:28 +# Fork: 1 of 1 +# Warmup Iteration 1: 40.188 ops/s +Iteration 1: 48.015 ops/s +Iteration 2: 60.841 ops/s +Iteration 3: 41.521 ops/s +Iteration 4: 45.350 ops/s +Iteration 5: 47.954 ops/s +Iteration 6: 62.838 ops/s +Iteration 7: 49.424 ops/s +Iteration 8: 57.602 ops/s +Iteration 9: 45.920 ops/s +Iteration 10: 36.525 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_2P_1RF": + 49.599 ±(99.9%) 12.737 ops/s [Average] + (min, avg, max) = (36.525, 49.599, 62.838), stdev = 8.424 + CI (99.9%): [36.863, 62.336] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_1P_1RF + +# Run progress: 63.89% complete, ETA 00:04:35 +# Fork: 1 of 1 +# Warmup Iteration 1: 29.430 ops/s +Iteration 1: 46.280 ops/s +Iteration 2: 53.870 ops/s +Iteration 3: 34.727 ops/s +Iteration 4: 34.179 ops/s +Iteration 5: 37.058 ops/s +Iteration 6: 48.568 ops/s +Iteration 7: 32.917 ops/s +Iteration 8: 44.958 ops/s +Iteration 9: 35.192 ops/s +Iteration 10: 40.836 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_1P_1RF": + 40.858 ±(99.9%) 10.888 ops/s [Average] + (min, avg, max) = (32.917, 40.858, 53.870), stdev = 7.202 + CI (99.9%): [29.971, 51.746] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 4 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_2P_1RF + +# Run progress: 79.17% complete, ETA 00:02:41 +# Fork: 1 of 1 +# Warmup Iteration 1: 31.783 ops/s +Iteration 1: 40.728 ops/s +Iteration 2: 46.217 ops/s +Iteration 3: 32.231 ops/s +Iteration 4: 49.542 ops/s +Iteration 5: 43.252 ops/s +Iteration 6: 54.555 ops/s +Iteration 7: 54.511 ops/s +Iteration 8: 46.290 ops/s +Iteration 9: 40.890 ops/s +Iteration 10: 40.730 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_2P_1RF": + 44.895 ±(99.9%) 10.388 ops/s [Average] + (min, avg, max) = (32.231, 44.895, 54.555), stdev = 6.871 + CI (99.9%): [34.507, 55.283] (assumes normal distribution) + + +# Run complete. Total time: 00:12:33 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.auto_commit_async_1P_1RF thrpt 10 53.286 ± 15.924 ops/s +ConsumerBenchmark.auto_commit_async_2P_1RF thrpt 10 45.976 ± 17.338 ops/s +ConsumerBenchmark.auto_commit_sync_1P_1RF thrpt 10 52.235 ± 12.441 ops/s +ConsumerBenchmark.auto_commit_sync_2P_1RF thrpt 10 49.599 ± 12.737 ops/s +ConsumerBenchmark.manual_commit_1P_1RF thrpt 10 40.858 ± 10.888 ops/s +ConsumerBenchmark.manual_commit_2P_1RF thrpt 10 44.895 ± 10.388 ops/s diff --git a/benchmarks/results/producer.txt b/benchmarks/results/producer.txt new file mode 100644 index 00000000..96fcee1c --- /dev/null +++ b/benchmarks/results/producer.txt @@ -0,0 +1,75 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_1P_1RF + +# Run progress: 0.00% complete, ETA 00:03:40 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.736 ops/s +Iteration 1: 0.991 ops/s +Iteration 2: 0.958 ops/s +Iteration 3: 0.891 ops/s +Iteration 4: 0.914 ops/s +Iteration 5: 0.973 ops/s +Iteration 6: 1.012 ops/s +Iteration 7: 0.860 ops/s +Iteration 8: 0.905 ops/s +Iteration 9: 0.871 ops/s +Iteration 10: 1.006 ops/s + + +Result "monix.kafka.benchmarks.ProducerBenchmark.monix_1P_1RF": + 0.938 ±(99.9%) 0.086 ops/s [Average] + (min, avg, max) = (0.860, 0.938, 1.012), stdev = 0.057 + CI (99.9%): [0.852, 1.024] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_2P_1RF + +# Run progress: 50.00% complete, ETA 00:01:57 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.875 ops/s +Iteration 1: 1.027 ops/s +Iteration 2: 1.038 ops/s +Iteration 3: 0.993 ops/s +Iteration 4: 0.998 ops/s +Iteration 5: 1.018 ops/s +Iteration 6: 0.985 ops/s +Iteration 7: 1.028 ops/s +Iteration 8: 0.982 ops/s +Iteration 9: 1.010 ops/s +Iteration 10: 0.917 ops/s + + +Result "monix.kafka.benchmarks.ProducerBenchmark.monix_2P_1RF": + 1.000 ±(99.9%) 0.052 ops/s [Average] + (min, avg, max) = (0.917, 1.000, 1.038), stdev = 0.035 + CI (99.9%): [0.947, 1.052] (assumes normal distribution) + + +# Run complete. Total time: 00:03:54 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ProducerBenchmark.monix_1P_1RF thrpt 10 0.938 ± 0.086 ops/s +ProducerBenchmark.monix_2P_1RF thrpt 10 1.000 ± 0.052 ops/s diff --git a/benchmarks/results/sink.txt b/benchmarks/results/sink.txt new file mode 100644 index 00000000..ee965a2f --- /dev/null +++ b/benchmarks/results/sink.txt @@ -0,0 +1,141 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_1P_1RF + +# Run progress: 21.43% complete, ETA 00:00:08 +# Fork: 1 of 1 +# Warmup Iteration 1: 9.375 ops/s +Iteration 1: 13.510 ops/s +Iteration 2: 18.241 ops/s +Iteration 3: 19.142 ops/s +Iteration 4: 19.561 ops/s +Iteration 5: 19.037 ops/s +Iteration 6: 16.952 ops/s +Iteration 7: 19.231 ops/s +Iteration 8: 17.509 ops/s +Iteration 9: 17.984 ops/s +Iteration 10: 19.508 ops/s + + +Result "monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_1P_1RF": + 18.068 ±(99.9%) 2.766 ops/s [Average] + (min, avg, max) = (13.510, 18.068, 19.561), stdev = 1.829 + CI (99.9%): [15.302, 20.833] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_2P_1RF + +# Run progress: 41.07% complete, ETA 00:02:46 +# Fork: 1 of 1 +# Warmup Iteration 1: 13.733 ops/s +Iteration 1: 19.880 ops/s +Iteration 2: 20.386 ops/s +Iteration 3: 18.965 ops/s +Iteration 4: 19.587 ops/s +Iteration 5: 16.391 ops/s +Iteration 6: 13.518 ops/s +Iteration 7: 16.592 ops/s +Iteration 8: 17.053 ops/s +Iteration 9: 17.305 ops/s +Iteration 10: 13.637 ops/s + + +Result "monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_2P_1RF": + 17.331 ±(99.9%) 3.679 ops/s [Average] + (min, avg, max) = (13.518, 17.331, 20.386), stdev = 2.433 + CI (99.9%): [13.653, 21.010] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.sync_2P_1RF + +# Run progress: 60.71% complete, ETA 00:02:28 +# Fork: 1 of 1 +# Warmup Iteration 1: 10.699 ops/s +Iteration 1: 15.467 ops/s +Iteration 2: 15.411 ops/s +Iteration 3: 16.970 ops/s +Iteration 4: 17.399 ops/s +Iteration 5: 16.215 ops/s +Iteration 6: 18.005 ops/s +Iteration 7: 19.683 ops/s +Iteration 8: 17.790 ops/s +Iteration 9: 21.806 ops/s +Iteration 10: 14.330 ops/s + + +Result "monix.kafka.benchmarks.ProducerSinkBenchmark.sync_2P_1RF": + 17.308 ±(99.9%) 3.334 ops/s [Average] + (min, avg, max) = (14.330, 17.308, 21.806), stdev = 2.205 + CI (99.9%): [13.974, 20.641] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.sync_sync_1P_1RF + +# Run progress: 80.36% complete, ETA 00:01:24 +# Fork: 1 of 1 +# Warmup Iteration 1: 11.903 ops/s +Iteration 1: 17.489 ops/s +Iteration 2: 16.606 ops/s +Iteration 3: 13.364 ops/s +Iteration 4: 14.373 ops/s +Iteration 5: 11.613 ops/s +Iteration 6: 15.312 ops/s +Iteration 7: 18.608 ops/s +Iteration 8: 17.567 ops/s +Iteration 9: 10.345 ops/s +Iteration 10: 15.375 ops/s + + +Result "monix.kafka.benchmarks.ProducerSinkBenchmark.sync_sync_1P_1RF": + 15.065 ±(99.9%) 4.056 ops/s [Average] + (min, avg, max) = (10.345, 15.065, 18.608), stdev = 2.683 + CI (99.9%): [11.009, 19.122] (assumes normal distribution) + + +# Run complete. Total time: 00:07:40 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ProducerSinkBenchmark.parallel_1P_1RF thrpt 10 18.068 ± 2.766 ops/s +ProducerSinkBenchmark.parallel_2P_1RF thrpt 10 17.331 ± 3.679 ops/s +ProducerSinkBenchmark.sync_2P_1RF thrpt 10 17.308 ± 3.334 ops/s +ProducerSinkBenchmark.sync_sync_1P_1RF thrpt 10 15.065 ± 4.056 ops/s diff --git a/benchmarks/src/main/resources/commands b/benchmarks/src/main/resources/commands new file mode 100644 index 00000000..8f7c15fc --- /dev/null +++ b/benchmarks/src/main/resources/commands @@ -0,0 +1,10 @@ +kafka-console-consumer --bootstrap-server broker:9092 --topic topic2 --from-beginning +kafka-console-producer --broker-list broker:9092 --topic topic_1P_1RF + +docker exec -it broker kafka-console-consumer --bootstrap-server broker:9092 --topic topic2 --from-beginning +docker exec -it broker /bin/bash + +docker stop $(docker ps -a -q) + +#from root directory +sbt 'benchmarks/jmh:run -o results.txt monix.kafka.benchmarks.KafkaProducerBenchmark' \ No newline at end of file diff --git a/benchmarks/src/main/resources/log4j2.xml b/benchmarks/src/main/resources/log4j2.xml new file mode 100644 index 00000000..c3b6c46a --- /dev/null +++ b/benchmarks/src/main/resources/log4j2.xml @@ -0,0 +1,16 @@ + + + + %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n + + + + + + + + + + + + \ No newline at end of file diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala new file mode 100644 index 00000000..ca3c473f --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala @@ -0,0 +1,30 @@ +package monix.kafka.benchmarks + +import monix.eval.Coeval + +import scala.util.Random + +trait BaseFixture { + + val producerTestId = "producer" + val sinkTestId = "sink" + val consumerTestId = "consumer" + val brokerUrl = "127.0.0.1:9092" + + val randomId: Coeval[String] = Coeval(Random.alphanumeric.filter(_.isLetter).take(20).mkString) + def getTopicNames(testId: String): (String, String) = { + val topic: (String, Int, Int) => String = { + (testId, partitions: Int, replicationFactor: Int) => + //syntax (P, RF) === (Partitions, Replication Factor) + s"topic_${testId}_${partitions}P_${replicationFactor}RF" + } + (topic(testId, 1, 1), topic(testId, 1, 2)) + } + + //monix-kafka benchmarks + val (topic_producer_1P_1RF, topic_producer_2P_1RF) = getTopicNames(producerTestId) + val (topic_sink_1P_1RF, topic_sink_2P_1RF) = getTopicNames(sinkTestId) + val (topic_consumer_1P_1RF, topic_consumer_2P_1RF) = getTopicNames(consumerTestId) + + +} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala new file mode 100644 index 00000000..ccc32cd3 --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -0,0 +1,85 @@ +package monix.kafka.benchmarks + +import java.util.concurrent.TimeUnit + +import monix.eval.Task +import monix.kafka.KafkaConsumerObservable +import monix.kafka.config.ObservableCommitType +import org.openjdk.jmh.annotations.{ + BenchmarkMode, + Fork, + Measurement, + Mode, + OutputTimeUnit, + Scope, + State, + Threads, + Warmup, + _ +} + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +@State(Scope.Thread) +@BenchmarkMode(Array(Mode.Throughput)) +@OutputTimeUnit(TimeUnit.SECONDS) +@Measurement(iterations = 10) +@Warmup(iterations = 1) +@Fork(1) +@Threads(4) +class ConsumerBenchmark extends MonixFixture { + + var size: Int = 1000 + var maxPool: Int = 5 + + // preparing test data + val t1 = produceGroupedSink(topic_consumer_1P_1RF, size * 2, 10, 1) + val t2 = produceGroupedSink(topic_consumer_2P_1RF, size * 2, 10, 1) + val f3 = Task.gather(List(t1, t2)).runToFuture(io) + val _ = Await.ready(f3, Duration.Inf) + + //syntax (P, P, RF) === (Parallelism factor, Partitions, Replication Factor) + @Benchmark + def manual_commit_1P_1RF(): Unit = { + val f = consumeManualCommit(topic_consumer_1P_1RF, size, maxPool).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + + @Benchmark + def manual_commit_2P_1RF(): Unit = { + val f = consumeManualCommit(topic_consumer_1P_1RF, size, maxPool).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + + @Benchmark + def auto_commit_async_1P_1RF(): Unit = { + val f = consumeAutoCommit(topic_consumer_1P_1RF, size, maxPool, ObservableCommitType.Async).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + + @Benchmark + def auto_commit_sync_1P_1RF(): Unit = { + val f = consumeAutoCommit(topic_consumer_1P_1RF, size, maxPool, ObservableCommitType.Sync).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + + @Benchmark + def auto_commit_async_2P_1RF(): Unit = { + val f = consumeAutoCommit(topic_consumer_2P_1RF, size, maxPool, ObservableCommitType.Async).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + + @Benchmark + def auto_commit_sync_2P_1RF(): Unit = { + val f = consumeAutoCommit(topic_consumer_2P_1RF, size, maxPool, ObservableCommitType.Sync).runToFuture(io) + Await.result(f, Duration.Inf) + f.cancel() + } + +} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala new file mode 100644 index 00000000..bdbc9d87 --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala @@ -0,0 +1,54 @@ +package monix.kafka.benchmarks + +import monix.eval.{Coeval, Task} +import monix.kafka.config.{AutoOffsetReset, ObservableCommitType} +import monix.kafka.{CommittableMessage, KafkaConsumerConfig, KafkaConsumerObservable, KafkaProducerConfig, KafkaProducerSink} +import monix.reactive.Observable +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerRecord + +trait MonixFixture extends BaseFixture{ + + val producerConf = KafkaProducerConfig.default.copy( + bootstrapServers = List(brokerUrl), + clientId = topic_producer_1P_1RF, + ) + + val consumerConf = Coeval(KafkaConsumerConfig.default.copy( + bootstrapServers = List(brokerUrl), + clientId = topic_consumer_1P_1RF, + groupId = randomId.value(), + enableAutoCommit = false, + autoOffsetReset = AutoOffsetReset.Earliest + )) + + + def produceGroupedSink(topic: String, size: Int, bufferSize: Int, parallelism: Int): Task[Unit] = { + Observable + .from(1 to size) + .map(i => new ProducerRecord[Integer, Integer](topic, i)) + .bufferTumbling(bufferSize) + .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = parallelism), io)) + } + + + def consumeManualCommit(topic: String, size: Int, maxPollRecords: Int): Task[CommittableMessage[Integer, Integer]] = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(topic)) + .take(size - 1) + .headL + } + + def consumeAutoCommit(topic: String, consumeRecords: Int, maxPollRecords: Int, commitType: ObservableCommitType): Task[ConsumerRecord[Integer, Integer]] = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + KafkaConsumerObservable[Integer, Integer](conf, List(topic)) + .take(consumeRecords - 1) + .headL + } + + + + + + +} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala new file mode 100644 index 00000000..88615ad3 --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala @@ -0,0 +1,43 @@ +package monix.kafka.benchmarks + +import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup} +import java.util.concurrent.TimeUnit + +import monix.eval.Task +import monix.kafka.KafkaProducer +import org.apache.kafka.clients.producer.RecordMetadata +import org.openjdk.jmh.annotations._ + +import scala.concurrent.duration.Duration +import scala.concurrent.Await + +@State(Scope.Thread) +@BenchmarkMode(Array(Mode.Throughput)) +@OutputTimeUnit(TimeUnit.SECONDS) +@Measurement(iterations = 10) +@Warmup(iterations = 1) +@Fork(1) +@Threads(1) +class ProducerBenchmark extends MonixFixture { + + var size: Int = 1000 + + val producer = KafkaProducer[Integer, Integer](producerConf, singleThread) + + def produceOneByOne(topic: String): Task[List[Option[RecordMetadata]]] = + Task.traverse((0 until size).toList)(i => producer.send(topic = topic_producer_1P_1RF, i)) + + //syntax (P, P, RF) === (Parallelism factor, Partitions, Replication Factor) + @Benchmark + def monix_1P_1RF(): Unit = { + val f = produceOneByOne(topic_producer_1P_1RF).runToFuture(singleThread) + Await.ready(f, Duration.Inf) + } + + @Benchmark + def monix_2P_1RF(): Unit = { + val f = produceOneByOne(topic_producer_2P_1RF).runToFuture(singleThread) + Await.ready(f, Duration.Inf) + } + +} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala new file mode 100644 index 00000000..8920e469 --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala @@ -0,0 +1,51 @@ +package monix.kafka.benchmarks + +import java.util.concurrent.TimeUnit + +import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +@State(Scope.Thread) +@BenchmarkMode(Array(Mode.Throughput)) +@OutputTimeUnit(TimeUnit.SECONDS) +@Measurement(iterations = 10) +@Warmup(iterations = 1) +@Fork(1) +@Threads(3) +class ProducerSinkBenchmark extends MonixFixture { + + var size: Int = 100 + + @Benchmark + def parallel_1P_1RF(): Unit = { + val f = produceGroupedSink(topic_sink_1P_1RF, size, 1, 100).runToFuture(io) + Await.ready(f, Duration.Inf) + } + + + @Benchmark + def parallel_2P_1RF(): Unit = { + val f = produceGroupedSink(topic_sink_2P_1RF, size, 1, 100).runToFuture(io) + Await.ready(f, Duration.Inf) + } + + + @Benchmark + def sync_sync_1P_1RF(): Unit = { + val f = produceGroupedSink(topic_sink_1P_1RF, size, 1, 1).runToFuture(singleThread) + Await.ready(f, Duration.Inf) + } + + + @Benchmark + def sync_2P_1RF(): Unit = { + val f = produceGroupedSink(topic_sink_2P_1RF, size, 1, 1).runToFuture(singleThread) + Await.ready(f, Duration.Inf) + } + + + + +} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala new file mode 100644 index 00000000..fcb0b77a --- /dev/null +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2014-2019 by its authors. Some rights reserved. + * See the project homepage at: https://github.com/monix/monix-kafka + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package monix.kafka + +import monix.execution.Scheduler + +package object benchmarks { + + /** I/O scheduler meant for tests. */ + lazy val io = Scheduler.io("kafka-io") + lazy val singleThread = Scheduler.singleThread("single-thread") +} diff --git a/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala new file mode 100644 index 00000000..b326eb25 --- /dev/null +++ b/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala @@ -0,0 +1,38 @@ +package benchmarks + +import monix.execution.Scheduler +import monix.execution.Scheduler.Implicits.global +import monix.kafka.{KafkaConsumerObservable, KafkaProducer} +import monix.kafka.benchmarks.MonixFixture +import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} + +import scala.concurrent.Await +import scala.concurrent.duration._ + +class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeAndAfterAll { + + + implicit lazy val io = Scheduler.io("monix-kafka-benchmark") + + override def beforeAll(): Unit = { + super.beforeAll() + produceGroupedSink(topic_consumer_1P_1RF, 1000, 1, 1).runSyncUnsafe() + } + + s"Monix ${topic_consumer_1P_1RF}" should "exist" in { + val producer = KafkaProducer[String, String](producerConf, global) + val t = producer.send(topic = topic_consumer_1P_1RF, "test") + + t.runSyncUnsafe().isDefined shouldBe true + } + + it should "allow " in { + val conf = consumerConf.value() + + val f2 = KafkaConsumerObservable[Integer, Integer](conf, List(topic_consumer_1P_1RF)).take(1000).toListL.runToFuture(io) + + val elements = Await.result(f2, 10.seconds) + elements.size shouldBe 1000 + } + +} diff --git a/benchmarks/src/test/scala/benchmarks/ProducerSpec.scala b/benchmarks/src/test/scala/benchmarks/ProducerSpec.scala new file mode 100644 index 00000000..de685581 --- /dev/null +++ b/benchmarks/src/test/scala/benchmarks/ProducerSpec.scala @@ -0,0 +1,17 @@ +package monix.kafka.benchmarks + +import monix.kafka.KafkaProducer +import monix.execution.Scheduler.Implicits.global +import org.scalatest.{FlatSpec, Matchers} + +class ProducerSpec extends FlatSpec with MonixFixture with Matchers { + + val producer = KafkaProducer[String, String](producerConf, global) + + s"Monix ${topic_producer_1P_1RF}" should "exist befor running Producer Benchmark" in { + val t = producer.send(topic = topic_producer_1P_1RF, "test") + + t.runSyncUnsafe().isDefined shouldBe true + } + +} diff --git a/benchmarks/start-kafka-cluster.sh b/benchmarks/start-kafka-cluster.sh new file mode 100644 index 00000000..fdd166d0 --- /dev/null +++ b/benchmarks/start-kafka-cluster.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +set -e + +function create_topic { + TOPIC_NAME=$1 + PARTITIONS=$2 + REPLICATION_FACTOR=$3 + echo "Creating topic ${TOPIC_NAME} with ${PARTITIONS} partitions and replication factor of ${REPLICATION_FACTOR}." + docker-compose -f ./docker-compose.yml exec -T broker kafka-topics --create --topic ${TOPIC_NAME} --partitions ${PARTITIONS} --replication-factor ${REPLICATION_FACTOR} --if-not-exists --zookeeper zookeeper:2181 +} + +echo "Starting Kafka cluster..." +docker-compose -f ./docker-compose.yml up -d zookeeper broker + +echo -e "Docker ps..." +docker ps + +sleep 15 + +create_topic topic_producer_1P_1RF 1 1 +create_topic topic_producer_2P_1RF 2 1 +create_topic topic_sink_1P_1RF 1 1 +create_topic topic_sink_2P_1RF 2 1 +create_topic topic_consumer_1P_1RF 1 1 +create_topic topic_consumer_2P_1RF 2 1 \ No newline at end of file diff --git a/build.sbt b/build.sbt index bb2c08be..03d25df7 100644 --- a/build.sbt +++ b/build.sbt @@ -1,3 +1,5 @@ +import pl.project13.scala.sbt.JmhPlugin + val monixVersion = "3.3.0" val allProjects = List( @@ -213,7 +215,7 @@ lazy val kafka1x = project.in(file("kafka-1.0.x")) .settings( name := "monix-kafka-1x", libraryDependencies ++= { - if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.0.0" % "test" exclude ("log4j", "log4j")) + if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.0" % "test" exclude ("log4j", "log4j")) else Seq.empty[ModuleID] }, libraryDependencies += "org.apache.kafka" % "kafka-clients" % "1.0.2" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") @@ -256,6 +258,23 @@ lazy val kafka9 = project.in(file("kafka-0.9.x")) ) ) +lazy val benchmarks = project.in(file("benchmarks")) + .settings(sharedSettings) + .settings(commonDependencies) + .settings( + name := "benchmarks", + organization := "io.monix", + scalaVersion := "2.12.10", + libraryDependencies ++= Seq( + //"com.typesafe.akka" %% "akka-stream-kafka" % "1.1.0", + //"com.typesafe.akka" %% "akka-stream" % "2.5.25" + ) + ) + .enablePlugins(JmhPlugin) + .aggregate(kafka1x) + .dependsOn(kafka1x) + + //------------- For Release enablePlugins(GitVersioning) diff --git a/project/plugins.sbt b/project/plugins.sbt index 6b819893..bed5235b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -5,3 +5,4 @@ addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "1.0.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.8.1") addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.6.1") addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.4.2") +addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.7") \ No newline at end of file From 0bf0bff2b4ebf1cf344745eb4bdc8dab748e6467 Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 25 Jan 2021 00:52:26 +0100 Subject: [PATCH 19/61] Benchmarks --- benchmarks/producer_results.txt | 61 +++++++++++++ benchmarks/readme.md | 44 +++++----- benchmarks/results.txt | 53 ++++++++++++ benchmarks/src/main/resources/commands | 4 +- .../monix/kafka/benchmarks/BaseFixture.scala | 21 ++--- .../kafka/benchmarks/ConsumerBenchmark.scala | 85 ++++++------------- .../monix/kafka/benchmarks/MonixFixture.scala | 50 +++-------- .../kafka/benchmarks/ProducerBenchmark.scala | 37 ++++---- .../benchmarks/ProducerSinkBenchmark.scala | 51 ----------- .../scala/monix/kafka/benchmarks/kafka.scala | 27 ------ .../test/scala/benchmarks/ConsumerSpec.scala | 38 --------- .../monix/kafka/benchmarks/ConsumerSpec.scala | 62 ++++++++++++++ .../kafka}/benchmarks/ProducerSpec.scala | 4 +- benchmarks/start-kafka-cluster.sh | 10 +-- build.sbt | 4 +- 15 files changed, 271 insertions(+), 280 deletions(-) create mode 100644 benchmarks/producer_results.txt create mode 100644 benchmarks/results.txt delete mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala delete mode 100644 benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala delete mode 100644 benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala create mode 100644 benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala rename benchmarks/src/test/scala/{ => monix/kafka}/benchmarks/ProducerSpec.scala (98%) diff --git a/benchmarks/producer_results.txt b/benchmarks/producer_results.txt new file mode 100644 index 00000000..5975d682 --- /dev/null +++ b/benchmarks/producer_results.txt @@ -0,0 +1,61 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 3 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_single_producer + +# Run progress: 0.00% complete, ETA 00:01:20 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.441 ops/s +Iteration 1: 0.479 ops/s +Iteration 2: 0.546 ops/s +Iteration 3: 0.487 ops/s + + +Result "monix.kafka.benchmarks.ProducerBenchmark.monix_single_producer": + 0.504 ±(99.9%) 0.668 ops/s [Average] + (min, avg, max) = (0.479, 0.504, 0.546), stdev = 0.037 + CI (99.9%): [≈ 0, 1.172] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 3 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_sink_producer + +# Run progress: 50.00% complete, ETA 00:00:48 +# Fork: 1 of 1 +# Warmup Iteration 1: 1.608 ops/s +Iteration 1: 2.134 ops/s +Iteration 2: 1.908 ops/s +Iteration 3: 1.540 ops/s + + +Result "monix.kafka.benchmarks.ProducerBenchmark.monix_sink_producer": + 1.861 ±(99.9%) 5.475 ops/s [Average] + (min, avg, max) = (1.540, 1.861, 2.134), stdev = 0.300 + CI (99.9%): [≈ 0, 7.336] (assumes normal distribution) + + +# Run complete. Total time: 00:01:31 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ProducerBenchmark.monix_single_producer thrpt 3 0.504 ± 0.668 ops/s +ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s diff --git a/benchmarks/readme.md b/benchmarks/readme.md index 5bc318f6..57f62f99 100644 --- a/benchmarks/readme.md +++ b/benchmarks/readme.md @@ -2,34 +2,28 @@ This document explains the approach followed to benchmark monix-kafka. -Although it would be nice to summit monix-kafka onto soak, stress or load tests, we have hardware limitations that makes no possible to run those. + Ideally, a Kafka performance benchmark should happen under some long stress test in a real Kafka cluster, +although, our hardware limitations we have to stick to running simpler basic tests that proves application performance on + a docker container. -Therefore we will stick to run performance testing on the different producer and consumer implementations that monix-kafka is exposing. -Being `KafkaProducer`, `KafkaProducerSink` and `KafkaConsumerObservable` (AutoCommit and Committable) and finally doing both producer and consumer. -Kafka is a distributed publisher/subscriber paradigm that have lots of configurables whose can directly impact on the performance of the - application running kafka (such as the number of partitions, replication factor, poll interval, records, buffer sizes and more...). - - For these performance tests we won't focus that much on these specific kafka configurations since these will run inside - a local virtual environment, which would not really reflect the real latency spent on the communication within the kafka cluster (network latency, lost packages, failures...), thus, we - do focus more on specific monix configuration (parallelism, async vs sync and commit order). - -Single `KafkaProducer` and `KafkaProducerSink` benchmark (they are different benchmarks but configuration is shared): - - | _Parallelism / Partitions, Replication Factor_ | __1P__, __1RF__ | __1P__, __2RF__ | __2P__, __2RF__ | - | :---: | :---: | :---: | :---: | - | __1__| ...| ... | ... | - | __2__| ... | ...| ... | - | __5__ | ... | ... | ... | - | __10__ | ... | ... | ... | +So these test will be executed using the same configurations () + +The benchmark will focus on the most basic `consumer` and `procucer` scenarios. + +Although Kafka is very configurable by nature, our benchmark will use the kafka default properties for [consumer](https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html) +and [producer](https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html). + +On the other hand, assume that all the used topics will have 2 partitions, and 1 replication factor. -Single `KafkaConsumer` (Auto commit and Manual commit) (consider including commit order) - | _Type / Partitions, Replication Factor_ | __1P__, __1RF__ | __1P__, __2RF__ | __2P__, __2RF__ | - | :---: | :---: | :---: | :---: | - | __async__| ...| ... | ... | - | __sync__| ... | ...| ... | +## Producer benchmarks + +This section includes benchmarks for single and sink producers. +Although some libraries like `alpakka-kafka` do not expose methods for producing single record, but only for sink. + -Pair `KafkaProducerSink` and `KafkaConsumer`: This test will consist in an observable being consumed by a kafka producer sink that produces -to an specific topic and then it gets consumed by a kafka consumer one. \ No newline at end of file +## Consumer benchmark +The consumer benchmark covers the manual and auto commit consumer implementations of the different libraries. +The manual commit will also cover producing committing back the consumed offsets. \ No newline at end of file diff --git a/benchmarks/results.txt b/benchmarks/results.txt new file mode 100644 index 00000000..366f6d34 --- /dev/null +++ b/benchmarks/results.txt @@ -0,0 +1,53 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 1 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit + +# Run progress: 0.00% complete, ETA 00:00:40 +# Fork: 1 of 1 +# Warmup Iteration 1: 545234.343 ops/s +Iteration 1: 605656.170 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": + 605656.170 ops/s + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 1 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit + +# Run progress: 50.00% complete, ETA 00:00:23 +# Fork: 1 of 1 +# Warmup Iteration 1: 578595.083 ops/s +Iteration 1: 578552.396 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit": + 578552.396 ops/s + + +# Run complete. Total time: 00:00:46 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 605656.170 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 578552.396 ops/s diff --git a/benchmarks/src/main/resources/commands b/benchmarks/src/main/resources/commands index 8f7c15fc..ab6e54e3 100644 --- a/benchmarks/src/main/resources/commands +++ b/benchmarks/src/main/resources/commands @@ -7,4 +7,6 @@ docker exec -it broker /bin/bash docker stop $(docker ps -a -q) #from root directory -sbt 'benchmarks/jmh:run -o results.txt monix.kafka.benchmarks.KafkaProducerBenchmark' \ No newline at end of file +sbt 'benchmarks/jmh:run -o producer_results.txt monix.kafka.benchmarks.ProducerBenchmark' + + sbt 'benchmarks/jmh:run -o consumer_results.txt monix.kafka.benchmarks.ConsumerBenchmark' \ No newline at end of file diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala index ca3c473f..a0ff4305 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala @@ -6,25 +6,14 @@ import scala.util.Random trait BaseFixture { - val producerTestId = "producer" - val sinkTestId = "sink" - val consumerTestId = "consumer" val brokerUrl = "127.0.0.1:9092" - val randomId: Coeval[String] = Coeval(Random.alphanumeric.filter(_.isLetter).take(20).mkString) - def getTopicNames(testId: String): (String, String) = { - val topic: (String, Int, Int) => String = { - (testId, partitions: Int, replicationFactor: Int) => - //syntax (P, RF) === (Partitions, Replication Factor) - s"topic_${testId}_${partitions}P_${replicationFactor}RF" - } - (topic(testId, 1, 1), topic(testId, 1, 2)) - } - //monix-kafka benchmarks - val (topic_producer_1P_1RF, topic_producer_2P_1RF) = getTopicNames(producerTestId) - val (topic_sink_1P_1RF, topic_sink_2P_1RF) = getTopicNames(sinkTestId) - val (topic_consumer_1P_1RF, topic_consumer_2P_1RF) = getTopicNames(consumerTestId) + // topic names + val monixTopic = "monix_topic" + val akkaTopic = "akka_topic" + val fs2Topic = "fs2_topic" + val zioTopic = "zio_topic" } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index ccc32cd3..1769d935 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -1,85 +1,56 @@ package monix.kafka.benchmarks import java.util.concurrent.TimeUnit - -import monix.eval.Task -import monix.kafka.KafkaConsumerObservable +//import monix.execution.Scheduler +import monix.kafka.{KafkaConsumerObservable, KafkaProducerSink} import monix.kafka.config.ObservableCommitType -import org.openjdk.jmh.annotations.{ - BenchmarkMode, - Fork, - Measurement, - Mode, - OutputTimeUnit, - Scope, - State, - Threads, - Warmup, - _ -} +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} import scala.concurrent.Await import scala.concurrent.duration.Duration +import scala.concurrent.duration._ @State(Scope.Thread) @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) -@Measurement(iterations = 10) +@Measurement(iterations = 1) @Warmup(iterations = 1) @Fork(1) -@Threads(4) +@Threads(3) class ConsumerBenchmark extends MonixFixture { var size: Int = 1000 - var maxPool: Int = 5 + var maxPollRecords: Int = 5 // preparing test data - val t1 = produceGroupedSink(topic_consumer_1P_1RF, size * 2, 10, 1) - val t2 = produceGroupedSink(topic_consumer_2P_1RF, size * 2, 10, 1) - val f3 = Task.gather(List(t1, t2)).runToFuture(io) - val _ = Await.ready(f3, Duration.Inf) - - //syntax (P, P, RF) === (Parallelism factor, Partitions, Replication Factor) - @Benchmark - def manual_commit_1P_1RF(): Unit = { - val f = consumeManualCommit(topic_consumer_1P_1RF, size, maxPool).runToFuture(io) - Await.result(f, Duration.Inf) - f.cancel() - } - - @Benchmark - def manual_commit_2P_1RF(): Unit = { - val f = consumeManualCommit(topic_consumer_1P_1RF, size, maxPool).runToFuture(io) - Await.result(f, Duration.Inf) - f.cancel() - } - - @Benchmark - def auto_commit_async_1P_1RF(): Unit = { - val f = consumeAutoCommit(topic_consumer_1P_1RF, size, maxPool, ObservableCommitType.Async).runToFuture(io) - Await.result(f, Duration.Inf) - f.cancel() - } - - @Benchmark - def auto_commit_sync_1P_1RF(): Unit = { - val f = consumeAutoCommit(topic_consumer_1P_1RF, size, maxPool, ObservableCommitType.Sync).runToFuture(io) - Await.result(f, Duration.Inf) - f.cancel() - } + val f = Observable + .from(1 to size) + .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) + .bufferTumbling(size) + .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) + .runToFuture(io) + Await.ready(f, 10.seconds) @Benchmark - def auto_commit_async_2P_1RF(): Unit = { - val f = consumeAutoCommit(topic_consumer_2P_1RF, size, maxPool, ObservableCommitType.Async).runToFuture(io) + def monix_manual_commit(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .take(size) + .headL Await.result(f, Duration.Inf) - f.cancel() } @Benchmark - def auto_commit_sync_2P_1RF(): Unit = { - val f = consumeAutoCommit(topic_consumer_2P_1RF, size, maxPool, ObservableCommitType.Sync).runToFuture(io) + def monix_auto_commit(): Unit = { + val conf = consumerConf.value().copy( + maxPollRecords = maxPollRecords, + observableCommitType = ObservableCommitType.Async) + KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) + .take(size) + .headL Await.result(f, Duration.Inf) - f.cancel() } } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala index bdbc9d87..4a0047dc 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala @@ -1,54 +1,30 @@ package monix.kafka.benchmarks -import monix.eval.{Coeval, Task} -import monix.kafka.config.{AutoOffsetReset, ObservableCommitType} -import monix.kafka.{CommittableMessage, KafkaConsumerConfig, KafkaConsumerObservable, KafkaProducerConfig, KafkaProducerSink} -import monix.reactive.Observable -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.clients.producer.ProducerRecord +import monix.eval.Coeval +import monix.execution.Scheduler +import monix.kafka.config.AutoOffsetReset +import monix.kafka.{KafkaConsumerConfig, KafkaProducerConfig} +//import monix.reactive.Observable +//import org.apache.kafka.clients.consumer.ConsumerRecord +//import org.apache.kafka.clients.producer.ProducerRecord trait MonixFixture extends BaseFixture{ + implicit lazy val io = Scheduler.io("monix-kafka-benchmark") + val producerConf = KafkaProducerConfig.default.copy( bootstrapServers = List(brokerUrl), - clientId = topic_producer_1P_1RF, + clientId = monixTopic, ) + // we set a different group id every time so then the consumer will always + // read from the beginning val consumerConf = Coeval(KafkaConsumerConfig.default.copy( bootstrapServers = List(brokerUrl), - clientId = topic_consumer_1P_1RF, + clientId = monixTopic, groupId = randomId.value(), enableAutoCommit = false, autoOffsetReset = AutoOffsetReset.Earliest )) - - def produceGroupedSink(topic: String, size: Int, bufferSize: Int, parallelism: Int): Task[Unit] = { - Observable - .from(1 to size) - .map(i => new ProducerRecord[Integer, Integer](topic, i)) - .bufferTumbling(bufferSize) - .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = parallelism), io)) - } - - - def consumeManualCommit(topic: String, size: Int, maxPollRecords: Int): Task[CommittableMessage[Integer, Integer]] = { - val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(topic)) - .take(size - 1) - .headL - } - - def consumeAutoCommit(topic: String, consumeRecords: Int, maxPollRecords: Int, commitType: ObservableCommitType): Task[ConsumerRecord[Integer, Integer]] = { - val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - KafkaConsumerObservable[Integer, Integer](conf, List(topic)) - .take(consumeRecords - 1) - .headL - } - - - - - - } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala index 88615ad3..c29c05db 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerBenchmark.scala @@ -1,43 +1,44 @@ package monix.kafka.benchmarks import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup} -import java.util.concurrent.TimeUnit +import java.util.concurrent.TimeUnit import monix.eval.Task -import monix.kafka.KafkaProducer -import org.apache.kafka.clients.producer.RecordMetadata +import monix.kafka.{KafkaProducer, KafkaProducerSink} +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord import org.openjdk.jmh.annotations._ -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import scala.concurrent.Await @State(Scope.Thread) @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) -@Measurement(iterations = 10) +@Measurement(iterations = 3) @Warmup(iterations = 1) @Fork(1) @Threads(1) class ProducerBenchmark extends MonixFixture { var size: Int = 1000 + val producer = KafkaProducer[Integer, Integer](producerConf, io) - val producer = KafkaProducer[Integer, Integer](producerConf, singleThread) - - def produceOneByOne(topic: String): Task[List[Option[RecordMetadata]]] = - Task.traverse((0 until size).toList)(i => producer.send(topic = topic_producer_1P_1RF, i)) + @Benchmark + def monix_single_producer(): Unit = { + val f = Task.traverse((0 until size).toList)(i => producer.send(topic = monixTopic, i)).runToFuture(io) + Await.ready(f, Duration.Inf) + } - //syntax (P, P, RF) === (Parallelism factor, Partitions, Replication Factor) @Benchmark - def monix_1P_1RF(): Unit = { - val f = produceOneByOne(topic_producer_1P_1RF).runToFuture(singleThread) + def monix_sink_producer(): Unit = { + val f = Observable + .from(1 to size) + .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) + .bufferTumbling(50) + .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) + .runToFuture(io) Await.ready(f, Duration.Inf) } - @Benchmark - def monix_2P_1RF(): Unit = { - val f = produceOneByOne(topic_producer_2P_1RF).runToFuture(singleThread) - Await.ready(f, Duration.Inf) - } - } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala deleted file mode 100644 index 8920e469..00000000 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ProducerSinkBenchmark.scala +++ /dev/null @@ -1,51 +0,0 @@ -package monix.kafka.benchmarks - -import java.util.concurrent.TimeUnit - -import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} - -import scala.concurrent.Await -import scala.concurrent.duration.Duration - -@State(Scope.Thread) -@BenchmarkMode(Array(Mode.Throughput)) -@OutputTimeUnit(TimeUnit.SECONDS) -@Measurement(iterations = 10) -@Warmup(iterations = 1) -@Fork(1) -@Threads(3) -class ProducerSinkBenchmark extends MonixFixture { - - var size: Int = 100 - - @Benchmark - def parallel_1P_1RF(): Unit = { - val f = produceGroupedSink(topic_sink_1P_1RF, size, 1, 100).runToFuture(io) - Await.ready(f, Duration.Inf) - } - - - @Benchmark - def parallel_2P_1RF(): Unit = { - val f = produceGroupedSink(topic_sink_2P_1RF, size, 1, 100).runToFuture(io) - Await.ready(f, Duration.Inf) - } - - - @Benchmark - def sync_sync_1P_1RF(): Unit = { - val f = produceGroupedSink(topic_sink_1P_1RF, size, 1, 1).runToFuture(singleThread) - Await.ready(f, Duration.Inf) - } - - - @Benchmark - def sync_2P_1RF(): Unit = { - val f = produceGroupedSink(topic_sink_2P_1RF, size, 1, 1).runToFuture(singleThread) - Await.ready(f, Duration.Inf) - } - - - - -} diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala deleted file mode 100644 index fcb0b77a..00000000 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/kafka.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Copyright (c) 2014-2019 by its authors. Some rights reserved. - * See the project homepage at: https://github.com/monix/monix-kafka - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package monix.kafka - -import monix.execution.Scheduler - -package object benchmarks { - - /** I/O scheduler meant for tests. */ - lazy val io = Scheduler.io("kafka-io") - lazy val singleThread = Scheduler.singleThread("single-thread") -} diff --git a/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala deleted file mode 100644 index b326eb25..00000000 --- a/benchmarks/src/test/scala/benchmarks/ConsumerSpec.scala +++ /dev/null @@ -1,38 +0,0 @@ -package benchmarks - -import monix.execution.Scheduler -import monix.execution.Scheduler.Implicits.global -import monix.kafka.{KafkaConsumerObservable, KafkaProducer} -import monix.kafka.benchmarks.MonixFixture -import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} - -import scala.concurrent.Await -import scala.concurrent.duration._ - -class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeAndAfterAll { - - - implicit lazy val io = Scheduler.io("monix-kafka-benchmark") - - override def beforeAll(): Unit = { - super.beforeAll() - produceGroupedSink(topic_consumer_1P_1RF, 1000, 1, 1).runSyncUnsafe() - } - - s"Monix ${topic_consumer_1P_1RF}" should "exist" in { - val producer = KafkaProducer[String, String](producerConf, global) - val t = producer.send(topic = topic_consumer_1P_1RF, "test") - - t.runSyncUnsafe().isDefined shouldBe true - } - - it should "allow " in { - val conf = consumerConf.value() - - val f2 = KafkaConsumerObservable[Integer, Integer](conf, List(topic_consumer_1P_1RF)).take(1000).toListL.runToFuture(io) - - val elements = Await.result(f2, 10.seconds) - elements.size shouldBe 1000 - } - -} diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala new file mode 100644 index 00000000..a260763d --- /dev/null +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala @@ -0,0 +1,62 @@ +package monix.kafka.benchmarks + +//import monix.execution.Scheduler +import monix.execution.Scheduler.Implicits.global +import monix.kafka.KafkaProducer +import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} +import zio.ZLayer +import zio.kafka.consumer._ +import zio.kafka.serde._ +import zio.Runtime + +//import scala.concurrent.Await +//import scala.concurrent.duration._ + +class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeAndAfterAll { + + + override def beforeAll(): Unit = { + super.beforeAll() + } +/* + s"Monix topic" should "exist" in { + val producer = KafkaProducer[String, String](producerConf, global) + val t = producer.send(topic = monixTopic, "test") + + t.runSyncUnsafe().isDefined shouldBe true + } + + it should "consume from monix topic" in { + KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "test").runSyncUnsafe() + + val f = KafkaConsumerObservable[Integer, Integer](consumerConf.value(), List(monixTopic)).take(1).toListL.runToFuture(io) + + val elements = Await.result(f, 10.seconds) + elements.size shouldBe 1 + }*/ + + s"Zio topic" should "exist" in { + // val producer = KafkaProducer[String, String](producerConf, global) + // val t = producer.send(topic = zioTopic, "test") + + // t.runSyncUnsafe().isDefined shouldBe true + } + + it should "consume from monix topic" in { + + val consumerSettings: ConsumerSettings = ConsumerSettings(List("localhost:9092")).withGroupId("group") + + val z = Consumer + .subscribeAnd(Subscription.topics(monixTopic, zioTopic)) + .plainStream(Serde.string, Serde.string) + .take(1) + .provideSomeLayer(ZLayer.fromManaged(Consumer.make(consumerSettings))) + .runLast + + KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "test").runSyncUnsafe() + + val r = Runtime.default.unsafeRun(z) + r shouldBe "test" + } + +} diff --git a/benchmarks/src/test/scala/benchmarks/ProducerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala similarity index 98% rename from benchmarks/src/test/scala/benchmarks/ProducerSpec.scala rename to benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala index de685581..e621f6e4 100644 --- a/benchmarks/src/test/scala/benchmarks/ProducerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala @@ -7,11 +7,11 @@ import org.scalatest.{FlatSpec, Matchers} class ProducerSpec extends FlatSpec with MonixFixture with Matchers { val producer = KafkaProducer[String, String](producerConf, global) - +/* s"Monix ${topic_producer_1P_1RF}" should "exist befor running Producer Benchmark" in { val t = producer.send(topic = topic_producer_1P_1RF, "test") t.runSyncUnsafe().isDefined shouldBe true } - +*/ } diff --git a/benchmarks/start-kafka-cluster.sh b/benchmarks/start-kafka-cluster.sh index fdd166d0..46f43e90 100644 --- a/benchmarks/start-kafka-cluster.sh +++ b/benchmarks/start-kafka-cluster.sh @@ -18,9 +18,7 @@ docker ps sleep 15 -create_topic topic_producer_1P_1RF 1 1 -create_topic topic_producer_2P_1RF 2 1 -create_topic topic_sink_1P_1RF 1 1 -create_topic topic_sink_2P_1RF 2 1 -create_topic topic_consumer_1P_1RF 1 1 -create_topic topic_consumer_2P_1RF 2 1 \ No newline at end of file +create_topic monix_topic 2 1 +create_topic akka_topic 2 1 +create_topic fs2_topic 2 1 +create_topic zio_topic 2 1 \ No newline at end of file diff --git a/build.sbt b/build.sbt index 03d25df7..677e66e0 100644 --- a/build.sbt +++ b/build.sbt @@ -266,8 +266,8 @@ lazy val benchmarks = project.in(file("benchmarks")) organization := "io.monix", scalaVersion := "2.12.10", libraryDependencies ++= Seq( - //"com.typesafe.akka" %% "akka-stream-kafka" % "1.1.0", - //"com.typesafe.akka" %% "akka-stream" % "2.5.25" + "dev.zio" %% "zio-streams" % "1.0.2", + "dev.zio" %% "zio-kafka" % "0.13.0" ) ) .enablePlugins(JmhPlugin) From 4974c62654d7852c9d72d107492acdfa9cb015d6 Mon Sep 17 00:00:00 2001 From: Pau Date: Wed, 27 Jan 2021 19:58:16 +0100 Subject: [PATCH 20/61] Fs2 --- .../monix/kafka/benchmarks/MonixFixture.scala | 18 +++++- .../monix/kafka/benchmarks/ConsumerSpec.scala | 64 ++++++++++++++----- build.sbt | 7 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 1 - 4 files changed, 68 insertions(+), 22 deletions(-) diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala index 4a0047dc..205791df 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala @@ -1,12 +1,14 @@ package monix.kafka.benchmarks +import cats.effect.IO import monix.eval.Coeval import monix.execution.Scheduler -import monix.kafka.config.AutoOffsetReset +import monix.kafka.config.{AutoOffsetReset => MonixAutoOffsetReset} import monix.kafka.{KafkaConsumerConfig, KafkaProducerConfig} //import monix.reactive.Observable //import org.apache.kafka.clients.consumer.ConsumerRecord //import org.apache.kafka.clients.producer.ProducerRecord +import fs2.kafka._ trait MonixFixture extends BaseFixture{ @@ -24,7 +26,19 @@ trait MonixFixture extends BaseFixture{ clientId = monixTopic, groupId = randomId.value(), enableAutoCommit = false, - autoOffsetReset = AutoOffsetReset.Earliest + autoOffsetReset = MonixAutoOffsetReset.Earliest )) + val fs2ConsumerSettings: ConsumerSettings[IO, String, String] = + ConsumerSettings[IO, String, String] + .withAutoOffsetReset(AutoOffsetReset.Earliest) + .withBootstrapServers("localhost:9092") + .withGroupId("group") + + val fs2ProducerSettings: ProducerSettings[IO, String, String] = + ProducerSettings[IO, String, String] + .withBootstrapServers("localhost:9092") + + + } diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala index a260763d..23ef7e98 100644 --- a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala @@ -1,16 +1,23 @@ package monix.kafka.benchmarks -//import monix.execution.Scheduler -import monix.execution.Scheduler.Implicits.global -import monix.kafka.KafkaProducer +import cats.effect.IO +import fs2.kafka.{KafkaConsumer, KafkaProducer, ProducerRecord, ProducerRecords, commitBatchWithin} import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} -import zio.ZLayer -import zio.kafka.consumer._ -import zio.kafka.serde._ -import zio.Runtime +import cats.effect._ +import cats.implicits._ + +//import monix.execution.Scheduler +//import monix.execution.Scheduler.Implicits.global +//import monix.kafka.KafkaProducer +//import zio.ZLayer +//import zio.kafka.consumer._ +//import zio.kafka.serde._ +//import zio.Runtime +// +//import fs2.kafka._ //import scala.concurrent.Await -//import scala.concurrent.duration._ +import scala.concurrent.duration._ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeAndAfterAll { @@ -42,21 +49,44 @@ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeA // t.runSyncUnsafe().isDefined shouldBe true } - it should "consume from monix topic" in { + /* it should "consume from monix topic" in { - val consumerSettings: ConsumerSettings = ConsumerSettings(List("localhost:9092")).withGroupId("group") + val consumerSettings: ConsumerSettings = ConsumerSettings(List("127.0.0.1:9092")).withGroupId("groupId") + .withClientId("client") val z = Consumer .subscribeAnd(Subscription.topics(monixTopic, zioTopic)) .plainStream(Serde.string, Serde.string) .take(1) .provideSomeLayer(ZLayer.fromManaged(Consumer.make(consumerSettings))) - .runLast - - KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "test").runSyncUnsafe() - - val r = Runtime.default.unsafeRun(z) - r shouldBe "test" - } + .runHead + + val r = Runtime.default.unsafeRunTask(z) + + + KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "zio-test").runSyncUnsafe() + + r shouldBe "zio-test" + }*/ + import cats.effect._ + import cats.implicits._ + import fs2.kafka._ + import scala.concurrent.duration._ + + val stream = + KafkaConsumer.stream(fs2ConsumerSettings) + .evalTap(_.subscribeTo("topic")) + .flatMap(_.stream) + .mapAsync(25) { committable => + IO.pure(committable.record -> committable.record.value) + .map { case (key, value) => + val record = ProducerRecord("topic", key, value) + ProducerRecords.one(record, committable.offset) + } + } + .through(KafkaProducer.pipe(fs2ProducerSettings)) + .map(_.passthrough) + .through(commitBatchWithin(500, 15.seconds)) + .compile } diff --git a/build.sbt b/build.sbt index 677e66e0..7020b724 100644 --- a/build.sbt +++ b/build.sbt @@ -195,7 +195,7 @@ lazy val commonDependencies = Seq( "com.typesafe.scala-logging" %% "scala-logging" % "3.9.2", "com.typesafe" % "config" % "1.4.1", "org.slf4j" % "log4j-over-slf4j" % "1.7.30", - "org.scala-lang.modules" %% "scala-collection-compat" % "2.3.2", + "org.scala-lang.modules" %% "scala-collection-compat" % "2.3.1", // For testing ... "ch.qos.logback" % "logback-classic" % "1.2.3" % "test", "org.scalatest" %% "scalatest" % "3.0.9" % "test", @@ -262,18 +262,21 @@ lazy val benchmarks = project.in(file("benchmarks")) .settings(sharedSettings) .settings(commonDependencies) .settings( + scalacOptions += "-Ypartial-unification", name := "benchmarks", organization := "io.monix", scalaVersion := "2.12.10", libraryDependencies ++= Seq( "dev.zio" %% "zio-streams" % "1.0.2", - "dev.zio" %% "zio-kafka" % "0.13.0" + "dev.zio" %% "zio-kafka" % "0.13.0", + "com.github.fd4s" %% "fs2-kafka" % "1.3.1" ) ) .enablePlugins(JmhPlugin) .aggregate(kafka1x) .dependsOn(kafka1x) +scalacOptions += "-Ypartial-unification" //------------- For Release diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index dacfddc2..82dfaeec 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -42,7 +42,6 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { groupId = "kafka-tests", clientId = "monix-kafka-1-0-consumer-test", autoOffsetReset = AutoOffsetReset.Earliest - ) test("publish one message when subscribed to topics list") { From f13d85574861b4c808d024effa620346c0a82118 Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 22 Mar 2021 11:30:09 +0100 Subject: [PATCH 21/61] Heartbeat poll --- .../kafka/benchmarks/ConsumerBenchmark.scala | 4 +- .../monix/kafka/benchmarks/ConsumerSpec.scala | 40 ++-- benchmarks/test.txt | 213 ++++++++++++++++++ 3 files changed, 236 insertions(+), 21 deletions(-) create mode 100644 benchmarks/test.txt diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 1769d935..d6417748 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -26,7 +26,7 @@ class ConsumerBenchmark extends MonixFixture { // preparing test data val f = Observable - .from(1 to size) + .from(1 to size * 10) .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) .bufferTumbling(size) .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) @@ -37,6 +37,7 @@ class ConsumerBenchmark extends MonixFixture { def monix_manual_commit(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) .take(size) .headL Await.result(f, Duration.Inf) @@ -46,6 +47,7 @@ class ConsumerBenchmark extends MonixFixture { def monix_auto_commit(): Unit = { val conf = consumerConf.value().copy( maxPollRecords = maxPollRecords, + observablePollHeartbeatRate = 1.milli, observableCommitType = ObservableCommitType.Async) KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) .take(size) diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala index 23ef7e98..57a99da1 100644 --- a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala @@ -68,25 +68,25 @@ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeA r shouldBe "zio-test" }*/ - import cats.effect._ - import cats.implicits._ - import fs2.kafka._ - import scala.concurrent.duration._ - - val stream = - KafkaConsumer.stream(fs2ConsumerSettings) - .evalTap(_.subscribeTo("topic")) - .flatMap(_.stream) - .mapAsync(25) { committable => - IO.pure(committable.record -> committable.record.value) - .map { case (key, value) => - val record = ProducerRecord("topic", key, value) - ProducerRecords.one(record, committable.offset) - } - } - .through(KafkaProducer.pipe(fs2ProducerSettings)) - .map(_.passthrough) - .through(commitBatchWithin(500, 15.seconds)) - .compile + //import cats.effect._ + //import cats.implicits._ + //import fs2.kafka._ + //import scala.concurrent.duration._ + + //val stream = + // KafkaConsumer.stream(fs2ConsumerSettings) + // .evalTap(_.subscribeTo("topic")) + // .flatMap(_.stream) + // .mapAsync(25) { committable => + // IO.pure(committable.record -> committable.record.value) + // .map { case (key, value) => + // val record = ProducerRecord("topic", key, value) + // ProducerRecords.one(record, committable.offset) + // } + // } + // .through(KafkaProducer.pipe(fs2ProducerSettings)) + // .map(_.passthrough) + // .through(commitBatchWithin(500, 15.seconds)) + // .compile } diff --git a/benchmarks/test.txt b/benchmarks/test.txt new file mode 100644 index 00000000..60e9373f --- /dev/null +++ b/benchmarks/test.txt @@ -0,0 +1,213 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 1 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit + +# Run progress: 0.00% complete, ETA 00:00:40 +# Fork: 1 of 1 +# Warmup Iteration 1: + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + + + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 1 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit + +# Run progress: 50.00% complete, ETA 00:00:11 +# Fork: 1 of 1 +# Warmup Iteration 1: + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) + at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) + at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) + at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) + at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) + at scala.concurrent.Await$.ready(package.scala:146) + at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + + + + +# Run complete. Total time: 00:00:22 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units From d3a442568e747c4216db4e02fd49aa32e0d04ed2 Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 22 Mar 2021 21:28:52 +0100 Subject: [PATCH 22/61] Benchmarks updated --- .../kafka/benchmarks/ConsumerBenchmark.scala | 22 +- .../monix/kafka/benchmarks/ConsumerSpec.scala | 93 ++------- benchmarks/test.txt | 186 ++--------------- .../monix/kafka/MonixKafkaTopicListTest.scala | 189 ------------------ 4 files changed, 45 insertions(+), 445 deletions(-) delete mode 100644 kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index d6417748..14d2146b 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -1,15 +1,12 @@ package monix.kafka.benchmarks import java.util.concurrent.TimeUnit -//import monix.execution.Scheduler import monix.kafka.{KafkaConsumerObservable, KafkaProducerSink} import monix.kafka.config.ObservableCommitType import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} -import scala.concurrent.Await -import scala.concurrent.duration.Duration import scala.concurrent.duration._ @State(Scope.Thread) @@ -25,22 +22,25 @@ class ConsumerBenchmark extends MonixFixture { var maxPollRecords: Int = 5 // preparing test data - val f = Observable + Observable .from(1 to size * 10) .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) .bufferTumbling(size) .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) - .runToFuture(io) - Await.ready(f, 10.seconds) + .runSyncUnsafe() + @Benchmark def monix_manual_commit(): Unit = { - val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + val conf = consumerConf.value().copy( + maxPollRecords = maxPollRecords, + observablePollHeartbeatRate = 1.milli, + ) KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) .mapEvalF(_.committableOffset.commitAsync()) - .take(size) + .take(100) .headL - Await.result(f, Duration.Inf) + .runSyncUnsafe() } @Benchmark @@ -50,9 +50,9 @@ class ConsumerBenchmark extends MonixFixture { observablePollHeartbeatRate = 1.milli, observableCommitType = ObservableCommitType.Async) KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) - .take(size) + .take(100) .headL - Await.result(f, Duration.Inf) + .runSyncUnsafe() } } diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala index 57a99da1..2b15c6ee 100644 --- a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala @@ -1,22 +1,12 @@ package monix.kafka.benchmarks -import cats.effect.IO -import fs2.kafka.{KafkaConsumer, KafkaProducer, ProducerRecord, ProducerRecords, commitBatchWithin} +import monix.execution.Scheduler.Implicits.global +import monix.kafka.{KafkaConsumerObservable, KafkaProducer, KafkaProducerSink} +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} -import cats.effect._ -import cats.implicits._ -//import monix.execution.Scheduler -//import monix.execution.Scheduler.Implicits.global -//import monix.kafka.KafkaProducer -//import zio.ZLayer -//import zio.kafka.consumer._ -//import zio.kafka.serde._ -//import zio.Runtime -// -//import fs2.kafka._ - -//import scala.concurrent.Await +import scala.concurrent.Await import scala.concurrent.duration._ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeAndAfterAll { @@ -24,69 +14,28 @@ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeA override def beforeAll(): Unit = { super.beforeAll() + Observable + .from(1 to 1000) + .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) + .bufferTumbling(100) + .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) + .runSyncUnsafe() } -/* - s"Monix topic" should "exist" in { - val producer = KafkaProducer[String, String](producerConf, global) - val t = producer.send(topic = monixTopic, "test") - - t.runSyncUnsafe().isDefined shouldBe true - } - - it should "consume from monix topic" in { - KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "test").runSyncUnsafe() - val f = KafkaConsumerObservable[Integer, Integer](consumerConf.value(), List(monixTopic)).take(1).toListL.runToFuture(io) + s"Monix ${monixTopic}" should "exist" in { + val producer = KafkaProducer[String, String](producerConf, global) + val t = producer.send(topic = monixTopic, "test") - val elements = Await.result(f, 10.seconds) - elements.size shouldBe 1 - }*/ - - s"Zio topic" should "exist" in { - // val producer = KafkaProducer[String, String](producerConf, global) - // val t = producer.send(topic = zioTopic, "test") - - // t.runSyncUnsafe().isDefined shouldBe true + t.runSyncUnsafe().isDefined shouldBe true } - /* it should "consume from monix topic" in { - - val consumerSettings: ConsumerSettings = ConsumerSettings(List("127.0.0.1:9092")).withGroupId("groupId") - .withClientId("client") + it should "allow " in { + val conf = consumerConf.value() - val z = Consumer - .subscribeAnd(Subscription.topics(monixTopic, zioTopic)) - .plainStream(Serde.string, Serde.string) - .take(1) - .provideSomeLayer(ZLayer.fromManaged(Consumer.make(consumerSettings))) - .runHead + val f2 = KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)).take(1000).toListL.runToFuture(io) - val r = Runtime.default.unsafeRunTask(z) - - - KafkaProducer[String, String](producerConf, global).send(topic = monixTopic, "zio-test").runSyncUnsafe() - - r shouldBe "zio-test" - }*/ - //import cats.effect._ - //import cats.implicits._ - //import fs2.kafka._ - //import scala.concurrent.duration._ - - //val stream = - // KafkaConsumer.stream(fs2ConsumerSettings) - // .evalTap(_.subscribeTo("topic")) - // .flatMap(_.stream) - // .mapAsync(25) { committable => - // IO.pure(committable.record -> committable.record.value) - // .map { case (key, value) => - // val record = ProducerRecord("topic", key, value) - // ProducerRecords.one(record, committable.offset) - // } - // } - // .through(KafkaProducer.pipe(fs2ProducerSettings)) - // .map(_.passthrough) - // .through(commitBatchWithin(500, 15.seconds)) - // .compile + val elements = Await.result(f2, 10.seconds) + elements.size shouldBe 1000 + } } diff --git a/benchmarks/test.txt b/benchmarks/test.txt index 60e9373f..c51b206f 100644 --- a/benchmarks/test.txt +++ b/benchmarks/test.txt @@ -11,93 +11,12 @@ # Run progress: 0.00% complete, ETA 00:00:40 # Fork: 1 of 1 -# Warmup Iteration 1: - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_auto_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_auto_commit_jmhTest.monix_auto_commit_Throughput(ConsumerBenchmark_monix_auto_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) +# Warmup Iteration 1: 12.499 ops/s +Iteration 1: 18.910 ops/s +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": + 18.910 ops/s # JMH version: 1.21 @@ -111,98 +30,17 @@ java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] # Benchmark mode: Throughput, ops/time # Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit -# Run progress: 50.00% complete, ETA 00:00:11 +# Run progress: 50.00% complete, ETA 00:00:45 # Fork: 1 of 1 -# Warmup Iteration 1: - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.util.concurrent.TimeoutException: Futures timed out after [10 seconds] - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259) - at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:378) - at monix.execution.CancelableFuture$Async.ready(CancelableFuture.scala:360) - at scala.concurrent.Await$.$anonfun$ready$1(package.scala:195) - at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:57) - at scala.concurrent.Await$.ready(package.scala:146) - at monix.kafka.benchmarks.ConsumerBenchmark.(ConsumerBenchmark.scala:34) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B1.(ConsumerBenchmark_jmhType_B1.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B2.(ConsumerBenchmark_jmhType_B2.java:3) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType_B3.(ConsumerBenchmark_jmhType_B3.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.(ConsumerBenchmark_jmhType.java:2) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest._jmh_tryInit_f_consumerbenchmark0_0(ConsumerBenchmark_monix_manual_commit_jmhTest.java:333) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:71) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) +# Warmup Iteration 1: 10.257 ops/s +Iteration 1: 8.765 ops/s +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit": + 8.765 ops/s -# Run complete. Total time: 00:00:22 +# Run complete. Total time: 00:01:23 REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial @@ -210,4 +48,6 @@ experiments, perform baseline and negative tests that provide experimental contr the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. Do not assume the numbers tell you what you want them to tell. -Benchmark Mode Cnt Score Error Units +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 18.910 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 8.765 ops/s diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala deleted file mode 100644 index 82dfaeec..00000000 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Copyright (c) 2014-2019 by its authors. Some rights reserved. - * See the project homepage at: https://github.com/monix/monix-kafka - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package monix.kafka - -import monix.eval.Task -import monix.execution.Scheduler.Implicits.global -import monix.kafka.config.AutoOffsetReset -import monix.reactive.Observable -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.TopicPartition -import org.scalatest.FunSuite - -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.jdk.CollectionConverters._ - -class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { - val topicName = "monix-kafka-tests" - - val producerCfg = KafkaProducerConfig.default.copy( - bootstrapServers = List("127.0.0.1:6001"), - clientId = "monix-kafka-1-0-producer-test" - ) - - val consumerCfg = KafkaConsumerConfig.default.copy( - bootstrapServers = List("127.0.0.1:6001"), - groupId = "kafka-tests", - clientId = "monix-kafka-1-0-consumer-test", - autoOffsetReset = AutoOffsetReset.Earliest - - test("publish one message when subscribed to topics list") { - - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - - val consumerTask = - KafkaConsumerObservable.createConsumer[String, String](consumerCfg, List(topicName)).executeOn(io) - val consumer = Await.result(consumerTask.runToFuture, 60.seconds) - - try { - // Publishing one message - val send = producer.send(topicName, "my-message") - Await.result(send.runToFuture, 30.seconds) - - val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList - assert(records === List("my-message")) - } finally { - Await.result(producer.close().runToFuture, Duration.Inf) - consumer.close() - } - } - } - - test("listen for one message when subscribed to topics list") { - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerCfg, List(topicName)).executeOn(io) - try { - // Publishing one message - val send = producer.send(topicName, "test-message") - Await.result(send.runToFuture, 30.seconds) - - val first = consumer.take(1).map(_.value()).firstL - val result = Await.result(first.runToFuture, 30.seconds) - assert(result === "test-message") - } finally { - Await.result(producer.close().runToFuture, Duration.Inf) - } - } - } - - test("full producer/consumer test when subscribed to topics list") { - withRunningKafka { - val count = 10000 - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerCfg, List(topicName)).executeOn(io).take(count) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .map(_.value()) - .toListL - - val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } - - test("manual commit consumer test when subscribed to topics list") { - withRunningKafka { - - val count = 10000 - val topicName = "monix-kafka-manual-commit-tests" - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .executeOn(io) - .bufferTumbling(count) - .map { messages => messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) } - .mapEval { case (values, batch) => Task.shift *> batch.commitSync().map(_ => values -> batch.offsets) } - .headL - - val ((result, offsets), _) = - Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) - - val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) - assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) - } - } - - test("manual async commit consumer test when subscribed to topics list") { - withRunningKafka { - - val totalRecords = 10000 - val topicName = "monix-kafka-manual-commit-tests" - - val producer = KafkaProducerSink[Integer, Integer](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](consumerCfg, List(topicName)) - - val pushT = Observable - .range(0, totalRecords) - .map(id => new ProducerRecord(topicName, Integer.valueOf(id.toInt), Integer.valueOf(id.toInt))) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .executeOn(io) - .bufferTumbling(totalRecords) - .mapEvalF { committableMessages => - CommittableOffsetBatch(committableMessages.map(_.committableOffset)).commitAsync() - .as(committableMessages) - } - .headL - - val (commitableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() - - val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) - assert(commitableMessages.map(_.record.value().toInt).sum === (0 until totalRecords).sum) - assert(commitableMessages.map(_.committableOffset.offset).last === totalRecords) - } - } - - test("publish to closed producer when subscribed to topics list") { - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val sendTask = producer.send(topicName, "test-message") - - val result = for { - //Force creation of producer - s1 <- producer.send(topicName, "test-message-1") - res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) - (_, s2) = res - s3 <- sendTask - } yield (s1, s2, s3) - - val (first, second, third) = Await.result(result.runToFuture, 60.seconds) - assert(first.isDefined && second.isRight && third.isEmpty) - } - } - -} From 0dee12bcde201ca936874f3fd4861dfd59bdeb6c Mon Sep 17 00:00:00 2001 From: Pau Date: Tue, 23 Mar 2021 19:29:07 +0100 Subject: [PATCH 23/61] Private poll heartbeat rate --- .../monix/kafka/KafkaConsumerConfig.scala | 11 +++++++--- .../monix/kafka/KafkaConsumerObservable.scala | 6 +++-- .../scala/monix/kafka/PollHeartBeatTest.scala | 22 +++++++++---------- 3 files changed, 23 insertions(+), 16 deletions(-) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 5680fb65..f32bfcd5 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -215,7 +215,7 @@ import scala.concurrent.duration._ * by this object can be set via the map, but in case of a duplicate * a value set on the case class will overwrite value set via properties. */ -final case class KafkaConsumerConfig( +case class KafkaConsumerConfig( bootstrapServers: List[String], fetchMinBytes: Int, fetchMaxBytes: Int, @@ -258,7 +258,6 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -303,6 +302,13 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) + private[kafka] var pollHeartbeatRate: FiniteDuration = 5.millis + + private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { + pollHeartbeatRate = interval + this + } + def toJavaMap: java.util.Map[String, Object] = toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava @@ -448,7 +454,6 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index daac6991..77e1a4b9 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -102,8 +102,9 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { - Task.sleep(config.observablePollHeartbeatRate) *> - Task.defer( + Task.sleep(config.pollHeartbeatRate) *> + //todo remove + Task.defer( Task.evalAsync { if (!isAcked) { consumer.synchronized { @@ -194,6 +195,7 @@ object KafkaConsumerObservable { consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val manualCommitConfig = cfg.copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) + .withPollHeartBeatRate(cfg.pollHeartbeatRate) new KafkaConsumerObservableManualCommit[K, V](manualCommitConfig, consumer) } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index 6f2003de..b3730b37 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -16,12 +16,12 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val topicName = "monix-kafka-tests" override implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) - val producerCfg = KafkaProducerConfig.default.copy( + val producerCfg: KafkaProducerConfig = KafkaProducerConfig.default.copy( bootstrapServers = List("127.0.0.1:6001"), clientId = "monix-kafka-1-0-producer-test" ) - val consumerCfg = KafkaConsumerConfig.default.copy( + val consumerCfg: KafkaConsumerConfig = KafkaConsumerConfig.default.copy( bootstrapServers = List("127.0.0.1:6001"), groupId = "kafka-tests", clientId = "monix-kafka-1-0-consumer-test", @@ -68,7 +68,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val pollHeartbeat = 1.millis val maxPollInterval = 100.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = 200.millis).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -110,7 +110,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val pollHeartbeat = 15.seconds val maxPollInterval = 100.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -133,6 +133,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { assert(pollHeartbeat > downstreamLatency) assert(maxPollInterval < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) whenReady(t.runToFuture.failed) { ex => @@ -142,18 +143,17 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } } - //todo //java.lang.IllegalStateException: Received 50 unexpected messages. //at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) test("slow committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { withRunningKafka { - val count = 50 + val count = 5 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 100.seconds val pollHeartbeat = 15.seconds val maxPollInterval = 10.seconds val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -167,7 +167,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .doOnNextF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()).guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value()))) Task.sleep(downstreamLatency) *> manualCommit } .take(count) @@ -175,7 +175,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat * 10 < downstreamLatency) + assert(pollHeartbeat < downstreamLatency) assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) assert(lastRecord.value().toInt === count) assert(count === lastCommittableOffset.offset) @@ -192,7 +192,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val pollHeartbeat = 3000.millis val maxPollInterval = 50000.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -238,7 +238,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val pollHeartbeat = 1.seconds val maxPollInterval = 200.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[Integer, Integer](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](fastPollHeartbeatConfig, List(topicName)) From bc6eb426a8ef8c9e143d66b38d7bfe0e12a5fc6b Mon Sep 17 00:00:00 2001 From: Pau Date: Tue, 23 Mar 2021 20:31:28 +0100 Subject: [PATCH 24/61] Some improvements --- build.sbt | 7 +- .../monix/kafka/KafkaConsumerObservable.scala | 5 +- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 3 +- .../kafka/MonixKafkaTopicRegexTest.scala | 2 +- .../scala/monix/kafka/PollHeartBeatTest.scala | 134 ++++-------------- 6 files changed, 39 insertions(+), 114 deletions(-) diff --git a/build.sbt b/build.sbt index 7020b724..0d7ee278 100644 --- a/build.sbt +++ b/build.sbt @@ -199,8 +199,7 @@ lazy val commonDependencies = Seq( // For testing ... "ch.qos.logback" % "logback-classic" % "1.2.3" % "test", "org.scalatest" %% "scalatest" % "3.0.9" % "test", - "org.scalacheck" %% "scalacheck" % "1.15.2" % "test" - ) + "org.scalacheck" %% "scalacheck" % "1.15.3" % "test") ) lazy val monixKafka = project.in(file(".")) @@ -215,10 +214,10 @@ lazy val kafka1x = project.in(file("kafka-1.0.x")) .settings( name := "monix-kafka-1x", libraryDependencies ++= { - if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.0" % "test" exclude ("log4j", "log4j")) + if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "2.0.0" % "test" exclude ("log4j", "log4j")) else Seq.empty[ModuleID] }, - libraryDependencies += "org.apache.kafka" % "kafka-clients" % "1.0.2" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") + libraryDependencies += "org.apache.kafka" % "kafka-clients" % "2.0.0" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") ) lazy val kafka11 = project.in(file("kafka-0.11.x")) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 77e1a4b9..ca07f4ed 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -24,6 +24,7 @@ import monix.reactive.Observable import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer} +import java.time.Duration import scala.jdk.CollectionConverters._ import scala.concurrent.blocking import scala.util.matching.Regex @@ -38,7 +39,7 @@ import scala.util.matching.Regex trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumerTask: Task[Consumer[K, V]] + protected[kafka] def consumerTask: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -108,7 +109,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.evalAsync { if (!isAcked) { consumer.synchronized { - val records = blocking(consumer.poll(0)) + val records = blocking(consumer.poll(Duration.ZERO)) if (!records.isEmpty) { val errorMsg = s"Received ${records.count()} unexpected messages." throw new IllegalStateException(errorMsg) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index d74de333..2c9ca8f8 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -72,7 +72,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( if (cancelable.isCanceled) Stop else { consumer.resume(assignment) - val next = blocking(consumer.poll(pollTimeoutMillis)) + val next = blocking(consumer.poll(java.time.Duration.ofMillis(pollTimeoutMillis))) consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 922ac987..4407c920 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -88,7 +88,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( try consumer.synchronized { val assignment = consumer.assignment() consumer.resume(assignment) - val next = blocking(consumer.poll(pollTimeoutMillis)) + val next = blocking(consumer.poll(java.time.Duration.ofMillis(pollTimeoutMillis))) consumer.pause(assignment) val result = next.asScala.map { record => CommittableMessage( @@ -106,6 +106,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( case NonFatal(ex) => Future.failed(ex) } + } ackFuture.syncOnComplete { diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index f09c6a1b..b29c3e56 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -56,7 +56,7 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { val send = producer.send(topicMatchingRegex, "my-message") Await.result(send.runToFuture, 30.seconds) - val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList + val records = consumer.poll(java.time.Duration.ofMillis(10.seconds.toMillis)).asScala.map(_.value()).toList assert(records === List("my-message")) } finally { Await.result(producer.close().runToFuture, Duration.Inf) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index b3730b37..7a0afb18 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -6,15 +6,16 @@ import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition import monix.execution.Scheduler.Implicits.global +import org.scalactic.source import org.scalatest.FunSuite import org.scalatest.concurrent.ScalaFutures +import org.scalacheck.Prop.forAll import scala.concurrent.duration._ -class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { +class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val topicName = "monix-kafka-tests" - override implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) val producerCfg: KafkaProducerConfig = KafkaProducerConfig.default.copy( bootstrapServers = List("127.0.0.1:6001"), @@ -28,7 +29,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { autoOffsetReset = AutoOffsetReset.Earliest ) - test("slow batches processing doesn't cause rebalancing") { + test("auto committable consumer with slow processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 @@ -61,7 +62,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { test("slow committable downstream with small poll heartbeat does not cause rebalancing") { withRunningKafka { - + forAll() val totalRecords = 100 val topicName = "monix-kafka-manual-commit-tests" val downstreamLatency = 200.millis @@ -102,7 +103,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } //unhappy scenario - test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalancing") { + test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalance") { withRunningKafka { val totalRecords = 200 val topicName = "monix-kafka-manual-commit-tests" @@ -143,15 +144,21 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } } - //java.lang.IllegalStateException: Received 50 unexpected messages. - //at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) - test("slow committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { + /* Important Note: + * This test will throw throw an `IllegalStateException: Received 50 unexpected messages` + * at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) + * This does not manifests after updating the `kafka-clients` dependency to `2.x` + */ + test("super slow committable downstream causes consumer rebalance") { withRunningKafka { - val count = 5 + val totalRecords = 3 val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 100.seconds - val pollHeartbeat = 15.seconds - val maxPollInterval = 10.seconds + val downstreamLatency = 55.seconds + val pollHeartbeat = 5.seconds + val maxPollInterval = 4.seconds + // the downstreamLatency is higher than the `maxPollInterval` + // but smaller than `pollHeartBeat`, kafka will trigger rebalance + // and the consumer will be kicked out of the consumer group. val fastPollHeartbeatConfig = consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) @@ -159,7 +166,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) val pushT = Observable - .fromIterable(1 to count) + .fromIterable(1 to totalRecords) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) .mapEval(producer.send) .lastL @@ -170,103 +177,20 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()).guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value()))) Task.sleep(downstreamLatency) *> manualCommit } - .take(count) + .take(totalRecords) .toListL - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat < downstreamLatency) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } - - - test("2slows committable downstream with high `maxPollInterval` and `pollHeartBeat` does not cause consumer rebalancing") { - withRunningKafka { - val count = 10 - val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 100.millis - val pollHeartbeat = 3000.millis - val maxPollInterval = 50000.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + assert(pollHeartbeat * 10 < downstreamLatency) + assert(maxPollInterval * 10 < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) - val pushT = Observable - .fromIterable(1 to count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL + implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) - val listT = consumer - .executeOn(io) - .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) - .as(committableMessage) - Task.sleep(downstreamLatency) *> manualCommit - } - .take(count) - .toListL - - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat > downstreamLatency) - assert(maxPollInterval > downstreamLatency) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) + whenReady(t.runToFuture.failed) { ex => + assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) + }(PatienceConfig(200.seconds, 1.seconds), source.Position.here) } } - test("slow downstream with long poll heart beat and smaller pollInterval causes rebalancing") { - withRunningKafka { - - val fastMessages = 5 - val slowMessages = 3 - val totalMessages = fastMessages + slowMessages - val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 4.seconds - // the downstream latency of `slowMessages` is higher than the - // `maxPollInterval` but smaller than `pollHeartBeat`, - // kafka will trigger rebalancing and the consumer - // will be kicked out of the consumer group. - val pollHeartbeat = 1.seconds - val maxPollInterval = 200.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) - val producer = KafkaProducer[Integer, Integer](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[Integer, Integer](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to totalMessages) - .map(Integer.valueOf) - .map(msg => new ProducerRecord(topicName, msg, msg)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().as(committableMessage)) - if(committableMessage.record.value() <= fastMessages) manualCommit - else Task.sleep(downstreamLatency) *> manualCommit - } - .take(totalMessages) - .toListL - - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat > downstreamLatency) - assert(committableMessages.map(_.record.value().toInt).sum === (1 to fastMessages).sum) - assert(lastRecord.value().toInt === fastMessages) - assert(lastCommittableOffset.offset === fastMessages) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } } From eca692788a047891ca74b9119c1e2c84634f5b3f Mon Sep 17 00:00:00 2001 From: Pau Date: Thu, 25 Mar 2021 00:42:57 +0100 Subject: [PATCH 25/61] Fix unexpected consumed records --- .../monix/kafka/KafkaConsumerObservable.scala | 11 +++++--- .../scala/monix/kafka/PollHeartBeatTest.scala | 27 ++++++++++--------- 2 files changed, 23 insertions(+), 15 deletions(-) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index ca07f4ed..4e01fa25 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,6 +16,7 @@ package monix.kafka +import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable, Scheduler} @@ -36,10 +37,10 @@ import scala.util.matching.Regex * [[KafkaConsumerConfig]] needed and see `monix/kafka/default.conf`, * (in the resource files) that is exposing all default values. */ -trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { +trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogging { protected def config: KafkaConsumerConfig - protected[kafka] def consumerTask: Task[Consumer[K, V]] + protected def consumerTask: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -109,13 +110,17 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.evalAsync { if (!isAcked) { consumer.synchronized { + // needed in order to ensure that the consummer assignment + // is paused, meaning that no messages will get lost + val assignment = consumer.assignment() + consumer.pause(assignment) val records = blocking(consumer.poll(Duration.ZERO)) if (!records.isEmpty) { val errorMsg = s"Received ${records.count()} unexpected messages." throw new IllegalStateException(errorMsg) } } - } else () + } }.onErrorHandle(ex => scheduler.reportFailure(ex)) ) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index 7a0afb18..ebe60519 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -9,7 +9,7 @@ import monix.execution.Scheduler.Implicits.global import org.scalactic.source import org.scalatest.FunSuite import org.scalatest.concurrent.ScalaFutures -import org.scalacheck.Prop.forAll +//import org.scalacheck.Prop.forAll import scala.concurrent.duration._ @@ -17,6 +17,8 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val topicName = "monix-kafka-tests" + override implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + val producerCfg: KafkaProducerConfig = KafkaProducerConfig.default.copy( bootstrapServers = List("127.0.0.1:6001"), clientId = "monix-kafka-1-0-producer-test" @@ -35,7 +37,8 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - heartbeatInterval = 10.millis + heartbeatInterval = 10.millis, + maxPollRecords = 1 ) val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -55,21 +58,21 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .flatMap(Observable.fromIterable) .toListL - val (result, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() + val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() assert(result.map(_.toInt).sum === (0 until count).sum) } } test("slow committable downstream with small poll heartbeat does not cause rebalancing") { withRunningKafka { - forAll() - val totalRecords = 100 + val totalRecords = 1000 val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 200.millis + val downstreamLatency = 40.millis val pollHeartbeat = 1.millis - val maxPollInterval = 100.millis + val maxPollInterval = 10.millis + val maxPollRecords = 1 val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -83,14 +86,14 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } .take(totalRecords) .toListL - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe() + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert(pollHeartbeat < maxPollInterval) @@ -111,7 +114,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val pollHeartbeat = 15.seconds val maxPollInterval = 100.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -160,7 +163,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { // but smaller than `pollHeartBeat`, kafka will trigger rebalance // and the consumer will be kicked out of the consumer group. val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) From 6165418fc12988a8f2fa6a987d0eeb8a1d198680 Mon Sep 17 00:00:00 2001 From: Pau Date: Thu, 25 Mar 2021 09:17:23 +0100 Subject: [PATCH 26/61] Benchmarks with different pollHeartbeatIntervals --- benchmarks/consumer-1.txt | 142 ++++++++++++++ benchmarks/consumer2.txt | 174 ++++++++++++++++++ benchmarks/results.txt | 53 ------ benchmarks/results/consumer.md | 18 ++ benchmarks/results/readme.md | 25 +++ .../kafka/benchmarks/ConsumerBenchmark.scala | 46 ++++- benchmarks/test.txt | 53 ------ .../monix/kafka/KafkaConsumerConfig.scala | 2 +- 8 files changed, 400 insertions(+), 113 deletions(-) create mode 100644 benchmarks/consumer-1.txt create mode 100644 benchmarks/consumer2.txt delete mode 100644 benchmarks/results.txt create mode 100644 benchmarks/results/consumer.md create mode 100644 benchmarks/results/readme.md delete mode 100644 benchmarks/test.txt diff --git a/benchmarks/consumer-1.txt b/benchmarks/consumer-1.txt new file mode 100644 index 00000000..8f5bcfe1 --- /dev/null +++ b/benchmarks/consumer-1.txt @@ -0,0 +1,142 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit + +# Run progress: 0.00% complete, ETA 00:07:20 +# Fork: 1 of 1 +# Warmup Iteration 1: 9.643 ops/s +Iteration 1: 13.072 ops/s +Iteration 2: 12.147 ops/s +Iteration 3: 11.022 ops/s +Iteration 4: 8.540 ops/s +Iteration 5: 9.754 ops/s +Iteration 6: 10.663 ops/s +Iteration 7: 11.656 ops/s +Iteration 8: 12.694 ops/s +Iteration 9: 11.146 ops/s +Iteration 10: 11.898 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": + 11.259 ±(99.9%) 2.065 ops/s [Average] + (min, avg, max) = (8.540, 11.259, 13.072), stdev = 1.366 + CI (99.9%): [9.195, 13.324] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit + +# Run progress: 25.00% complete, ETA 00:06:06 +# Fork: 1 of 1 +# Warmup Iteration 1: + +java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monix_manual_commit()V + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_thrpt_jmhStub(ConsumerBenchmark_monix_manual_commit_jmhTest.java:119) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:83) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + + + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1 + +# Run progress: 50.00% complete, ETA 00:02:13 +# Fork: 1 of 1 +# Warmup Iteration 1: 7.288 ops/s +Iteration 1: 9.133 ops/s +Iteration 2: 9.689 ops/s +Iteration 3: 10.091 ops/s +Iteration 4: 10.151 ops/s +Iteration 5: 10.438 ops/s +Iteration 6: 10.438 ops/s +Iteration 7: 10.539 ops/s +Iteration 8: 10.347 ops/s +Iteration 9: 8.787 ops/s +Iteration 10: 9.061 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1": + 9.867 ±(99.9%) 0.989 ops/s [Average] + (min, avg, max) = (8.787, 9.867, 10.539), stdev = 0.654 + CI (99.9%): [8.878, 10.856] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000 + +# Run progress: 75.00% complete, ETA 00:01:24 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.980 ops/s +Iteration 1: 0.991 ops/s +Iteration 2: 0.991 ops/s +Iteration 3: 0.993 ops/s +Iteration 4: 0.993 ops/s +Iteration 5: 0.993 ops/s +Iteration 6: 0.993 ops/s +Iteration 7: 0.994 ops/s +Iteration 8: 0.994 ops/s +Iteration 9: 0.994 ops/s +Iteration 10: 0.994 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000": + 0.993 ±(99.9%) 0.002 ops/s [Average] + (min, avg, max) = (0.991, 0.993, 0.994), stdev = 0.001 + CI (99.9%): [0.991, 0.995] (assumes normal distribution) + + +# Run complete. Total time: 00:06:15 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 10 11.259 ± 2.065 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 9.867 ± 0.989 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.993 ± 0.002 ops/s diff --git a/benchmarks/consumer2.txt b/benchmarks/consumer2.txt new file mode 100644 index 00000000..57790247 --- /dev/null +++ b/benchmarks/consumer2.txt @@ -0,0 +1,174 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit + +# Run progress: 0.00% complete, ETA 00:09:10 +# Fork: 1 of 1 +# Warmup Iteration 1: 8.696 ops/s +Iteration 1: 9.784 ops/s +Iteration 2: 12.588 ops/s +Iteration 3: 12.075 ops/s +Iteration 4: 8.459 ops/s +Iteration 5: 10.894 ops/s +Iteration 6: 9.174 ops/s +Iteration 7: 11.828 ops/s +Iteration 8: 10.768 ops/s +Iteration 9: 9.139 ops/s +Iteration 10: 10.985 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": + 10.569 ±(99.9%) 2.096 ops/s [Average] + (min, avg, max) = (8.459, 10.569, 12.588), stdev = 1.387 + CI (99.9%): [8.473, 12.666] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1 + +# Run progress: 20.00% complete, ETA 00:08:09 +# Fork: 1 of 1 +# Warmup Iteration 1: 6.852 ops/s +Iteration 1: 11.218 ops/s +Iteration 2: 9.514 ops/s +Iteration 3: 10.431 ops/s +Iteration 4: 9.078 ops/s +Iteration 5: 9.250 ops/s +Iteration 6: 10.335 ops/s +Iteration 7: 9.936 ops/s +Iteration 8: 12.642 ops/s +Iteration 9: 9.415 ops/s +Iteration 10: 11.382 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1": + 10.320 ±(99.9%) 1.720 ops/s [Average] + (min, avg, max) = (9.078, 10.320, 12.642), stdev = 1.137 + CI (99.9%): [8.601, 12.040] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100 + +# Run progress: 40.00% complete, ETA 00:06:06 +# Fork: 1 of 1 +# Warmup Iteration 1: 4.483 ops/s +Iteration 1: 4.252 ops/s +Iteration 2: 4.466 ops/s +Iteration 3: 4.261 ops/s +Iteration 4: 4.578 ops/s +Iteration 5: 4.676 ops/s +Iteration 6: 4.380 ops/s +Iteration 7: 4.597 ops/s +Iteration 8: 4.703 ops/s +Iteration 9: 4.526 ops/s +Iteration 10: 4.745 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100": + 4.518 ±(99.9%) 0.266 ops/s [Average] + (min, avg, max) = (4.252, 4.518, 4.745), stdev = 0.176 + CI (99.9%): [4.252, 4.784] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000 + +# Run progress: 60.00% complete, ETA 00:04:04 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.977 ops/s +Iteration 1: 0.992 ops/s +Iteration 2: 0.994 ops/s +Iteration 3: 0.992 ops/s +Iteration 4: 0.992 ops/s +Iteration 5: 0.995 ops/s +Iteration 6: 0.994 ops/s +Iteration 7: 0.994 ops/s +Iteration 8: 0.995 ops/s +Iteration 9: 0.995 ops/s +Iteration 10: 0.994 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000": + 0.994 ±(99.9%) 0.002 ops/s [Average] + (min, avg, max) = (0.992, 0.994, 0.995), stdev = 0.001 + CI (99.9%): [0.992, 0.995] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat3000 + +# Run progress: 80.00% complete, ETA 00:02:02 +# Fork: 1 of 1 +# Warmup Iteration 1: 0.329 ops/s +Iteration 1: 0.332 ops/s +Iteration 2: 0.331 ops/s +Iteration 3: 0.332 ops/s +Iteration 4: 0.332 ops/s +Iteration 5: 0.332 ops/s +Iteration 6: 0.333 ops/s +Iteration 7: 0.333 ops/s +Iteration 8: 0.333 ops/s +Iteration 9: 0.332 ops/s +Iteration 10: 0.333 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat3000": + 0.332 ±(99.9%) 0.001 ops/s [Average] + (min, avg, max) = (0.331, 0.332, 0.333), stdev = 0.001 + CI (99.9%): [0.332, 0.333] (assumes normal distribution) + + +# Run complete. Total time: 00:10:31 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s diff --git a/benchmarks/results.txt b/benchmarks/results.txt deleted file mode 100644 index 366f6d34..00000000 --- a/benchmarks/results.txt +++ /dev/null @@ -1,53 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 1 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit - -# Run progress: 0.00% complete, ETA 00:00:40 -# Fork: 1 of 1 -# Warmup Iteration 1: 545234.343 ops/s -Iteration 1: 605656.170 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": - 605656.170 ops/s - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 1 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit - -# Run progress: 50.00% complete, ETA 00:00:23 -# Fork: 1 of 1 -# Warmup Iteration 1: 578595.083 ops/s -Iteration 1: 578552.396 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit": - 578552.396 ops/s - - -# Run complete. Total time: 00:00:46 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 605656.170 ops/s -ConsumerBenchmark.monix_manual_commit thrpt 578552.396 ops/s diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md new file mode 100644 index 00000000..dab14b94 --- /dev/null +++ b/benchmarks/results/consumer.md @@ -0,0 +1,18 @@ + +## RC1 + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.auto_commit_async_1P_1RF thrpt 10 53.286 ± 15.924 ops/s +ConsumerBenchmark.auto_commit_async_2P_1RF thrpt 10 45.976 ± 17.338 ops/s +ConsumerBenchmark.auto_commit_sync_1P_1RF thrpt 10 52.235 ± 12.441 ops/s +ConsumerBenchmark.auto_commit_sync_2P_1RF thrpt 10 49.599 ± 12.737 ops/s +ConsumerBenchmark.manual_commit_1P_1RF thrpt 10 40.858 ± 10.888 ops/s +ConsumerBenchmark.manual_commit_2P_1RF thrpt 10 44.895 ± 10.388 ops/s + +## RC2 - Introduces PollHeartbeatInterval + +ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s \ No newline at end of file diff --git a/benchmarks/results/readme.md b/benchmarks/results/readme.md new file mode 100644 index 00000000..6f2b612d --- /dev/null +++ b/benchmarks/results/readme.md @@ -0,0 +1,25 @@ +###Monix Kafka Benchmarks + +This document explains the approach followed to benchmark monix-kafka. + + Ideally, a Kafka performance benchmark should happen under some long stress test in a real Kafka cluster, +although, our hardware limitations we have to stick to running simpler basic tests that proves application performance on + a docker container. + +The benchmark will focus on the most basic `consumer` and `procucer` scenarios. + +Although Kafka is very configurable by nature, our benchmark will use the kafka default properties for [consumer](https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html) +and [producer](https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html). + +It also assumes that all the used topics will have 2 partitions, and 1 replication factor. + +## Producer benchmarks + +This section includes benchmarks for single and sink producers. +Although some libraries like `alpakka-kafka` do not expose methods for producing single record, but only for sink. + + +## Consumer benchmark +The consumer benchmark covers the manual and auto commit consumer implementations of the different libraries. +The manual commit will also cover producing committing back the consumed offsets. +In the `monix` performance test scenarios, compares with a different range `pollHeartbeatRate` [10, 100, 1000, 3000]. \ No newline at end of file diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 14d2146b..e2f6bc2e 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -31,11 +31,46 @@ class ConsumerBenchmark extends MonixFixture { @Benchmark - def monix_manual_commit(): Unit = { - val conf = consumerConf.value().copy( - maxPollRecords = maxPollRecords, - observablePollHeartbeatRate = 1.milli, - ) + def monix_manual_commit_heartbeat1(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(1.millis) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(100) + .headL + .runSyncUnsafe() + } + + @Benchmark + def monix_manual_commit_heartbeat100(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(100.millis) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(100) + .headL + .runSyncUnsafe() + } + + @Benchmark + def monix_manual_commit_heartbeat1000(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(1000.millis) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(100) + .headL + .runSyncUnsafe() + } + + @Benchmark + def monix_manual_commit_heartbeat3000(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(3000.millis) + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) .mapEvalF(_.committableOffset.commitAsync()) .take(100) @@ -47,7 +82,6 @@ class ConsumerBenchmark extends MonixFixture { def monix_auto_commit(): Unit = { val conf = consumerConf.value().copy( maxPollRecords = maxPollRecords, - observablePollHeartbeatRate = 1.milli, observableCommitType = ObservableCommitType.Async) KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) .take(100) diff --git a/benchmarks/test.txt b/benchmarks/test.txt deleted file mode 100644 index c51b206f..00000000 --- a/benchmarks/test.txt +++ /dev/null @@ -1,53 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 1 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit - -# Run progress: 0.00% complete, ETA 00:00:40 -# Fork: 1 of 1 -# Warmup Iteration 1: 12.499 ops/s -Iteration 1: 18.910 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": - 18.910 ops/s - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 1 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit - -# Run progress: 50.00% complete, ETA 00:00:45 -# Fork: 1 of 1 -# Warmup Iteration 1: 10.257 ops/s -Iteration 1: 8.765 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit": - 8.765 ops/s - - -# Run complete. Total time: 00:01:23 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 18.910 ops/s -ConsumerBenchmark.monix_manual_commit thrpt 8.765 ops/s diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index f32bfcd5..f0c0ed85 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -302,7 +302,7 @@ case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 5.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 100.millis private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { pollHeartbeatRate = interval From 610eb741cfc5205c7e653cecb2e4cccef44e2e0c Mon Sep 17 00:00:00 2001 From: Pau Date: Thu, 25 Mar 2021 19:43:23 +0100 Subject: [PATCH 27/61] New benchmark results --- benchmarks/consumer3.txt | 46 +++++++++++++++++++ benchmarks/consumer4.txt | 14 ++++++ benchmarks/results/consumer.md | 8 +++- .../monix/kafka/benchmarks/BaseFixture.scala | 1 - .../kafka/benchmarks/ConsumerBenchmark.scala | 38 +++++++++------ .../monix/kafka/benchmarks/ConsumerSpec.scala | 12 ++++- 6 files changed, 101 insertions(+), 18 deletions(-) create mode 100644 benchmarks/consumer3.txt create mode 100644 benchmarks/consumer4.txt diff --git a/benchmarks/consumer3.txt b/benchmarks/consumer3.txt new file mode 100644 index 00000000..173efd83 --- /dev/null +++ b/benchmarks/consumer3.txt @@ -0,0 +1,46 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit + +# Run progress: 0.00% complete, ETA 00:03:40 +# Fork: 1 of 1 +# Warmup Iteration 1: 8.247 ops/s +Iteration 1: 10.887 ops/s +Iteration 2: 11.890 ops/s +Iteration 3: 9.795 ops/s +Iteration 4: 11.231 ops/s +Iteration 5: 12.354 ops/s +Iteration 6: 13.061 ops/s +Iteration 7: 13.098 ops/s +Iteration 8: 13.229 ops/s +Iteration 9: 12.850 ops/s +Iteration 10: 9.933 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": + 11.833 ±(99.9%) 1.977 ops/s [Average] + (min, avg, max) = (9.795, 11.833, 13.229), stdev = 1.308 + CI (99.9%): [9.856, 13.809] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit + +# Run progress: 50.00% complete, ETA 00:02:02 +# Fork: 1 of 1 +# Warmup Iteration 1: \ No newline at end of file diff --git a/benchmarks/consumer4.txt b/benchmarks/consumer4.txt new file mode 100644 index 00000000..88f204b7 --- /dev/null +++ b/benchmarks/consumer4.txt @@ -0,0 +1,14 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 1 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit + +# Run progress: 0.00% complete, ETA 00:00:20 +# Fork: 1 of 1 +# Warmup Iteration 1: \ No newline at end of file diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index dab14b94..99ed086c 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -9,10 +9,14 @@ ConsumerBenchmark.auto_commit_sync_2P_1RF thrpt 10 49.599 ± 12.737 ops/s ConsumerBenchmark.manual_commit_1P_1RF thrpt 10 40.858 ± 10.888 ops/s ConsumerBenchmark.manual_commit_2P_1RF thrpt 10 44.895 ± 10.388 ops/s -## RC2 - Introduces PollHeartbeatInterval +## RC2 - Introduces PollHeartbeatRate +Without pollHeartBeatRate +--- ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s \ No newline at end of file +ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s +--- +Without pollHeartBeatRate diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala index a0ff4305..7d50913e 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala @@ -15,5 +15,4 @@ trait BaseFixture { val fs2Topic = "fs2_topic" val zioTopic = "zio_topic" - } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index e2f6bc2e..3086f3d9 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -2,13 +2,11 @@ package monix.kafka.benchmarks import java.util.concurrent.TimeUnit import monix.kafka.{KafkaConsumerObservable, KafkaProducerSink} -import monix.kafka.config.ObservableCommitType +//import monix.kafka.config.ObservableCommitType import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} -import scala.concurrent.duration._ - @State(Scope.Thread) @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) @@ -29,7 +27,18 @@ class ConsumerBenchmark extends MonixFixture { .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) .runSyncUnsafe() + @Benchmark + def monix_manual_commit(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(100) + .headL + .runSyncUnsafe() + } + /* @Benchmark def monix_manual_commit_heartbeat1(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) @@ -77,16 +86,17 @@ class ConsumerBenchmark extends MonixFixture { .headL .runSyncUnsafe() } - - @Benchmark - def monix_auto_commit(): Unit = { - val conf = consumerConf.value().copy( - maxPollRecords = maxPollRecords, - observableCommitType = ObservableCommitType.Async) - KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) - .take(100) - .headL - .runSyncUnsafe() - } + */ + + //@Benchmark + //def monix_auto_commit(): Unit = { + // val conf = consumerConf.value().copy( + // maxPollRecords = maxPollRecords, + // observableCommitType = ObservableCommitType.Async) + // KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) + // .take(100) + // .headL + // .runSyncUnsafe() + //} } diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala index 2b15c6ee..b2009471 100644 --- a/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ConsumerSpec.scala @@ -29,7 +29,7 @@ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeA t.runSyncUnsafe().isDefined shouldBe true } - it should "allow " in { + it should "consume with autocommit" in { val conf = consumerConf.value() val f2 = KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)).take(1000).toListL.runToFuture(io) @@ -38,4 +38,14 @@ class ConsumerSpec extends FlatSpec with MonixFixture with Matchers with BeforeA elements.size shouldBe 1000 } + it should "consume with manual commit" in { + val conf = consumerConf.value() + + val f2 = KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .take(1000).toListL.runToFuture(io) + + val elements = Await.result(f2, 10.seconds) + elements.size shouldBe 1000 + } + } From f0861e7208dc0bd7c9f49ac78d53358148ca488e Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 26 Mar 2021 00:45:30 +0100 Subject: [PATCH 28/61] More benchmarking --- benchmarks/results/consumer.md | 27 +++++--- .../kafka/benchmarks/ConsumerBenchmark.scala | 68 ++++++++++++------- .../monix/kafka/KafkaConsumerObservable.scala | 4 +- 3 files changed, 61 insertions(+), 38 deletions(-) diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 99ed086c..1d1bf6e2 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -1,17 +1,13 @@ -## RC1 +## RC7 +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 11.876 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 11.964 ops/s -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.auto_commit_async_1P_1RF thrpt 10 53.286 ± 15.924 ops/s -ConsumerBenchmark.auto_commit_async_2P_1RF thrpt 10 45.976 ± 17.338 ops/s -ConsumerBenchmark.auto_commit_sync_1P_1RF thrpt 10 52.235 ± 12.441 ops/s -ConsumerBenchmark.auto_commit_sync_2P_1RF thrpt 10 49.599 ± 12.737 ops/s -ConsumerBenchmark.manual_commit_1P_1RF thrpt 10 40.858 ± 10.888 ops/s -ConsumerBenchmark.manual_commit_2P_1RF thrpt 10 44.895 ± 10.388 ops/s -## RC2 - Introduces PollHeartbeatRate +## RC8 - (Introduces PollHeartbeatRate) -Without pollHeartBeatRate +1fork 1thread --- ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s @@ -19,4 +15,13 @@ ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s --- -Without pollHeartBeatRate +1fork 3thrads +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 10 16.270 ± 3.339 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 15.053 ± 0.959 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 9.525 ± 1.131 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 2.968 ± 0.010 ops/s + + + + diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 3086f3d9..ceafc201 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -1,11 +1,14 @@ package monix.kafka.benchmarks +import monix.kafka.config.ObservableCommitType + import java.util.concurrent.TimeUnit import monix.kafka.{KafkaConsumerObservable, KafkaProducerSink} //import monix.kafka.config.ObservableCommitType import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} +import scala.concurrent.duration._ @State(Scope.Thread) @BenchmarkMode(Array(Mode.Throughput)) @@ -21,24 +24,39 @@ class ConsumerBenchmark extends MonixFixture { // preparing test data Observable - .from(1 to size * 10) + .from(0 to size) .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) .bufferTumbling(size) .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) .runSyncUnsafe() + //@Benchmark + //def monix_manual_commit(): Unit = { + // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + // .withPollHeartBeatRate(100.millis) + + // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + // .mapEvalF(_.committableOffset.commitAsync()) + // .take(100) + // .headL + // .runSyncUnsafe() + //} + + @Benchmark - def monix_manual_commit(): Unit = { - val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + def monix_auto_commit(): Unit = { + val conf = consumerConf.value().copy( + maxPollRecords = maxPollRecords, + observableCommitType = ObservableCommitType.Async) + .withPollHeartBeatRate(10.millis) - KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) - .mapEvalF(_.committableOffset.commitAsync()) + KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) .take(100) .headL .runSyncUnsafe() } - /* + @Benchmark def monix_manual_commit_heartbeat1(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) @@ -51,6 +69,18 @@ class ConsumerBenchmark extends MonixFixture { .runSyncUnsafe() } + //@Benchmark + //def monix_manual_commit_heartbeat10(): Unit = { + // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + // .withPollHeartBeatRate(10.millis) + + // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + // .mapEvalF(_.committableOffset.commitAsync()) + // .take(100) + // .headL + // .runSyncUnsafe() + //} + @Benchmark def monix_manual_commit_heartbeat100(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) @@ -75,28 +105,18 @@ class ConsumerBenchmark extends MonixFixture { .runSyncUnsafe() } - @Benchmark - def monix_manual_commit_heartbeat3000(): Unit = { - val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - .withPollHeartBeatRate(3000.millis) - - KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) - .mapEvalF(_.committableOffset.commitAsync()) - .take(100) - .headL - .runSyncUnsafe() - } - */ - //@Benchmark - //def monix_auto_commit(): Unit = { - // val conf = consumerConf.value().copy( - // maxPollRecords = maxPollRecords, - // observableCommitType = ObservableCommitType.Async) - // KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) + //def monix_manual_commit_heartbeat3000(): Unit = { + // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + // .withPollHeartBeatRate(3000.millis) + + // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + // .mapEvalF(_.committableOffset.commitAsync()) // .take(100) // .headL // .runSyncUnsafe() //} + + } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 4e01fa25..76f35b2c 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -105,8 +105,6 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg */ private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { Task.sleep(config.pollHeartbeatRate) *> - //todo remove - Task.defer( Task.evalAsync { if (!isAcked) { consumer.synchronized { @@ -121,7 +119,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg } } } - }.onErrorHandle(ex => scheduler.reportFailure(ex)) + }.onErrorHandle(ex => scheduler.reportFailure(ex) ) } From d37e3a2f4bc362a65c5465fc9f275485c2b232bb Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 26 Mar 2021 10:21:06 +0100 Subject: [PATCH 29/61] Increase consumed records on perf test --- benchmarks/consumer-1.txt | 142 ------------ benchmarks/{consumer2.txt => consumer-t1.txt} | 182 +++++++-------- benchmarks/consumer3.txt | 46 ---- benchmarks/consumer4.txt | 14 -- benchmarks/results/consumer.md | 35 +-- benchmarks/results/consumer.txt | 208 ------------------ benchmarks/results/producer.txt | 75 ------- .../kafka/benchmarks/ConsumerBenchmark.scala | 82 +++---- .../monix/kafka/KafkaConsumerObservable.scala | 40 ++-- 9 files changed, 165 insertions(+), 659 deletions(-) delete mode 100644 benchmarks/consumer-1.txt rename benchmarks/{consumer2.txt => consumer-t1.txt} (50%) delete mode 100644 benchmarks/consumer3.txt delete mode 100644 benchmarks/consumer4.txt delete mode 100644 benchmarks/results/consumer.txt delete mode 100644 benchmarks/results/producer.txt diff --git a/benchmarks/consumer-1.txt b/benchmarks/consumer-1.txt deleted file mode 100644 index 8f5bcfe1..00000000 --- a/benchmarks/consumer-1.txt +++ /dev/null @@ -1,142 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit - -# Run progress: 0.00% complete, ETA 00:07:20 -# Fork: 1 of 1 -# Warmup Iteration 1: 9.643 ops/s -Iteration 1: 13.072 ops/s -Iteration 2: 12.147 ops/s -Iteration 3: 11.022 ops/s -Iteration 4: 8.540 ops/s -Iteration 5: 9.754 ops/s -Iteration 6: 10.663 ops/s -Iteration 7: 11.656 ops/s -Iteration 8: 12.694 ops/s -Iteration 9: 11.146 ops/s -Iteration 10: 11.898 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": - 11.259 ±(99.9%) 2.065 ops/s [Average] - (min, avg, max) = (8.540, 11.259, 13.072), stdev = 1.366 - CI (99.9%): [9.195, 13.324] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit - -# Run progress: 25.00% complete, ETA 00:06:06 -# Fork: 1 of 1 -# Warmup Iteration 1: - -java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monix_manual_commit()V - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_thrpt_jmhStub(ConsumerBenchmark_monix_manual_commit_jmhTest.java:119) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monix_manual_commit_jmhTest.monix_manual_commit_Throughput(ConsumerBenchmark_monix_manual_commit_jmhTest.java:83) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - - - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1 - -# Run progress: 50.00% complete, ETA 00:02:13 -# Fork: 1 of 1 -# Warmup Iteration 1: 7.288 ops/s -Iteration 1: 9.133 ops/s -Iteration 2: 9.689 ops/s -Iteration 3: 10.091 ops/s -Iteration 4: 10.151 ops/s -Iteration 5: 10.438 ops/s -Iteration 6: 10.438 ops/s -Iteration 7: 10.539 ops/s -Iteration 8: 10.347 ops/s -Iteration 9: 8.787 ops/s -Iteration 10: 9.061 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1": - 9.867 ±(99.9%) 0.989 ops/s [Average] - (min, avg, max) = (8.787, 9.867, 10.539), stdev = 0.654 - CI (99.9%): [8.878, 10.856] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000 - -# Run progress: 75.00% complete, ETA 00:01:24 -# Fork: 1 of 1 -# Warmup Iteration 1: 0.980 ops/s -Iteration 1: 0.991 ops/s -Iteration 2: 0.991 ops/s -Iteration 3: 0.993 ops/s -Iteration 4: 0.993 ops/s -Iteration 5: 0.993 ops/s -Iteration 6: 0.993 ops/s -Iteration 7: 0.994 ops/s -Iteration 8: 0.994 ops/s -Iteration 9: 0.994 ops/s -Iteration 10: 0.994 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000": - 0.993 ±(99.9%) 0.002 ops/s [Average] - (min, avg, max) = (0.991, 0.993, 0.994), stdev = 0.001 - CI (99.9%): [0.991, 0.995] (assumes normal distribution) - - -# Run complete. Total time: 00:06:15 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 10 11.259 ± 2.065 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 9.867 ± 0.989 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.993 ± 0.002 ops/s diff --git a/benchmarks/consumer2.txt b/benchmarks/consumer-t1.txt similarity index 50% rename from benchmarks/consumer2.txt rename to benchmarks/consumer-t1.txt index 57790247..0a84cdd1 100644 --- a/benchmarks/consumer2.txt +++ b/benchmarks/consumer-t1.txt @@ -7,27 +7,27 @@ # Timeout: 10 min per iteration # Threads: 1 thread, will synchronize iterations # Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit10ms # Run progress: 0.00% complete, ETA 00:09:10 # Fork: 1 of 1 -# Warmup Iteration 1: 8.696 ops/s -Iteration 1: 9.784 ops/s -Iteration 2: 12.588 ops/s -Iteration 3: 12.075 ops/s -Iteration 4: 8.459 ops/s -Iteration 5: 10.894 ops/s -Iteration 6: 9.174 ops/s -Iteration 7: 11.828 ops/s -Iteration 8: 10.768 ops/s -Iteration 9: 9.139 ops/s -Iteration 10: 10.985 ops/s +# Warmup Iteration 1: 8.357 ops/s +Iteration 1: 9.274 ops/s +Iteration 2: 10.391 ops/s +Iteration 3: 10.828 ops/s +Iteration 4: 10.854 ops/s +Iteration 5: 11.172 ops/s +Iteration 6: 10.864 ops/s +Iteration 7: 9.852 ops/s +Iteration 8: 11.435 ops/s +Iteration 9: 12.843 ops/s +Iteration 10: 13.383 ops/s -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": - 10.569 ±(99.9%) 2.096 ops/s [Average] - (min, avg, max) = (8.459, 10.569, 12.588), stdev = 1.387 - CI (99.9%): [8.473, 12.666] (assumes normal distribution) +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit10ms": + 11.090 ±(99.9%) 1.883 ops/s [Average] + (min, avg, max) = (9.274, 11.090, 13.383), stdev = 1.245 + CI (99.9%): [9.207, 12.972] (assumes normal distribution) # JMH version: 1.21 @@ -39,27 +39,27 @@ Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": # Timeout: 10 min per iteration # Threads: 1 thread, will synchronize iterations # Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1 +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000ms -# Run progress: 20.00% complete, ETA 00:08:09 +# Run progress: 20.00% complete, ETA 00:07:35 # Fork: 1 of 1 -# Warmup Iteration 1: 6.852 ops/s -Iteration 1: 11.218 ops/s -Iteration 2: 9.514 ops/s -Iteration 3: 10.431 ops/s -Iteration 4: 9.078 ops/s -Iteration 5: 9.250 ops/s -Iteration 6: 10.335 ops/s -Iteration 7: 9.936 ops/s -Iteration 8: 12.642 ops/s -Iteration 9: 9.415 ops/s -Iteration 10: 11.382 ops/s +# Warmup Iteration 1: 0.979 ops/s +Iteration 1: 0.992 ops/s +Iteration 2: 0.991 ops/s +Iteration 3: 0.992 ops/s +Iteration 4: 0.994 ops/s +Iteration 5: 0.993 ops/s +Iteration 6: 0.995 ops/s +Iteration 7: 0.993 ops/s +Iteration 8: 0.994 ops/s +Iteration 9: 0.994 ops/s +Iteration 10: 0.994 ops/s -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1": - 10.320 ±(99.9%) 1.720 ops/s [Average] - (min, avg, max) = (9.078, 10.320, 12.642), stdev = 1.137 - CI (99.9%): [8.601, 12.040] (assumes normal distribution) +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000ms": + 0.993 ±(99.9%) 0.002 ops/s [Average] + (min, avg, max) = (0.991, 0.993, 0.995), stdev = 0.001 + CI (99.9%): [0.991, 0.995] (assumes normal distribution) # JMH version: 1.21 @@ -71,27 +71,27 @@ Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1" # Timeout: 10 min per iteration # Threads: 1 thread, will synchronize iterations # Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100 +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100ms -# Run progress: 40.00% complete, ETA 00:06:06 +# Run progress: 40.00% complete, ETA 00:05:41 # Fork: 1 of 1 -# Warmup Iteration 1: 4.483 ops/s -Iteration 1: 4.252 ops/s -Iteration 2: 4.466 ops/s -Iteration 3: 4.261 ops/s -Iteration 4: 4.578 ops/s -Iteration 5: 4.676 ops/s -Iteration 6: 4.380 ops/s -Iteration 7: 4.597 ops/s -Iteration 8: 4.703 ops/s -Iteration 9: 4.526 ops/s -Iteration 10: 4.745 ops/s +# Warmup Iteration 1: 4.706 ops/s +Iteration 1: 4.789 ops/s +Iteration 2: 4.786 ops/s +Iteration 3: 4.772 ops/s +Iteration 4: 4.797 ops/s +Iteration 5: 4.802 ops/s +Iteration 6: 4.790 ops/s +Iteration 7: 4.799 ops/s +Iteration 8: 4.809 ops/s +Iteration 9: 4.794 ops/s +Iteration 10: 4.778 ops/s -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100": - 4.518 ±(99.9%) 0.266 ops/s [Average] - (min, avg, max) = (4.252, 4.518, 4.745), stdev = 0.176 - CI (99.9%): [4.252, 4.784] (assumes normal distribution) +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100ms": + 4.792 ±(99.9%) 0.017 ops/s [Average] + (min, avg, max) = (4.772, 4.792, 4.809), stdev = 0.011 + CI (99.9%): [4.775, 4.809] (assumes normal distribution) # JMH version: 1.21 @@ -103,27 +103,27 @@ Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10 # Timeout: 10 min per iteration # Threads: 1 thread, will synchronize iterations # Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000 +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10ms -# Run progress: 60.00% complete, ETA 00:04:04 +# Run progress: 60.00% complete, ETA 00:03:47 # Fork: 1 of 1 -# Warmup Iteration 1: 0.977 ops/s -Iteration 1: 0.992 ops/s -Iteration 2: 0.994 ops/s -Iteration 3: 0.992 ops/s -Iteration 4: 0.992 ops/s -Iteration 5: 0.995 ops/s -Iteration 6: 0.994 ops/s -Iteration 7: 0.994 ops/s -Iteration 8: 0.995 ops/s -Iteration 9: 0.995 ops/s -Iteration 10: 0.994 ops/s +# Warmup Iteration 1: 7.146 ops/s +Iteration 1: 7.938 ops/s +Iteration 2: 7.865 ops/s +Iteration 3: 8.426 ops/s +Iteration 4: 8.316 ops/s +Iteration 5: 8.212 ops/s +Iteration 6: 8.450 ops/s +Iteration 7: 8.294 ops/s +Iteration 8: 8.287 ops/s +Iteration 9: 8.248 ops/s +Iteration 10: 8.452 ops/s -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000": - 0.994 ±(99.9%) 0.002 ops/s [Average] - (min, avg, max) = (0.992, 0.994, 0.995), stdev = 0.001 - CI (99.9%): [0.992, 0.995] (assumes normal distribution) +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10ms": + 8.249 ±(99.9%) 0.305 ops/s [Average] + (min, avg, max) = (7.865, 8.249, 8.452), stdev = 0.202 + CI (99.9%): [7.944, 8.554] (assumes normal distribution) # JMH version: 1.21 @@ -135,30 +135,30 @@ Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10 # Timeout: 10 min per iteration # Threads: 1 thread, will synchronize iterations # Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat3000 +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1ms -# Run progress: 80.00% complete, ETA 00:02:02 +# Run progress: 80.00% complete, ETA 00:01:53 # Fork: 1 of 1 -# Warmup Iteration 1: 0.329 ops/s -Iteration 1: 0.332 ops/s -Iteration 2: 0.331 ops/s -Iteration 3: 0.332 ops/s -Iteration 4: 0.332 ops/s -Iteration 5: 0.332 ops/s -Iteration 6: 0.333 ops/s -Iteration 7: 0.333 ops/s -Iteration 8: 0.333 ops/s -Iteration 9: 0.332 ops/s -Iteration 10: 0.333 ops/s +# Warmup Iteration 1: 7.982 ops/s +Iteration 1: 9.347 ops/s +Iteration 2: 10.001 ops/s +Iteration 3: 9.958 ops/s +Iteration 4: 10.002 ops/s +Iteration 5: 10.018 ops/s +Iteration 6: 10.169 ops/s +Iteration 7: 10.057 ops/s +Iteration 8: 10.475 ops/s +Iteration 9: 10.230 ops/s +Iteration 10: 10.122 ops/s -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat3000": - 0.332 ±(99.9%) 0.001 ops/s [Average] - (min, avg, max) = (0.331, 0.332, 0.333), stdev = 0.001 - CI (99.9%): [0.332, 0.333] (assumes normal distribution) +Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1ms": + 10.038 ±(99.9%) 0.433 ops/s [Average] + (min, avg, max) = (9.347, 10.038, 10.475), stdev = 0.287 + CI (99.9%): [9.605, 10.471] (assumes normal distribution) -# Run complete. Total time: 00:10:31 +# Run complete. Total time: 00:09:29 REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial @@ -166,9 +166,9 @@ experiments, perform baseline and negative tests that provide experimental contr the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. Do not assume the numbers tell you what you want them to tell. -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit10ms thrpt 10 11.090 ± 1.883 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 0.993 ± 0.002 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 4.792 ± 0.017 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 8.249 ± 0.305 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 10.038 ± 0.433 ops/s diff --git a/benchmarks/consumer3.txt b/benchmarks/consumer3.txt deleted file mode 100644 index 173efd83..00000000 --- a/benchmarks/consumer3.txt +++ /dev/null @@ -1,46 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit - -# Run progress: 0.00% complete, ETA 00:03:40 -# Fork: 1 of 1 -# Warmup Iteration 1: 8.247 ops/s -Iteration 1: 10.887 ops/s -Iteration 2: 11.890 ops/s -Iteration 3: 9.795 ops/s -Iteration 4: 11.231 ops/s -Iteration 5: 12.354 ops/s -Iteration 6: 13.061 ops/s -Iteration 7: 13.098 ops/s -Iteration 8: 13.229 ops/s -Iteration 9: 12.850 ops/s -Iteration 10: 9.933 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit": - 11.833 ±(99.9%) 1.977 ops/s [Average] - (min, avg, max) = (9.795, 11.833, 13.229), stdev = 1.308 - CI (99.9%): [9.856, 13.809] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit - -# Run progress: 50.00% complete, ETA 00:02:02 -# Fork: 1 of 1 -# Warmup Iteration 1: \ No newline at end of file diff --git a/benchmarks/consumer4.txt b/benchmarks/consumer4.txt deleted file mode 100644 index 88f204b7..00000000 --- a/benchmarks/consumer4.txt +++ /dev/null @@ -1,14 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 1 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit - -# Run progress: 0.00% complete, ETA 00:00:20 -# Fork: 1 of 1 -# Warmup Iteration 1: \ No newline at end of file diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 1d1bf6e2..803ce756 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -1,26 +1,35 @@ ## RC7 + +### 1fork 1thread Benchmark Mode Cnt Score Error Units ConsumerBenchmark.monix_auto_commit thrpt 11.876 ops/s ConsumerBenchmark.monix_manual_commit thrpt 11.964 ops/s +### 1 fork 3 thrads +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit thrpt 10 15.305 ± 2.823 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 10 17.860 ± 1.691 ops/s -## RC8 - (Introduces PollHeartbeatRate) -1fork 1thread +## RC8 - (Introduces PollHeartbeatRate) +### 1fork 1thread --- -ConsumerBenchmark.monix_auto_commit thrpt 10 10.569 ± 2.096 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 10.320 ± 1.720 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 4.518 ± 0.266 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 0.994 ± 0.002 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat3000 thrpt 10 0.332 ± 0.001 ops/s +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit10ms thrpt 10 11.090 ± 1.883 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 0.993 ± 0.002 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 4.792 ± 0.017 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 8.249 ± 0.305 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 10.038 ± 0.433 ops/s --- -1fork 3thrads -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 10 16.270 ± 3.339 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1 thrpt 10 15.053 ± 0.959 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100 thrpt 10 9.525 ± 1.131 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000 thrpt 10 2.968 ± 0.010 ops/s +### 1 fork 3 threads +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monix_auto_commit10ms thrpt 10 17.266 ± 2.231 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 2.971 ± 0.009 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 9.477 ± 0.064 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 14.710 ± 1.660 ops/s +ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 15.494 ± 4.163 ops/s + diff --git a/benchmarks/results/consumer.txt b/benchmarks/results/consumer.txt deleted file mode 100644 index 5f5589fc..00000000 --- a/benchmarks/results/consumer.txt +++ /dev/null @@ -1,208 +0,0 @@ - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_1P_1RF - -# Run progress: 2.78% complete, ETA 00:05:06 -# Fork: 1 of 1 -# Warmup Iteration 1: 39.905 ops/s -Iteration 1: 57.642 ops/s -Iteration 2: 57.671 ops/s -Iteration 3: 44.793 ops/s -Iteration 4: 64.277 ops/s -Iteration 5: 63.957 ops/s -Iteration 6: 43.770 ops/s -Iteration 7: 56.371 ops/s -Iteration 8: 62.515 ops/s -Iteration 9: 49.956 ops/s -Iteration 10: 31.907 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_1P_1RF": - 53.286 ±(99.9%) 15.924 ops/s [Average] - (min, avg, max) = (31.907, 53.286, 64.277), stdev = 10.533 - CI (99.9%): [37.362, 69.210] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_2P_1RF - -# Run progress: 18.06% complete, ETA 00:09:43 -# Fork: 1 of 1 -# Warmup Iteration 1: 44.691 ops/s -Iteration 1: 39.595 ops/s -Iteration 2: 51.434 ops/s -Iteration 3: 43.905 ops/s -Iteration 4: 26.324 ops/s -Iteration 5: 28.480 ops/s -Iteration 6: 45.607 ops/s -Iteration 7: 56.213 ops/s -Iteration 8: 56.639 ops/s -Iteration 9: 57.454 ops/s -Iteration 10: 54.110 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_async_2P_1RF": - 45.976 ±(99.9%) 17.338 ops/s [Average] - (min, avg, max) = (26.324, 45.976, 57.454), stdev = 11.468 - CI (99.9%): [28.638, 63.314] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_1P_1RF - -# Run progress: 33.33% complete, ETA 00:08:16 -# Fork: 1 of 1 -# Warmup Iteration 1: 34.182 ops/s -Iteration 1: 48.052 ops/s -Iteration 2: 47.124 ops/s -Iteration 3: 44.219 ops/s -Iteration 4: 58.554 ops/s -Iteration 5: 66.543 ops/s -Iteration 6: 60.585 ops/s -Iteration 7: 46.291 ops/s -Iteration 8: 41.104 ops/s -Iteration 9: 57.524 ops/s -Iteration 10: 52.359 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_1P_1RF": - 52.235 ±(99.9%) 12.441 ops/s [Average] - (min, avg, max) = (41.104, 52.235, 66.543), stdev = 8.229 - CI (99.9%): [39.794, 64.677] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_2P_1RF - -# Run progress: 48.61% complete, ETA 00:06:28 -# Fork: 1 of 1 -# Warmup Iteration 1: 40.188 ops/s -Iteration 1: 48.015 ops/s -Iteration 2: 60.841 ops/s -Iteration 3: 41.521 ops/s -Iteration 4: 45.350 ops/s -Iteration 5: 47.954 ops/s -Iteration 6: 62.838 ops/s -Iteration 7: 49.424 ops/s -Iteration 8: 57.602 ops/s -Iteration 9: 45.920 ops/s -Iteration 10: 36.525 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.auto_commit_sync_2P_1RF": - 49.599 ±(99.9%) 12.737 ops/s [Average] - (min, avg, max) = (36.525, 49.599, 62.838), stdev = 8.424 - CI (99.9%): [36.863, 62.336] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_1P_1RF - -# Run progress: 63.89% complete, ETA 00:04:35 -# Fork: 1 of 1 -# Warmup Iteration 1: 29.430 ops/s -Iteration 1: 46.280 ops/s -Iteration 2: 53.870 ops/s -Iteration 3: 34.727 ops/s -Iteration 4: 34.179 ops/s -Iteration 5: 37.058 ops/s -Iteration 6: 48.568 ops/s -Iteration 7: 32.917 ops/s -Iteration 8: 44.958 ops/s -Iteration 9: 35.192 ops/s -Iteration 10: 40.836 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_1P_1RF": - 40.858 ±(99.9%) 10.888 ops/s [Average] - (min, avg, max) = (32.917, 40.858, 53.870), stdev = 7.202 - CI (99.9%): [29.971, 51.746] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 4 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_2P_1RF - -# Run progress: 79.17% complete, ETA 00:02:41 -# Fork: 1 of 1 -# Warmup Iteration 1: 31.783 ops/s -Iteration 1: 40.728 ops/s -Iteration 2: 46.217 ops/s -Iteration 3: 32.231 ops/s -Iteration 4: 49.542 ops/s -Iteration 5: 43.252 ops/s -Iteration 6: 54.555 ops/s -Iteration 7: 54.511 ops/s -Iteration 8: 46.290 ops/s -Iteration 9: 40.890 ops/s -Iteration 10: 40.730 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.manual_commit_2P_1RF": - 44.895 ±(99.9%) 10.388 ops/s [Average] - (min, avg, max) = (32.231, 44.895, 54.555), stdev = 6.871 - CI (99.9%): [34.507, 55.283] (assumes normal distribution) - - -# Run complete. Total time: 00:12:33 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.auto_commit_async_1P_1RF thrpt 10 53.286 ± 15.924 ops/s -ConsumerBenchmark.auto_commit_async_2P_1RF thrpt 10 45.976 ± 17.338 ops/s -ConsumerBenchmark.auto_commit_sync_1P_1RF thrpt 10 52.235 ± 12.441 ops/s -ConsumerBenchmark.auto_commit_sync_2P_1RF thrpt 10 49.599 ± 12.737 ops/s -ConsumerBenchmark.manual_commit_1P_1RF thrpt 10 40.858 ± 10.888 ops/s -ConsumerBenchmark.manual_commit_2P_1RF thrpt 10 44.895 ± 10.388 ops/s diff --git a/benchmarks/results/producer.txt b/benchmarks/results/producer.txt deleted file mode 100644 index 96fcee1c..00000000 --- a/benchmarks/results/producer.txt +++ /dev/null @@ -1,75 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_1P_1RF - -# Run progress: 0.00% complete, ETA 00:03:40 -# Fork: 1 of 1 -# Warmup Iteration 1: 0.736 ops/s -Iteration 1: 0.991 ops/s -Iteration 2: 0.958 ops/s -Iteration 3: 0.891 ops/s -Iteration 4: 0.914 ops/s -Iteration 5: 0.973 ops/s -Iteration 6: 1.012 ops/s -Iteration 7: 0.860 ops/s -Iteration 8: 0.905 ops/s -Iteration 9: 0.871 ops/s -Iteration 10: 1.006 ops/s - - -Result "monix.kafka.benchmarks.ProducerBenchmark.monix_1P_1RF": - 0.938 ±(99.9%) 0.086 ops/s [Average] - (min, avg, max) = (0.860, 0.938, 1.012), stdev = 0.057 - CI (99.9%): [0.852, 1.024] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_2P_1RF - -# Run progress: 50.00% complete, ETA 00:01:57 -# Fork: 1 of 1 -# Warmup Iteration 1: 0.875 ops/s -Iteration 1: 1.027 ops/s -Iteration 2: 1.038 ops/s -Iteration 3: 0.993 ops/s -Iteration 4: 0.998 ops/s -Iteration 5: 1.018 ops/s -Iteration 6: 0.985 ops/s -Iteration 7: 1.028 ops/s -Iteration 8: 0.982 ops/s -Iteration 9: 1.010 ops/s -Iteration 10: 0.917 ops/s - - -Result "monix.kafka.benchmarks.ProducerBenchmark.monix_2P_1RF": - 1.000 ±(99.9%) 0.052 ops/s [Average] - (min, avg, max) = (0.917, 1.000, 1.038), stdev = 0.035 - CI (99.9%): [0.947, 1.052] (assumes normal distribution) - - -# Run complete. Total time: 00:03:54 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ProducerBenchmark.monix_1P_1RF thrpt 10 0.938 ± 0.086 ops/s -ProducerBenchmark.monix_2P_1RF thrpt 10 1.000 ± 0.052 ops/s diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index ceafc201..1910d799 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -19,104 +19,82 @@ import scala.concurrent.duration._ @Threads(3) class ConsumerBenchmark extends MonixFixture { - var size: Int = 1000 - var maxPollRecords: Int = 5 + var totalRecords: Int = 1500 + val consumedRecords = 1000 + var maxPollRecords: Int = 1 // preparing test data - Observable - .from(0 to size) - .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) - .bufferTumbling(size) - .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) - .runSyncUnsafe() - - //@Benchmark - //def monix_manual_commit(): Unit = { - // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - // .withPollHeartBeatRate(100.millis) - - // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) - // .mapEvalF(_.committableOffset.commitAsync()) - // .take(100) - // .headL - // .runSyncUnsafe() - //} - + @Setup + def setup(): Unit = { + Observable + .from(0 to totalRecords) + .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) + .bufferTumbling(totalRecords) + .consumeWith(KafkaProducerSink(producerConf.copy(monixSinkParallelism = 10), io)) + .runSyncUnsafe() + } @Benchmark - def monix_auto_commit(): Unit = { + def monix_auto_commit10ms(): Unit = { val conf = consumerConf.value().copy( maxPollRecords = maxPollRecords, observableCommitType = ObservableCommitType.Async) .withPollHeartBeatRate(10.millis) KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) - .take(100) + .take(consumedRecords) .headL .runSyncUnsafe() } @Benchmark - def monix_manual_commit_heartbeat1(): Unit = { + def monix_manual_commit_heartbeat1ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(1.millis) KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) .mapEvalF(_.committableOffset.commitAsync()) - .take(100) + .take(consumedRecords) .headL .runSyncUnsafe() } - //@Benchmark - //def monix_manual_commit_heartbeat10(): Unit = { - // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - // .withPollHeartBeatRate(10.millis) + @Benchmark + def monix_manual_commit_heartbeat10ms(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(10.millis) - // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) - // .mapEvalF(_.committableOffset.commitAsync()) - // .take(100) - // .headL - // .runSyncUnsafe() - //} + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(consumedRecords) + .headL + .runSyncUnsafe() + } @Benchmark - def monix_manual_commit_heartbeat100(): Unit = { + def monix_manual_commit_heartbeat100ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(100.millis) KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) .mapEvalF(_.committableOffset.commitAsync()) - .take(100) + .take(consumedRecords) .headL .runSyncUnsafe() } @Benchmark - def monix_manual_commit_heartbeat1000(): Unit = { + def monix_manual_commit_heartbeat1000ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(1000.millis) KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) .mapEvalF(_.committableOffset.commitAsync()) - .take(100) + .take(consumedRecords) .headL .runSyncUnsafe() } - //@Benchmark - //def monix_manual_commit_heartbeat3000(): Unit = { - // val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) - // .withPollHeartBeatRate(3000.millis) - - // KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) - // .mapEvalF(_.committableOffset.commitAsync()) - // .take(100) - // .headL - // .runSyncUnsafe() - //} - - } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 76f35b2c..0795c486 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -29,6 +29,7 @@ import java.time.Duration import scala.jdk.CollectionConverters._ import scala.concurrent.blocking import scala.util.matching.Regex +import scala.concurrent.duration._ /** Exposes an `Observable` that consumes a Kafka stream by * means of a Kafka Consumer client. @@ -99,28 +100,31 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg * This allows producer process commit calls and also keeps consumer alive even * with long batch processing. * - * If polling fails the error is reported to the subscriber through the scheduler. + * If polling fails the error is reported to the subscriber through the scheduler. * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { - Task.sleep(config.pollHeartbeatRate) *> - Task.evalAsync { - if (!isAcked) { - consumer.synchronized { - // needed in order to ensure that the consummer assignment - // is paused, meaning that no messages will get lost - val assignment = consumer.assignment() - consumer.pause(assignment) - val records = blocking(consumer.poll(Duration.ZERO)) - if (!records.isEmpty) { - val errorMsg = s"Received ${records.count()} unexpected messages." - throw new IllegalStateException(errorMsg) - } - } - } - }.onErrorHandle(ex => scheduler.reportFailure(ex) - ) + Task.sleep(config.pollHeartbeatRate) >> + Task.eval { + if (!isAcked) { + consumer.synchronized { + // needed in order to ensure that the consummer assignment + // is paused, meaning that no messages will get lost + val assignment = consumer.assignment() + consumer.pause(assignment) + val records = blocking(consumer.poll(Duration.ZERO)) + if (!records.isEmpty) { + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) + } + } + } + } + .onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.seconds) + } } From fc1cc4aafc864ac18449080cce5dd2d87d0f819b Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 09:40:35 +0100 Subject: [PATCH 30/61] Set kafka-clients dependency to 1.1.1 --- benchmarks/consumer-t1.txt | 174 ------------------ benchmarks/producer_results.txt | 61 ------ benchmarks/readme.md | 29 --- benchmarks/results/consumer.md | 11 +- benchmarks/results/producer.md | 11 ++ benchmarks/results/sink.txt | 141 -------------- benchmarks/src/main/resources/commands | 9 +- build.sbt | 4 +- .../monix/kafka/KafkaConsumerConfig.scala | 7 +- .../monix/kafka/KafkaConsumerObservable.scala | 6 +- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../kafka/MonixKafkaTopicRegexTest.scala | 4 +- .../scala/monix/kafka/PollHeartBeatTest.scala | 4 +- 14 files changed, 39 insertions(+), 426 deletions(-) delete mode 100644 benchmarks/consumer-t1.txt delete mode 100644 benchmarks/producer_results.txt delete mode 100644 benchmarks/readme.md create mode 100644 benchmarks/results/producer.md delete mode 100644 benchmarks/results/sink.txt diff --git a/benchmarks/consumer-t1.txt b/benchmarks/consumer-t1.txt deleted file mode 100644 index 0a84cdd1..00000000 --- a/benchmarks/consumer-t1.txt +++ /dev/null @@ -1,174 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit10ms - -# Run progress: 0.00% complete, ETA 00:09:10 -# Fork: 1 of 1 -# Warmup Iteration 1: 8.357 ops/s -Iteration 1: 9.274 ops/s -Iteration 2: 10.391 ops/s -Iteration 3: 10.828 ops/s -Iteration 4: 10.854 ops/s -Iteration 5: 11.172 ops/s -Iteration 6: 10.864 ops/s -Iteration 7: 9.852 ops/s -Iteration 8: 11.435 ops/s -Iteration 9: 12.843 ops/s -Iteration 10: 13.383 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_auto_commit10ms": - 11.090 ±(99.9%) 1.883 ops/s [Average] - (min, avg, max) = (9.274, 11.090, 13.383), stdev = 1.245 - CI (99.9%): [9.207, 12.972] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000ms - -# Run progress: 20.00% complete, ETA 00:07:35 -# Fork: 1 of 1 -# Warmup Iteration 1: 0.979 ops/s -Iteration 1: 0.992 ops/s -Iteration 2: 0.991 ops/s -Iteration 3: 0.992 ops/s -Iteration 4: 0.994 ops/s -Iteration 5: 0.993 ops/s -Iteration 6: 0.995 ops/s -Iteration 7: 0.993 ops/s -Iteration 8: 0.994 ops/s -Iteration 9: 0.994 ops/s -Iteration 10: 0.994 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1000ms": - 0.993 ±(99.9%) 0.002 ops/s [Average] - (min, avg, max) = (0.991, 0.993, 0.995), stdev = 0.001 - CI (99.9%): [0.991, 0.995] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100ms - -# Run progress: 40.00% complete, ETA 00:05:41 -# Fork: 1 of 1 -# Warmup Iteration 1: 4.706 ops/s -Iteration 1: 4.789 ops/s -Iteration 2: 4.786 ops/s -Iteration 3: 4.772 ops/s -Iteration 4: 4.797 ops/s -Iteration 5: 4.802 ops/s -Iteration 6: 4.790 ops/s -Iteration 7: 4.799 ops/s -Iteration 8: 4.809 ops/s -Iteration 9: 4.794 ops/s -Iteration 10: 4.778 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat100ms": - 4.792 ±(99.9%) 0.017 ops/s [Average] - (min, avg, max) = (4.772, 4.792, 4.809), stdev = 0.011 - CI (99.9%): [4.775, 4.809] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10ms - -# Run progress: 60.00% complete, ETA 00:03:47 -# Fork: 1 of 1 -# Warmup Iteration 1: 7.146 ops/s -Iteration 1: 7.938 ops/s -Iteration 2: 7.865 ops/s -Iteration 3: 8.426 ops/s -Iteration 4: 8.316 ops/s -Iteration 5: 8.212 ops/s -Iteration 6: 8.450 ops/s -Iteration 7: 8.294 ops/s -Iteration 8: 8.287 ops/s -Iteration 9: 8.248 ops/s -Iteration 10: 8.452 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat10ms": - 8.249 ±(99.9%) 0.305 ops/s [Average] - (min, avg, max) = (7.865, 8.249, 8.452), stdev = 0.202 - CI (99.9%): [7.944, 8.554] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1ms - -# Run progress: 80.00% complete, ETA 00:01:53 -# Fork: 1 of 1 -# Warmup Iteration 1: 7.982 ops/s -Iteration 1: 9.347 ops/s -Iteration 2: 10.001 ops/s -Iteration 3: 9.958 ops/s -Iteration 4: 10.002 ops/s -Iteration 5: 10.018 ops/s -Iteration 6: 10.169 ops/s -Iteration 7: 10.057 ops/s -Iteration 8: 10.475 ops/s -Iteration 9: 10.230 ops/s -Iteration 10: 10.122 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monix_manual_commit_heartbeat1ms": - 10.038 ±(99.9%) 0.433 ops/s [Average] - (min, avg, max) = (9.347, 10.038, 10.475), stdev = 0.287 - CI (99.9%): [9.605, 10.471] (assumes normal distribution) - - -# Run complete. Total time: 00:09:29 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit10ms thrpt 10 11.090 ± 1.883 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 0.993 ± 0.002 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 4.792 ± 0.017 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 8.249 ± 0.305 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 10.038 ± 0.433 ops/s diff --git a/benchmarks/producer_results.txt b/benchmarks/producer_results.txt deleted file mode 100644 index 5975d682..00000000 --- a/benchmarks/producer_results.txt +++ /dev/null @@ -1,61 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 3 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_single_producer - -# Run progress: 0.00% complete, ETA 00:01:20 -# Fork: 1 of 1 -# Warmup Iteration 1: 0.441 ops/s -Iteration 1: 0.479 ops/s -Iteration 2: 0.546 ops/s -Iteration 3: 0.487 ops/s - - -Result "monix.kafka.benchmarks.ProducerBenchmark.monix_single_producer": - 0.504 ±(99.9%) 0.668 ops/s [Average] - (min, avg, max) = (0.479, 0.504, 0.546), stdev = 0.037 - CI (99.9%): [≈ 0, 1.172] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 3 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerBenchmark.monix_sink_producer - -# Run progress: 50.00% complete, ETA 00:00:48 -# Fork: 1 of 1 -# Warmup Iteration 1: 1.608 ops/s -Iteration 1: 2.134 ops/s -Iteration 2: 1.908 ops/s -Iteration 3: 1.540 ops/s - - -Result "monix.kafka.benchmarks.ProducerBenchmark.monix_sink_producer": - 1.861 ±(99.9%) 5.475 ops/s [Average] - (min, avg, max) = (1.540, 1.861, 2.134), stdev = 0.300 - CI (99.9%): [≈ 0, 7.336] (assumes normal distribution) - - -# Run complete. Total time: 00:01:31 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ProducerBenchmark.monix_single_producer thrpt 3 0.504 ± 0.668 ops/s -ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s diff --git a/benchmarks/readme.md b/benchmarks/readme.md deleted file mode 100644 index 57f62f99..00000000 --- a/benchmarks/readme.md +++ /dev/null @@ -1,29 +0,0 @@ -###Monix Kafka Benchmarks - -This document explains the approach followed to benchmark monix-kafka. - - Ideally, a Kafka performance benchmark should happen under some long stress test in a real Kafka cluster, -although, our hardware limitations we have to stick to running simpler basic tests that proves application performance on - a docker container. - - - -So these test will be executed using the same configurations () - -The benchmark will focus on the most basic `consumer` and `procucer` scenarios. - -Although Kafka is very configurable by nature, our benchmark will use the kafka default properties for [consumer](https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html) -and [producer](https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html). - -On the other hand, assume that all the used topics will have 2 partitions, and 1 replication factor. - - -## Producer benchmarks - -This section includes benchmarks for single and sink producers. -Although some libraries like `alpakka-kafka` do not expose methods for producing single record, but only for sink. - - -## Consumer benchmark -The consumer benchmark covers the manual and auto commit consumer implementations of the different libraries. -The manual commit will also cover producing committing back the consumed offsets. \ No newline at end of file diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 803ce756..f83d9208 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -3,13 +3,13 @@ ### 1fork 1thread Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 11.876 ops/s -ConsumerBenchmark.monix_manual_commit thrpt 11.964 ops/s +ConsumerBenchmark.monix_auto_commit thrpt 10 11.912 ± 0.617 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 10 11.519 ± 2.247 ops/s -### 1 fork 3 thrads +### 1 fork 3 threads Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 10 15.305 ± 2.823 ops/s -ConsumerBenchmark.monix_manual_commit thrpt 10 17.860 ± 1.691 ops/s +ConsumerBenchmark.monix_auto_commit thrpt 10 16.186 ± 0.920 ops/s +ConsumerBenchmark.monix_manual_commit thrpt 10 16.319 ± 1.465 ops/s ## RC8 - (Introduces PollHeartbeatRate) @@ -34,3 +34,4 @@ ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 15.494 ± 4.1 + diff --git a/benchmarks/results/producer.md b/benchmarks/results/producer.md new file mode 100644 index 00000000..e47b5a16 --- /dev/null +++ b/benchmarks/results/producer.md @@ -0,0 +1,11 @@ + +## RC7 + +Benchmark Mode Cnt Score Error Units +ProducerBenchmark.monix_single_producer thrpt 3 0.504 ± 0.668 ops/s +ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s + +## RC8 + + + diff --git a/benchmarks/results/sink.txt b/benchmarks/results/sink.txt deleted file mode 100644 index ee965a2f..00000000 --- a/benchmarks/results/sink.txt +++ /dev/null @@ -1,141 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_1P_1RF - -# Run progress: 21.43% complete, ETA 00:00:08 -# Fork: 1 of 1 -# Warmup Iteration 1: 9.375 ops/s -Iteration 1: 13.510 ops/s -Iteration 2: 18.241 ops/s -Iteration 3: 19.142 ops/s -Iteration 4: 19.561 ops/s -Iteration 5: 19.037 ops/s -Iteration 6: 16.952 ops/s -Iteration 7: 19.231 ops/s -Iteration 8: 17.509 ops/s -Iteration 9: 17.984 ops/s -Iteration 10: 19.508 ops/s - - -Result "monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_1P_1RF": - 18.068 ±(99.9%) 2.766 ops/s [Average] - (min, avg, max) = (13.510, 18.068, 19.561), stdev = 1.829 - CI (99.9%): [15.302, 20.833] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_2P_1RF - -# Run progress: 41.07% complete, ETA 00:02:46 -# Fork: 1 of 1 -# Warmup Iteration 1: 13.733 ops/s -Iteration 1: 19.880 ops/s -Iteration 2: 20.386 ops/s -Iteration 3: 18.965 ops/s -Iteration 4: 19.587 ops/s -Iteration 5: 16.391 ops/s -Iteration 6: 13.518 ops/s -Iteration 7: 16.592 ops/s -Iteration 8: 17.053 ops/s -Iteration 9: 17.305 ops/s -Iteration 10: 13.637 ops/s - - -Result "monix.kafka.benchmarks.ProducerSinkBenchmark.parallel_2P_1RF": - 17.331 ±(99.9%) 3.679 ops/s [Average] - (min, avg, max) = (13.518, 17.331, 20.386), stdev = 2.433 - CI (99.9%): [13.653, 21.010] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.sync_2P_1RF - -# Run progress: 60.71% complete, ETA 00:02:28 -# Fork: 1 of 1 -# Warmup Iteration 1: 10.699 ops/s -Iteration 1: 15.467 ops/s -Iteration 2: 15.411 ops/s -Iteration 3: 16.970 ops/s -Iteration 4: 17.399 ops/s -Iteration 5: 16.215 ops/s -Iteration 6: 18.005 ops/s -Iteration 7: 19.683 ops/s -Iteration 8: 17.790 ops/s -Iteration 9: 21.806 ops/s -Iteration 10: 14.330 ops/s - - -Result "monix.kafka.benchmarks.ProducerSinkBenchmark.sync_2P_1RF": - 17.308 ±(99.9%) 3.334 ops/s [Average] - (min, avg, max) = (14.330, 17.308, 21.806), stdev = 2.205 - CI (99.9%): [13.974, 20.641] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ProducerSinkBenchmark.sync_sync_1P_1RF - -# Run progress: 80.36% complete, ETA 00:01:24 -# Fork: 1 of 1 -# Warmup Iteration 1: 11.903 ops/s -Iteration 1: 17.489 ops/s -Iteration 2: 16.606 ops/s -Iteration 3: 13.364 ops/s -Iteration 4: 14.373 ops/s -Iteration 5: 11.613 ops/s -Iteration 6: 15.312 ops/s -Iteration 7: 18.608 ops/s -Iteration 8: 17.567 ops/s -Iteration 9: 10.345 ops/s -Iteration 10: 15.375 ops/s - - -Result "monix.kafka.benchmarks.ProducerSinkBenchmark.sync_sync_1P_1RF": - 15.065 ±(99.9%) 4.056 ops/s [Average] - (min, avg, max) = (10.345, 15.065, 18.608), stdev = 2.683 - CI (99.9%): [11.009, 19.122] (assumes normal distribution) - - -# Run complete. Total time: 00:07:40 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ProducerSinkBenchmark.parallel_1P_1RF thrpt 10 18.068 ± 2.766 ops/s -ProducerSinkBenchmark.parallel_2P_1RF thrpt 10 17.331 ± 3.679 ops/s -ProducerSinkBenchmark.sync_2P_1RF thrpt 10 17.308 ± 3.334 ops/s -ProducerSinkBenchmark.sync_sync_1P_1RF thrpt 10 15.065 ± 4.056 ops/s diff --git a/benchmarks/src/main/resources/commands b/benchmarks/src/main/resources/commands index ab6e54e3..bc87b061 100644 --- a/benchmarks/src/main/resources/commands +++ b/benchmarks/src/main/resources/commands @@ -1,12 +1,15 @@ -kafka-console-consumer --bootstrap-server broker:9092 --topic topic2 --from-beginning + +# list topics kafka-console-producer --broker-list broker:9092 --topic topic_1P_1RF +# consume from beginning docker exec -it broker kafka-console-consumer --bootstrap-server broker:9092 --topic topic2 --from-beginning docker exec -it broker /bin/bash docker stop $(docker ps -a -q) -#from root directory +# run producer benchmark sbt 'benchmarks/jmh:run -o producer_results.txt monix.kafka.benchmarks.ProducerBenchmark' - sbt 'benchmarks/jmh:run -o consumer_results.txt monix.kafka.benchmarks.ConsumerBenchmark' \ No newline at end of file +# run consumer benchmark +sbt 'benchmarks/jmh:run -o consumer_results.txt monix.kafka.benchmarks.ConsumerBenchmark' \ No newline at end of file diff --git a/build.sbt b/build.sbt index 0d7ee278..3e385bca 100644 --- a/build.sbt +++ b/build.sbt @@ -214,10 +214,10 @@ lazy val kafka1x = project.in(file("kafka-1.0.x")) .settings( name := "monix-kafka-1x", libraryDependencies ++= { - if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "2.0.0" % "test" exclude ("log4j", "log4j")) + if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.1" % "test" exclude ("log4j", "log4j")) else Seq.empty[ModuleID] }, - libraryDependencies += "org.apache.kafka" % "kafka-clients" % "2.0.0" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") + libraryDependencies += "org.apache.kafka" % "kafka-clients" % "1.1.1" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") ) lazy val kafka11 = project.in(file("kafka-0.11.x")) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index f0c0ed85..7f8033bf 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -18,8 +18,8 @@ package monix.kafka import java.io.File import java.util.Properties - import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.internal.InternalApi import monix.kafka.config._ import scala.jdk.CollectionConverters._ @@ -304,6 +304,11 @@ case class KafkaConsumerConfig( private[kafka] var pollHeartbeatRate: FiniteDuration = 100.millis + /** + * + * @return + */ + @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { pollHeartbeatRate = interval this diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 0795c486..dba75d9f 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -25,7 +25,6 @@ import monix.reactive.Observable import monix.reactive.observers.Subscriber import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer} -import java.time.Duration import scala.jdk.CollectionConverters._ import scala.concurrent.blocking import scala.util.matching.Regex @@ -74,7 +73,6 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void - //runLoop(c, out).void } { consumer => // Forced asynchronous boundary Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess @@ -110,10 +108,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg if (!isAcked) { consumer.synchronized { // needed in order to ensure that the consummer assignment - // is paused, meaning that no messages will get lost + // is paused, meaning that no messages will get lost. val assignment = consumer.assignment() consumer.pause(assignment) - val records = blocking(consumer.poll(Duration.ZERO)) + val records = blocking(consumer.poll(0)) if (!records.isEmpty) { val errorMsg = s"Received ${records.count()} unexpected messages." throw new IllegalStateException(errorMsg) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 2c9ca8f8..719863f0 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -72,7 +72,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( if (cancelable.isCanceled) Stop else { consumer.resume(assignment) - val next = blocking(consumer.poll(java.time.Duration.ofMillis(pollTimeoutMillis))) + val next = blocking(consumer.poll(0)) consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 4407c920..81630129 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -88,7 +88,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( try consumer.synchronized { val assignment = consumer.assignment() consumer.resume(assignment) - val next = blocking(consumer.poll(java.time.Duration.ofMillis(pollTimeoutMillis))) + val next = blocking(consumer.poll(0)) consumer.pause(assignment) val result = next.asScala.map { record => CommittableMessage( diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index b29c3e56..5d4713d3 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -24,7 +24,7 @@ import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.scalatest.FunSuite -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Await import scala.concurrent.duration._ @@ -56,7 +56,7 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { val send = producer.send(topicMatchingRegex, "my-message") Await.result(send.runToFuture, 30.seconds) - val records = consumer.poll(java.time.Duration.ofMillis(10.seconds.toMillis)).asScala.map(_.value()).toList + val records = consumer.poll(0).asScala.map(_.value()).toList assert(records === List("my-message")) } finally { Await.result(producer.close().runToFuture, Duration.Inf) diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index ebe60519..2843c4bf 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -86,7 +86,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync().guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value())))) + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } @@ -177,7 +177,7 @@ class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .doOnNextF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()).guarantee(Task.eval(println("Consumed message: " + committableMessage.record.value()))) + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) Task.sleep(downstreamLatency) *> manualCommit } .take(totalRecords) From 6a54a9caf9590c000ee1037347a76e3e5fd592f6 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 09:51:51 +0100 Subject: [PATCH 31/61] RC8 - ProducerBenchmarks --- benchmarks/results/producer.md | 8 ++++++-- benchmarks/start-kafka-cluster.sh | 5 +---- build.sbt | 5 ----- 3 files changed, 7 insertions(+), 11 deletions(-) diff --git a/benchmarks/results/producer.md b/benchmarks/results/producer.md index e47b5a16..4b2122b3 100644 --- a/benchmarks/results/producer.md +++ b/benchmarks/results/producer.md @@ -1,11 +1,15 @@ ## RC7 +### 10iterations 1fork 1thread Benchmark Mode Cnt Score Error Units ProducerBenchmark.monix_single_producer thrpt 3 0.504 ± 0.668 ops/s ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s -## RC8 - +## RC8 +### 10iterations 1fork 3threads +Benchmark Mode Cnt Score Error Units +ProducerBenchmark.monix_single_producer thrpt 10 0.981 ± 0.202 ops/s +ProducerBenchmark.monix_sink_producer thrpt 10 3.241 ± 0.191 ops/s diff --git a/benchmarks/start-kafka-cluster.sh b/benchmarks/start-kafka-cluster.sh index 46f43e90..e20ff4fa 100644 --- a/benchmarks/start-kafka-cluster.sh +++ b/benchmarks/start-kafka-cluster.sh @@ -18,7 +18,4 @@ docker ps sleep 15 -create_topic monix_topic 2 1 -create_topic akka_topic 2 1 -create_topic fs2_topic 2 1 -create_topic zio_topic 2 1 \ No newline at end of file +create_topic monix_topic 2 1 \ No newline at end of file diff --git a/build.sbt b/build.sbt index 3e385bca..71c8814a 100644 --- a/build.sbt +++ b/build.sbt @@ -265,11 +265,6 @@ lazy val benchmarks = project.in(file("benchmarks")) name := "benchmarks", organization := "io.monix", scalaVersion := "2.12.10", - libraryDependencies ++= Seq( - "dev.zio" %% "zio-streams" % "1.0.2", - "dev.zio" %% "zio-kafka" % "0.13.0", - "com.github.fd4s" %% "fs2-kafka" % "1.3.1" - ) ) .enablePlugins(JmhPlugin) .aggregate(kafka1x) From 450b086135974acd1cfae0596948534c3c2bac6c Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 10:07:20 +0100 Subject: [PATCH 32/61] Provided scala compat --- build.sbt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 71c8814a..ae63539c 100644 --- a/build.sbt +++ b/build.sbt @@ -195,7 +195,7 @@ lazy val commonDependencies = Seq( "com.typesafe.scala-logging" %% "scala-logging" % "3.9.2", "com.typesafe" % "config" % "1.4.1", "org.slf4j" % "log4j-over-slf4j" % "1.7.30", - "org.scala-lang.modules" %% "scala-collection-compat" % "2.3.1", + "org.scala-lang.modules" %% "scala-collection-compat" % "2.3.2" % "provided;optional", // For testing ... "ch.qos.logback" % "logback-classic" % "1.2.3" % "test", "org.scalatest" %% "scalatest" % "3.0.9" % "test", @@ -265,6 +265,7 @@ lazy val benchmarks = project.in(file("benchmarks")) name := "benchmarks", organization := "io.monix", scalaVersion := "2.12.10", + libraryDependencies ++= Seq("org.scala-lang.modules" %% "scala-collection-compat" % "2.3.2") ) .enablePlugins(JmhPlugin) .aggregate(kafka1x) From 20416f07bb73ed8c8c107e1d2285450a9abc34d4 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 10:19:08 +0100 Subject: [PATCH 33/61] Downgrade scalacheck dependency --- .../monix/kafka/benchmarks/BaseFixture.scala | 3 --- .../monix/kafka/benchmarks/MonixFixture.scala | 17 ----------------- build.sbt | 2 +- 3 files changed, 1 insertion(+), 21 deletions(-) diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala index 7d50913e..932eb48b 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala @@ -11,8 +11,5 @@ trait BaseFixture { // topic names val monixTopic = "monix_topic" - val akkaTopic = "akka_topic" - val fs2Topic = "fs2_topic" - val zioTopic = "zio_topic" } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala index 205791df..e99d9a6a 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/MonixFixture.scala @@ -1,14 +1,9 @@ package monix.kafka.benchmarks -import cats.effect.IO import monix.eval.Coeval import monix.execution.Scheduler import monix.kafka.config.{AutoOffsetReset => MonixAutoOffsetReset} import monix.kafka.{KafkaConsumerConfig, KafkaProducerConfig} -//import monix.reactive.Observable -//import org.apache.kafka.clients.consumer.ConsumerRecord -//import org.apache.kafka.clients.producer.ProducerRecord -import fs2.kafka._ trait MonixFixture extends BaseFixture{ @@ -29,16 +24,4 @@ trait MonixFixture extends BaseFixture{ autoOffsetReset = MonixAutoOffsetReset.Earliest )) - val fs2ConsumerSettings: ConsumerSettings[IO, String, String] = - ConsumerSettings[IO, String, String] - .withAutoOffsetReset(AutoOffsetReset.Earliest) - .withBootstrapServers("localhost:9092") - .withGroupId("group") - - val fs2ProducerSettings: ProducerSettings[IO, String, String] = - ProducerSettings[IO, String, String] - .withBootstrapServers("localhost:9092") - - - } diff --git a/build.sbt b/build.sbt index ae63539c..b1685fe6 100644 --- a/build.sbt +++ b/build.sbt @@ -199,7 +199,7 @@ lazy val commonDependencies = Seq( // For testing ... "ch.qos.logback" % "logback-classic" % "1.2.3" % "test", "org.scalatest" %% "scalatest" % "3.0.9" % "test", - "org.scalacheck" %% "scalacheck" % "1.15.3" % "test") + "org.scalacheck" %% "scalacheck" % "1.15.2" % "test") ) lazy val monixKafka = project.in(file(".")) From de7922158aaff309e9b8608737c3d318529dccfd Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 15:43:55 +0100 Subject: [PATCH 34/61] Clean up --- benchmarks/results/consumer.md | 6 - benchmarks/results/producer.md | 1 - .../monix/kafka/benchmarks/ProducerSpec.scala | 8 +- .../monix/kafka/KafkaConsumerConfig.scala | 2 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 157 ++++++++++++++++++ .../scala/monix/kafka/PollHeartBeatTest.scala | 1 - 6 files changed, 162 insertions(+), 13 deletions(-) create mode 100644 kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index f83d9208..41bcc231 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -29,9 +29,3 @@ ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 2.971 ± 0.0 ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 9.477 ± 0.064 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 14.710 ± 1.660 ops/s ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 15.494 ± 4.163 ops/s - - - - - - diff --git a/benchmarks/results/producer.md b/benchmarks/results/producer.md index 4b2122b3..04e68cf0 100644 --- a/benchmarks/results/producer.md +++ b/benchmarks/results/producer.md @@ -12,4 +12,3 @@ ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s Benchmark Mode Cnt Score Error Units ProducerBenchmark.monix_single_producer thrpt 10 0.981 ± 0.202 ops/s ProducerBenchmark.monix_sink_producer thrpt 10 3.241 ± 0.191 ops/s - diff --git a/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala b/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala index e621f6e4..c5c292ca 100644 --- a/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala +++ b/benchmarks/src/test/scala/monix/kafka/benchmarks/ProducerSpec.scala @@ -7,11 +7,11 @@ import org.scalatest.{FlatSpec, Matchers} class ProducerSpec extends FlatSpec with MonixFixture with Matchers { val producer = KafkaProducer[String, String](producerConf, global) -/* - s"Monix ${topic_producer_1P_1RF}" should "exist befor running Producer Benchmark" in { - val t = producer.send(topic = topic_producer_1P_1RF, "test") + + s"Monix ${monixTopic}" should "exist befor running Producer Benchmark" in { + val t = producer.send(topic = monixTopic, "test") t.runSyncUnsafe().isDefined shouldBe true } -*/ + } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 7f8033bf..06b6bc7e 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -302,7 +302,7 @@ case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 100.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 10.millis /** * diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala new file mode 100644 index 00000000..3ead708e --- /dev/null +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -0,0 +1,157 @@ +/* + * Copyright (c) 2014-2019 by its authors. Some rights reserved. + * See the project homepage at: https://github.com/monix/monix-kafka + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package monix.kafka + +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.producer.ProducerRecord +import org.scalatest.FunSuite + +import scala.collection.JavaConverters._ +import scala.concurrent.Await +import scala.concurrent.duration._ + +class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { + val topicName = "monix-kafka-tests" + + val producerCfg = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + test("publish one message when subscribed to topics list") { + + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + + val consumerTask = + KafkaConsumerObservable.createConsumer[String, String](consumerCfg, List(topicName)).executeOn(io) + val consumer = Await.result(consumerTask.runToFuture, 60.seconds) + + try { + // Publishing one message + val send = producer.send(topicName, "my-message") + Await.result(send.runToFuture, 30.seconds) + + val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList + assert(records === List("my-message")) + } finally { + Await.result(producer.close().runToFuture, Duration.Inf) + consumer.close() + } + } + } + + test("listen for one message when subscribed to topics list") { + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerCfg, List(topicName)).executeOn(io) + try { + // Publishing one message + val send = producer.send(topicName, "test-message") + Await.result(send.runToFuture, 30.seconds) + + val first = consumer.take(1).map(_.value()).firstL + val result = Await.result(first.runToFuture, 30.seconds) + assert(result === "test-message") + } finally { + Await.result(producer.close().runToFuture, Duration.Inf) + } + } + } + + test("full producer/consumer test when subscribed to topics list") { + withRunningKafka { + val count = 10000 + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerCfg, List(topicName)).executeOn(io).take(count) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .map(_.value()) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } + + test("manual commit consumer test when subscribed to topics list") { + withRunningKafka { + + val count = 10000 + val topicName = "monix-kafka-manual-commit-tests" + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](consumerCfg, List(topicName)) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .executeOn(io) + .bufferTumbling(count) + .map { messages => messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) } + .mapEval { case (values, batch) => Task.shift *> batch.commitSync().map(_ => values -> batch.offsets) } + .headL + + val ((result, offsets), _) = + Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + + val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) + } + } + + test("publish to closed producer when subscribed to topics list") { + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val sendTask = producer.send(topicName, "test-message") + + val result = for { + //Force creation of producer + s1 <- producer.send(topicName, "test-message-1") + res <- Task.parZip2(producer.close(), Task.parSequence(List.fill(10)(sendTask)).attempt) + (_, s2) = res + s3 <- sendTask + } yield (s1, s2, s3) + + val (first, second, third) = Await.result(result.runToFuture, 60.seconds) + assert(first.isDefined && second.isRight && third.isEmpty) + } + } +} \ No newline at end of file diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala index 2843c4bf..bd3183cc 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -9,7 +9,6 @@ import monix.execution.Scheduler.Implicits.global import org.scalactic.source import org.scalatest.FunSuite import org.scalatest.concurrent.ScalaFutures -//import org.scalacheck.Prop.forAll import scala.concurrent.duration._ From 2b3e3e0d5f5836e9bda414920f30d79500b62646 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 27 Mar 2021 16:54:42 +0100 Subject: [PATCH 35/61] Fixed topic regex tests --- .../test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala | 5 +++-- .../test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala | 6 +++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index d62077b0..b4943dda 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -45,8 +45,8 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { ) test("publish one message when subscribed to topics regex") { - withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) val consumerTask = KafkaConsumerObservable.createConsumer[String, String](consumerCfg, topicsRegex).executeOn(io) val consumer = Await.result(consumerTask.runToFuture, 60.seconds) @@ -66,6 +66,7 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { } test("listen for one message when subscribed to topics regex") { + withRunningKafka { val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io) @@ -104,4 +105,4 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } -} +} \ No newline at end of file diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index 5d4713d3..b4943dda 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -24,7 +24,7 @@ import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.scalatest.FunSuite -import scala.jdk.CollectionConverters._ +import scala.collection.JavaConverters._ import scala.concurrent.Await import scala.concurrent.duration._ @@ -56,7 +56,7 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { val send = producer.send(topicMatchingRegex, "my-message") Await.result(send.runToFuture, 30.seconds) - val records = consumer.poll(0).asScala.map(_.value()).toList + val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList assert(records === List("my-message")) } finally { Await.result(producer.close().runToFuture, Duration.Inf) @@ -105,4 +105,4 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } -} +} \ No newline at end of file From da099c468110665961f4a117cce26b90b9f35985 Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 28 Mar 2021 00:43:28 +0100 Subject: [PATCH 36/61] Typo --- .../monix/kafka/KafkaConsumerConfig.scala | 12 +- .../monix/kafka/KafkaConsumerObservable.scala | 116 +++++----- .../monix/kafka/MonixKafkaTopicListTest.scala | 74 ------- .../scala/monix/kafka/PollHeartBeatTest.scala | 198 ++++++++++++++++++ .../monix/kafka/KafkaConsumerConfig.scala | 4 - .../monix/kafka/KafkaConsumerObservable.scala | 3 +- .../kafka/MonixKafkaTopicRegexTest.scala | 1 - 7 files changed, 270 insertions(+), 138 deletions(-) create mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index f9e42e61..3aad12c5 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -18,8 +18,8 @@ package monix.kafka import java.io.File import java.util.Properties - import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.internal.InternalApi import monix.kafka.config._ import scala.jdk.CollectionConverters._ @@ -252,7 +252,6 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -296,6 +295,14 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) + private[kafka] var pollHeartbeatRate: FiniteDuration = 10.millis + + @InternalApi + private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { + pollHeartbeatRate = interval + this + } + def toJavaMap: java.util.Map[String, Object] = toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava @@ -440,7 +447,6 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 4903131f..09edbb44 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -19,7 +19,7 @@ package monix.kafka import cats.effect.Resource import monix.eval.Task import monix.execution.Ack.{Continue, Stop} -import monix.execution.{Ack, Callback, Cancelable} +import monix.execution.{Ack, Callback, Cancelable, Scheduler} import monix.kafka.config.ObservableCommitOrder import monix.reactive.Observable import monix.reactive.observers.Subscriber @@ -27,6 +27,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer} import scala.jdk.CollectionConverters._ +import scala.concurrent.duration._ import scala.concurrent.blocking import scala.util.matching.Regex @@ -39,6 +40,7 @@ import scala.util.matching.Regex */ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig + protected def consumer: Task[Consumer[K, V]] @volatile @@ -55,6 +57,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { val callback = new Callback[Throwable, Unit] { def onSuccess(value: Unit): Unit = out.onComplete() + def onError(ex: Throwable): Unit = out.onError(ex) } @@ -106,20 +109,31 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * This allows producer process commit calls and also keeps consumer alive even * with long batch processing. * + * If polling fails the error is reported to the subscriber through the scheduler. + * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task.sleep(config.observablePollHeartbeatRate) *> - Task.evalAsync( + private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { + Task.sleep(config.pollHeartbeatRate) >> + Task.eval { if (!isAcked) { consumer.synchronized { + // needed in order to ensure that the consumer assignment + // is paused, meaning that no messages will get lost. + val assignment = consumer.assignment() + consumer.pause(assignment) val records = blocking(consumer.poll(0)) if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) } } - } else () - ) + } + } + .onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.seconds) + } } } @@ -127,30 +141,28 @@ object KafkaConsumerObservable { /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param consumer is a factory for the - * `org.apache.kafka.clients.consumer.KafkaConsumer` - * instance to use for consuming from Kafka + * `org.apache.kafka.clients.consumer.KafkaConsumer` + * instance to use for consuming from Kafka */ def apply[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = new KafkaConsumerObservableAutoCommit[K, V](cfg, consumer) /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param topics is the list of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) apply(cfg, consumer) @@ -158,15 +170,14 @@ object KafkaConsumerObservable { /** Builds a [[KafkaConsumerObservable]] instance. * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) apply(cfg, consumer) @@ -186,18 +197,17 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param consumer is a factory for the - * `org.apache.kafka.clients.consumer.KafkaConsumer` - * instance to use for consuming from Kafka + * `org.apache.kafka.clients.consumer.KafkaConsumer` + * instance to use for consuming from Kafka */ def manualCommit[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val manualCommitConfig = cfg.copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) new KafkaConsumerObservableManualCommit[K, V](manualCommitConfig, consumer) @@ -217,16 +227,15 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param topics is the list of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) manualCommit(cfg, consumer) @@ -246,16 +255,15 @@ object KafkaConsumerObservable { * .subscribe() * }}} * - * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the - * consumer; also make sure to see `monix/kafka/default.conf` for - * the default values being used. Auto commit will disabled and - * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! - * + * @param cfg is the [[KafkaConsumerConfig]] needed for initializing the + * consumer; also make sure to see `monix/kafka/default.conf` for + * the default values being used. Auto commit will disabled and + * observable commit order will turned to [[monix.kafka.config.ObservableCommitOrder.NoAck NoAck]] forcibly! * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) manualCommit(cfg, consumer) @@ -263,8 +271,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given list of topics. */ def createConsumer[K, V](config: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap @@ -278,8 +286,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given topics regex. */ def createConsumer[K, V](config: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap blocking { diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index d9f30775..5114227f 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -185,78 +185,4 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow batches processing doesn't cause rebalancing") { - withRunningKafka { - val count = 10000 - - val consumerConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - observablePollHeartbeatRate = 100.millis - ) - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .take(count) - .map(_.value()) - .bufferTumbling(count / 4) - .mapEval(s => - Task.sleep(2.second) >> Task.delay { - println(s) - s - }) - .flatMap(Observable.fromIterable) - .toListL - - val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } - - test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { - withRunningKafka { - - val count = 250 - val topicName = "monix-kafka-manual-commit-tests" - val delay = 200.millis - val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .doOnNextF { committableMessage => - Task.sleep(delay) *> - CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync - } - .take(count) - .toListL - - val (committableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat * 10 < delay) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } - } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala new file mode 100644 index 00000000..f7a17e0b --- /dev/null +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala @@ -0,0 +1,198 @@ +package monix.kafka + +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.TopicPartition +import org.scalactic.source +import org.scalatest.FunSuite +import org.scalatest.concurrent.ScalaFutures + +import scala.concurrent.duration._ + +class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { + + val topicName = "monix-kafka-tests" + + override implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + + val producerCfg: KafkaProducerConfig = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg: KafkaConsumerConfig = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + test("auto committable consumer with slow processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + heartbeatInterval = 10.millis, + maxPollRecords = 1 + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } + + test("slow committable downstream with small poll heartbeat does not cause rebalancing") { + withRunningKafka { + val totalRecords = 1000 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 40.millis + val pollHeartbeat = 1.millis + val maxPollInterval = 10.millis + val maxPollRecords = 1 + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < downstreamLatency) + assert(pollHeartbeat < maxPollInterval) + assert(maxPollInterval < downstreamLatency) + assert((1 to totalRecords).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === totalRecords) + assert(totalRecords === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + + //unhappy scenario + test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalance") { + withRunningKafka { + val totalRecords = 200 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 2.seconds + val pollHeartbeat = 15.seconds + val maxPollInterval = 100.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat > downstreamLatency) + assert(maxPollInterval < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) + whenReady(t.runToFuture.failed) { ex => + assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) + } + + } + } + + /* Important Note: + * This test will throw throw an `IllegalStateException: Received 50 unexpected messages` + * at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) + * This does not manifests after updating the `kafka-clients` dependency to `2.x` + */ + test("super slow committable downstream causes consumer rebalance") { + withRunningKafka { + val totalRecords = 3 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 55.seconds + val pollHeartbeat = 5.seconds + val maxPollInterval = 4.seconds + // the downstreamLatency is higher than the `maxPollInterval` + // but smaller than `pollHeartBeat`, kafka will trigger rebalance + // and the consumer will be kicked out of the consumer group. + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat * 10 < downstreamLatency) + assert(maxPollInterval * 10 < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + + val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) + whenReady(t.runToFuture.failed) { ex => + assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) + }(PatienceConfig(200.seconds, 1.seconds), source.Position.here) + } + } + +} diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 06b6bc7e..cb1b261f 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -304,10 +304,6 @@ case class KafkaConsumerConfig( private[kafka] var pollHeartbeatRate: FiniteDuration = 10.millis - /** - * - * @return - */ @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { pollHeartbeatRate = interval diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index dba75d9f..ce04ad8d 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -107,7 +107,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg Task.eval { if (!isAcked) { consumer.synchronized { - // needed in order to ensure that the consummer assignment + // needed in order to ensure that the consumer assignment // is paused, meaning that no messages will get lost. val assignment = consumer.assignment() consumer.pause(assignment) @@ -123,7 +123,6 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg Task.now(scheduler.reportFailure(ex)) >> Task.sleep(1.seconds) } - } } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index b4943dda..ff567170 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -66,7 +66,6 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { } test("listen for one message when subscribed to topics regex") { - withRunningKafka { val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io) From a5e6577ad3b41b00217cf17ae37aaeacdefc8a0d Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 28 Mar 2021 13:40:20 +0200 Subject: [PATCH 37/61] Update embedded kafka dependency --- build.sbt | 2 +- .../src/test/scala/monix/kafka/.DS_Store | Bin 0 -> 6148 bytes .../scala/monix/kafka/PollHeartBeatTest.scala | 198 ------------------ .../monix/kafka/PollHeartbeatdTest.scala | 108 ++++++++++ 4 files changed, 109 insertions(+), 199 deletions(-) create mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store delete mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala create mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala diff --git a/build.sbt b/build.sbt index b1685fe6..0a9d45ae 100644 --- a/build.sbt +++ b/build.sbt @@ -227,7 +227,7 @@ lazy val kafka11 = project.in(file("kafka-0.11.x")) .settings( name := "monix-kafka-11", libraryDependencies ++= { - if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.0.0" % "test" exclude ("log4j", "log4j")) + if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.1" % "test" exclude ("log4j", "log4j")) else Seq.empty[ModuleID] }, libraryDependencies += "org.apache.kafka" % "kafka-clients" % "0.11.0.3" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store b/kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store new file mode 100644 index 0000000000000000000000000000000000000000..5008ddfcf53c02e82d7eee2e57c38e5672ef89f6 GIT binary patch literal 6148 zcmeH~Jr2S!425mzP>H1@V-^m;4Wg<&0T*E43hX&L&p$$qDprKhvt+--jT7}7np#A3 zem<@ulZcFPQ@L2!n>{z**++&mCkOWA81W14cNZlEfg7;MkzE(HCqgga^y>{tEnwC%0;vJ&^%eQ zLs35+`xjp>T0 new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .take(count) - .map(_.value()) - .bufferTumbling(count / 4) - .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) - .flatMap(Observable.fromIterable) - .toListL - - val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } - - test("slow committable downstream with small poll heartbeat does not cause rebalancing") { - withRunningKafka { - val totalRecords = 1000 - val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 40.millis - val pollHeartbeat = 1.millis - val maxPollInterval = 10.millis - val maxPollRecords = 1 - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to totalRecords) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) - .as(committableMessage) - Task.sleep(downstreamLatency) *> manualCommit - } - .take(totalRecords) - .toListL - - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat * 10 < downstreamLatency) - assert(pollHeartbeat < maxPollInterval) - assert(maxPollInterval < downstreamLatency) - assert((1 to totalRecords).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === totalRecords) - assert(totalRecords === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } - - //unhappy scenario - test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalance") { - withRunningKafka { - val totalRecords = 200 - val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 2.seconds - val pollHeartbeat = 15.seconds - val maxPollInterval = 100.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to totalRecords) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) - .as(committableMessage) - Task.sleep(downstreamLatency) *> manualCommit - } - .take(totalRecords) - .toListL - - assert(pollHeartbeat > downstreamLatency) - assert(maxPollInterval < downstreamLatency) - assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) - - val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) - whenReady(t.runToFuture.failed) { ex => - assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) - } - - } - } - - /* Important Note: - * This test will throw throw an `IllegalStateException: Received 50 unexpected messages` - * at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) - * This does not manifests after updating the `kafka-clients` dependency to `2.x` - */ - test("super slow committable downstream causes consumer rebalance") { - withRunningKafka { - val totalRecords = 3 - val topicName = "monix-kafka-manual-commit-tests" - val downstreamLatency = 55.seconds - val pollHeartbeat = 5.seconds - val maxPollInterval = 4.seconds - // the downstreamLatency is higher than the `maxPollInterval` - // but smaller than `pollHeartBeat`, kafka will trigger rebalance - // and the consumer will be kicked out of the consumer group. - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to totalRecords) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .doOnNextF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) - Task.sleep(downstreamLatency) *> manualCommit - } - .take(totalRecords) - .toListL - - assert(pollHeartbeat * 10 < downstreamLatency) - assert(maxPollInterval * 10 < downstreamLatency) - assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) - - implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) - - val t = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1) - whenReady(t.runToFuture.failed) { ex => - assert(ex.getMessage.contains("the group has already rebalanced and assigned the partitions to another member")) - }(PatienceConfig(200.seconds, 1.seconds), source.Position.here) - } - } - -} diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala new file mode 100644 index 00000000..b5dfec1c --- /dev/null +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala @@ -0,0 +1,108 @@ +/* + * Copyright (c) 2014-2019 by its authors. Some rights reserved. + * See the project homepage at: https://github.com/monix/monix-kafka + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package monix.kafka + +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.scalatest.FunSuite + +import scala.collection.JavaConverters._ +import scala.concurrent.Await +import scala.concurrent.duration._ + +class PollHeartbeatdTest extends FunSuite with KafkaTestKit { + val topicsRegex = "monix-kafka-tests-.*".r + val topicMatchingRegex = "monix-kafka-tests-anything" + + val producerCfg = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + test("publish one message when subscribed to topics regex") { + + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val consumerTask = KafkaConsumerObservable.createConsumer[String, String](consumerCfg, topicsRegex).executeOn(io) + val consumer = Await.result(consumerTask.runToFuture, 60.seconds) + + try { + // Publishing one message + val send = producer.send(topicMatchingRegex, "my-message") + Await.result(send.runToFuture, 30.seconds) + + val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList + assert(records === List("my-message")) + } finally { + Await.result(producer.close().runToFuture, Duration.Inf) + consumer.close() + } + } + } + + test("listen for one message when subscribed to topics regex") { + + withRunningKafka { + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io) + try { + // Publishing one message + val send = producer.send(topicMatchingRegex, "test-message") + Await.result(send.runToFuture, 30.seconds) + + val first = consumer.take(1).map(_.value()).firstL + val result = Await.result(first.runToFuture, 30.seconds) + assert(result === "test-message") + } finally { + Await.result(producer.close().runToFuture, Duration.Inf) + } + } + } + + test("full producer/consumer test when subscribed to topics regex") { + withRunningKafka { + val count = 10000 + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io).take(count) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicMatchingRegex, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .map(_.value()) + .toListL + + val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } +} From 874ea8aaeae4cb8e254693fdd9983b51ddc588d2 Mon Sep 17 00:00:00 2001 From: Pau Date: Mon, 29 Mar 2021 01:00:57 +0200 Subject: [PATCH 38/61] Port back to older kafka subprojects --- .../monix/kafka/KafkaConsumerConfig.scala | 88 +++---- .../monix/kafka/KafkaConsumerObservable.scala | 57 ++--- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 4 +- .../monix/kafka/KafkaConsumerObservable.scala | 31 +-- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- kafka-0.11.x/src/test/resources/log4j2.xml | 30 +++ kafka-0.11.x/src/test/resources/logback.xml | 11 - .../monix/kafka/MonixKafkaTopicListTest.scala | 8 +- .../scala/monix/kafka/PollHeartbeatTest.scala | 218 ++++++++++++++++++ .../monix/kafka/PollHeartbeatdTest.scala | 108 --------- 13 files changed, 341 insertions(+), 222 deletions(-) create mode 100644 kafka-0.11.x/src/test/resources/log4j2.xml delete mode 100644 kafka-0.11.x/src/test/resources/logback.xml create mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala delete mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index ad60d84c..1247f28b 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -198,7 +198,7 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. + * @param pollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. * Specifies heartbeat time between KafkaConsumer#poll attempts. * * @param properties map of other properties that will be passed to @@ -207,48 +207,48 @@ import scala.concurrent.duration._ * a value set on the case class will overwrite value set via properties. */ final case class KafkaConsumerConfig( - bootstrapServers: List[String], - fetchMinBytes: Int, - groupId: String, - heartbeatInterval: FiniteDuration, - maxPartitionFetchBytes: Int, - sessionTimeout: FiniteDuration, - sslKeyPassword: Option[String], - sslKeyStorePassword: Option[String], - sslKeyStoreLocation: Option[String], - sslTrustStoreLocation: Option[String], - sslTrustStorePassword: Option[String], - autoOffsetReset: AutoOffsetReset, - connectionsMaxIdleTime: FiniteDuration, - enableAutoCommit: Boolean, - excludeInternalTopics: Boolean, - maxPollRecords: Int, - maxPollInterval: FiniteDuration, - receiveBufferInBytes: Int, - requestTimeout: FiniteDuration, - saslKerberosServiceName: Option[String], - saslMechanism: String, - securityProtocol: SecurityProtocol, - sendBufferInBytes: Int, - sslEnabledProtocols: List[SSLProtocol], - sslKeystoreType: String, - sslProtocol: SSLProtocol, - sslProvider: Option[String], - sslTruststoreType: String, - checkCRCs: Boolean, - clientId: String, - fetchMaxWaitTime: FiniteDuration, - metadataMaxAge: FiniteDuration, - metricReporters: List[String], - metricsNumSamples: Int, - metricsSampleWindow: FiniteDuration, - reconnectBackoffTime: FiniteDuration, - retryBackoffTime: FiniteDuration, - observableCommitType: ObservableCommitType, - observableCommitOrder: ObservableCommitOrder, - observableSeekOnStart: ObservableSeekOnStart, - observablePollHeartbeatRate: FiniteDuration, - properties: Map[String, String]) { + bootstrapServers: List[String], + fetchMinBytes: Int, + groupId: String, + heartbeatInterval: FiniteDuration, + maxPartitionFetchBytes: Int, + sessionTimeout: FiniteDuration, + sslKeyPassword: Option[String], + sslKeyStorePassword: Option[String], + sslKeyStoreLocation: Option[String], + sslTrustStoreLocation: Option[String], + sslTrustStorePassword: Option[String], + autoOffsetReset: AutoOffsetReset, + connectionsMaxIdleTime: FiniteDuration, + enableAutoCommit: Boolean, + excludeInternalTopics: Boolean, + maxPollRecords: Int, + maxPollInterval: FiniteDuration, + receiveBufferInBytes: Int, + requestTimeout: FiniteDuration, + saslKerberosServiceName: Option[String], + saslMechanism: String, + securityProtocol: SecurityProtocol, + sendBufferInBytes: Int, + sslEnabledProtocols: List[SSLProtocol], + sslKeystoreType: String, + sslProtocol: SSLProtocol, + sslProvider: Option[String], + sslTruststoreType: String, + checkCRCs: Boolean, + clientId: String, + fetchMaxWaitTime: FiniteDuration, + metadataMaxAge: FiniteDuration, + metricReporters: List[String], + metricsNumSamples: Int, + metricsSampleWindow: FiniteDuration, + reconnectBackoffTime: FiniteDuration, + retryBackoffTime: FiniteDuration, + observableCommitType: ObservableCommitType, + observableCommitOrder: ObservableCommitOrder, + observableSeekOnStart: ObservableSeekOnStart, + pollHeartbeatRate: FiniteDuration, + properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( "bootstrap.servers" -> bootstrapServers.mkString(","), @@ -433,7 +433,7 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, + pollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 4903131f..9038dad5 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,10 +16,9 @@ package monix.kafka -import cats.effect.Resource import monix.eval.Task import monix.execution.Ack.{Continue, Stop} -import monix.execution.{Ack, Callback, Cancelable} +import monix.execution.{Ack, Callback, Cancelable, Scheduler} import monix.kafka.config.ObservableCommitOrder import monix.reactive.Observable import monix.reactive.observers.Subscriber @@ -28,6 +27,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsume import scala.jdk.CollectionConverters._ import scala.concurrent.blocking +import scala.concurrent.duration.DurationInt import scala.util.matching.Regex /** Exposes an `Observable` that consumes a Kafka stream by @@ -39,7 +39,7 @@ import scala.util.matching.Regex */ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumer: Task[Consumer[K, V]] + protected def consumerTask: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -66,25 +66,15 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - Resource - .make(consumer) { c => - // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess - } - .use { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollHeartbeat(c) - // If polling fails the error is reported to the subscriber and - // wait 1sec as a rule of thumb leaving enough time for the consumer - // to recover and reassign partitions - .onErrorHandleWith(ex => Task(out.onError(ex))) - .loopForever - .startAndForget - .flatMap(_ => runLoop(c, out)) - } + consumerTask.bracket { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void + } { consumer => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + } startConsuming.runAsync(cb) } } @@ -106,20 +96,31 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * This allows producer process commit calls and also keeps consumer alive even * with long batch processing. * + * If polling fails the error is reported to the subscriber through the scheduler. + * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task.sleep(config.observablePollHeartbeatRate) *> - Task.evalAsync( + private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { + Task.sleep(config.pollHeartbeatRate) >> + Task.eval { if (!isAcked) { consumer.synchronized { + // needed in order to ensure that the consumer assignment + // is paused, meaning that no messages will get lost. + val assignment = consumer.assignment() + consumer.pause(assignment) val records = blocking(consumer.poll(0)) if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) } } - } else () - ) + } + } + .onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.second) + } } } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 09e0e200..d74de333 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 36e0f985..8c1e494a 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 303fdeb6..f5f8e54f 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -195,7 +195,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - observablePollHeartbeatRate = 10.millis + pollHeartbeatRate = 10.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -228,7 +228,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val delay = 200.millis val pollHeartbeat = 2.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, observablePollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = 200.millis, pollHeartbeatRate = pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 09edbb44..fec4b0f2 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,6 @@ package monix.kafka -import cats.effect.Resource import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable, Scheduler} @@ -41,7 +40,7 @@ import scala.util.matching.Regex trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumer: Task[Consumer[K, V]] + protected def consumerTask: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -69,25 +68,15 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - Resource - .make(consumer) { c => - // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess - } - .use { c => - // Skipping all available messages on all partitions - if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) - else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) - // A task to execute on both cancellation and normal termination - pollHeartbeat(c) - // If polling fails the error is reported to the subscriber and - // wait 1sec as a rule of thumb leaving enough time for the consumer - // to recover and reassign partitions - .onErrorHandleWith(ex => Task(out.onError(ex))) - .loopForever - .startAndForget - .flatMap(_ => runLoop(c, out)) - } + consumerTask.bracket { c => + // Skipping all available messages on all partitions + if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) + else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) + Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void + } { consumer => + // Forced asynchronous boundary + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + } startConsuming.runAsync(cb) } } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 09e0e200..d74de333 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 4407a927..bbf697b0 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumer: Task[Consumer[K, V]]) + override protected val consumerTask: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles diff --git a/kafka-0.11.x/src/test/resources/log4j2.xml b/kafka-0.11.x/src/test/resources/log4j2.xml new file mode 100644 index 00000000..bafe0424 --- /dev/null +++ b/kafka-0.11.x/src/test/resources/log4j2.xml @@ -0,0 +1,30 @@ + + + + logs + logs/archive + %-5level %d [%t] %c:%M(%L): %m%n + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-0.11.x/src/test/resources/logback.xml b/kafka-0.11.x/src/test/resources/logback.xml deleted file mode 100644 index 2beb83c9..00000000 --- a/kafka-0.11.x/src/test/resources/logback.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n - - - - - - - diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 5114227f..64a6b4c4 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -108,7 +108,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { test("manual commit consumer test when subscribed to topics list") { withRunningKafka { - val count = 10000 + val count = 1000 val topicName = "monix-kafka-manual-commit-tests" val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -117,20 +117,20 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val pushT = Observable .range(0, count) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) + .bufferTumbling(10) .consumeWith(producer) val listT = consumer .executeOn(io) .bufferTumbling(count) .map { messages => messages.map(_.record.value()) -> CommittableOffsetBatch(messages.map(_.committableOffset)) } - .mapEval { case (values, batch) => Task.shift *> batch.commitSync().map(_ => values -> batch.offsets) } + .mapEval { case (values, batch) => batch.commitSync().map(_ => values -> batch.offsets) } .headL val ((result, offsets), _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) - val properOffsets = Map(new TopicPartition(topicName, 0) -> 10000) + val properOffsets = Map(new TopicPartition(topicName, 0) -> count) assert(result.map(_.toInt).sum === (0 until count).sum && offsets === properOffsets) } } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala new file mode 100644 index 00000000..3a7b1e9b --- /dev/null +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -0,0 +1,218 @@ +/* + * Copyright (c) 2014-2019 by its authors. Some rights reserved. + * See the project homepage at: https://github.com/monix/monix-kafka + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package monix.kafka + +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.TopicPartition +import org.scalactic.source +import org.scalatest.{FunSuite, Ignore} +import org.scalatest.concurrent.ScalaFutures + +import scala.concurrent.duration._ + +class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { + + val topicName = "monix-kafka-tests" + + val producerCfg = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + + test("auto committable consumer with slow processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + heartbeatInterval = 10.millis, + maxPollRecords = 1 + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } + + test("slow committable downstream with small poll heartbeat does not cause rebalancing") { + withRunningKafka { + val totalRecords = 1000 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 40.millis + val pollHeartbeat = 1.millis + val maxPollInterval = 10.millis + val maxPollRecords = 1 + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < downstreamLatency) + assert(pollHeartbeat < maxPollInterval) + assert(maxPollInterval < downstreamLatency) + assert((1 to totalRecords).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === totalRecords) + assert(totalRecords === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + + //unhappy scenario + test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalance") { + withRunningKafka { + val totalRecords = 200 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 2.seconds + val pollHeartbeat = 15.seconds + val maxPollInterval = 100.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat > downstreamLatency) + assert(maxPollInterval < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture + + // todo - we check that value never returns, + // which is correct in this scenario since if the `maxPollInterval` + // is higher than `pollHeartBeat` and `downstreamLatency` + // on the other hand, it would be ideal to receive the following error message from kafka + // "the group has already rebalanced and assigned the partitions to another member" + // as it happens from kafka-client 1.1.0, see tests from kafka1x. + assert(f.value === None) + } + } + + test("super slow committable downstream causes consumer rebalance") { + withRunningKafka { + val totalRecords = 3 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 55.seconds + val pollHeartbeat = 5.seconds + val maxPollInterval = 4.seconds + // the downstreamLatency is higher than the `maxPollInterval` + // but smaller than `pollHeartBeat`, kafka will trigger rebalance + // and the consumer will be kicked out of the consumer group. + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat * 10 < downstreamLatency) + assert(maxPollInterval * 10 < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + + val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture + + // todo - we check that value never returns, + // which is correct in this scenario since if the `maxPollInterval` + // is higher than `pollHeartBeat` and `downstreamLatency` + // on the other hand, it would be ideal to receive the following error message from kafka + // "the group has already rebalanced and assigned the partitions to another member" + // as it happens from kafka-client 1.1.0, see tests from kafka1x. + assert(f.value === None) + } + } + +} diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala deleted file mode 100644 index b5dfec1c..00000000 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatdTest.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Copyright (c) 2014-2019 by its authors. Some rights reserved. - * See the project homepage at: https://github.com/monix/monix-kafka - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package monix.kafka - -import monix.eval.Task -import monix.execution.Scheduler.Implicits.global -import monix.kafka.config.AutoOffsetReset -import monix.reactive.Observable -import org.apache.kafka.clients.producer.ProducerRecord -import org.scalatest.FunSuite - -import scala.collection.JavaConverters._ -import scala.concurrent.Await -import scala.concurrent.duration._ - -class PollHeartbeatdTest extends FunSuite with KafkaTestKit { - val topicsRegex = "monix-kafka-tests-.*".r - val topicMatchingRegex = "monix-kafka-tests-anything" - - val producerCfg = KafkaProducerConfig.default.copy( - bootstrapServers = List("127.0.0.1:6001"), - clientId = "monix-kafka-1-0-producer-test" - ) - - val consumerCfg = KafkaConsumerConfig.default.copy( - bootstrapServers = List("127.0.0.1:6001"), - groupId = "kafka-tests", - clientId = "monix-kafka-1-0-consumer-test", - autoOffsetReset = AutoOffsetReset.Earliest - ) - - test("publish one message when subscribed to topics regex") { - - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val consumerTask = KafkaConsumerObservable.createConsumer[String, String](consumerCfg, topicsRegex).executeOn(io) - val consumer = Await.result(consumerTask.runToFuture, 60.seconds) - - try { - // Publishing one message - val send = producer.send(topicMatchingRegex, "my-message") - Await.result(send.runToFuture, 30.seconds) - - val records = consumer.poll(10.seconds.toMillis).asScala.map(_.value()).toList - assert(records === List("my-message")) - } finally { - Await.result(producer.close().runToFuture, Duration.Inf) - consumer.close() - } - } - } - - test("listen for one message when subscribed to topics regex") { - - withRunningKafka { - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io) - try { - // Publishing one message - val send = producer.send(topicMatchingRegex, "test-message") - Await.result(send.runToFuture, 30.seconds) - - val first = consumer.take(1).map(_.value()).firstL - val result = Await.result(first.runToFuture, 30.seconds) - assert(result === "test-message") - } finally { - Await.result(producer.close().runToFuture, Duration.Inf) - } - } - } - - test("full producer/consumer test when subscribed to topics regex") { - withRunningKafka { - val count = 10000 - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io).take(count) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicMatchingRegex, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .map(_.value()) - .toListL - - val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } -} From 7bb610c1393d064b33e64c61f368e2d5beed6b2d Mon Sep 17 00:00:00 2001 From: Pau Date: Tue, 30 Mar 2021 18:54:30 +0200 Subject: [PATCH 39/61] A --- build.sbt | 1 + kafka-0.10.x/src/main/resources/log4j2.xml | 16 ++++++++++ .../kafka/MergeByCommitCallbackTest.scala | 1 - kafka-0.11.x/src/main/resources/log4j2.xml | 30 +++++++++++++++++++ .../src/test/resources/log4j.properties | 0 .../scala/monix/kafka/PollHeartbeatTest.scala | 3 +- .../src/test/resources/log4j.properties | 18 +++++++++++ ...BeatTest.scala => PollHeartbeatTest.scala} | 2 +- 8 files changed, 67 insertions(+), 4 deletions(-) create mode 100644 kafka-0.10.x/src/main/resources/log4j2.xml create mode 100644 kafka-0.11.x/src/main/resources/log4j2.xml create mode 100644 kafka-0.11.x/src/test/resources/log4j.properties create mode 100644 kafka-1.0.x/src/test/resources/log4j.properties rename kafka-1.0.x/src/test/scala/monix/kafka/{PollHeartBeatTest.scala => PollHeartbeatTest.scala} (99%) diff --git a/build.sbt b/build.sbt index 0a9d45ae..c5400bb7 100644 --- a/build.sbt +++ b/build.sbt @@ -228,6 +228,7 @@ lazy val kafka11 = project.in(file("kafka-0.11.x")) name := "monix-kafka-11", libraryDependencies ++= { if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.1" % "test" exclude ("log4j", "log4j")) + Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.0-kafka1.1-nosr" % Test) else Seq.empty[ModuleID] }, libraryDependencies += "org.apache.kafka" % "kafka-clients" % "0.11.0.3" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") diff --git a/kafka-0.10.x/src/main/resources/log4j2.xml b/kafka-0.10.x/src/main/resources/log4j2.xml new file mode 100644 index 00000000..c3b6c46a --- /dev/null +++ b/kafka-0.10.x/src/main/resources/log4j2.xml @@ -0,0 +1,16 @@ + + + + %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala index f0107b63..7d854a66 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MergeByCommitCallbackTest.scala @@ -9,7 +9,6 @@ import org.scalatest.{FunSuite, Matchers} import scala.concurrent.duration._ import scala.concurrent.Await import monix.execution.Scheduler.Implicits.global -import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition import org.scalacheck.Gen import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks diff --git a/kafka-0.11.x/src/main/resources/log4j2.xml b/kafka-0.11.x/src/main/resources/log4j2.xml new file mode 100644 index 00000000..bafe0424 --- /dev/null +++ b/kafka-0.11.x/src/main/resources/log4j2.xml @@ -0,0 +1,30 @@ + + + + logs + logs/archive + %-5level %d [%t] %c:%M(%L): %m%n + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-0.11.x/src/test/resources/log4j.properties b/kafka-0.11.x/src/test/resources/log4j.properties new file mode 100644 index 00000000..e69de29b diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 3a7b1e9b..ac234b26 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -23,8 +23,7 @@ import monix.kafka.config.AutoOffsetReset import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition -import org.scalactic.source -import org.scalatest.{FunSuite, Ignore} +import org.scalatest.FunSuite import org.scalatest.concurrent.ScalaFutures import scala.concurrent.duration._ diff --git a/kafka-1.0.x/src/test/resources/log4j.properties b/kafka-1.0.x/src/test/resources/log4j.properties new file mode 100644 index 00000000..191ccb93 --- /dev/null +++ b/kafka-1.0.x/src/test/resources/log4j.properties @@ -0,0 +1,18 @@ +rootLogger.level = info +rootLogger.appenderRefs = stdout +rootLogger.appenderRef.stdout.ref = STDOUT + +appenders = stdout + +appender.stdout.name = STDOUT +appender.stdout.type = Console +appender.stdout.layout.type = PatternLayout +appender.stdout.layout.pattern =%d{yyyy-MM-dd HH:mm:ss.SSS} [%level] [%t] %c - %m%n + +loggers = kafka, kafka-consumer + +logger.kafka.name = org.apache.kafka +logger.kafka.level = warn + +logger.kafka-consumer.name = org.apache.kafka.clients.consumer +logger.kafka-consumer.level = info \ No newline at end of file diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala similarity index 99% rename from kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala rename to kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index bd3183cc..1f622c3d 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartBeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -12,7 +12,7 @@ import org.scalatest.concurrent.ScalaFutures import scala.concurrent.duration._ -class PollHeartBeatTest extends FunSuite with KafkaTestKit with ScalaFutures { +class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val topicName = "monix-kafka-tests" From a3210c3688e2faf55fae8a13cb980890bffbd99d Mon Sep 17 00:00:00 2001 From: Pau Date: Tue, 30 Mar 2021 20:54:32 +0200 Subject: [PATCH 40/61] Make poll interval rate private on kafka 10 and 9 --- build.sbt | 1 - .../monix/kafka/KafkaConsumerConfig.scala | 12 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 3 +- .../scala/monix/kafka/PollHeartbeatTest.scala | 217 ++++++++++++++++++ .../main/resources/monix/kafka/default.conf | 2 - .../monix/kafka/KafkaConsumerConfig.scala | 2 +- .../kafka/MonixKafkaTopicRegexTest.scala | 3 +- .../monix/kafka/KafkaConsumerConfig.scala | 12 +- .../monix/kafka/KafkaConsumerObservable.scala | 24 +- .../scala/monix/kafka/MonixKafkaTest.scala | 4 +- .../monix/kafka/KafkaConsumerConfig.scala | 2 +- 11 files changed, 258 insertions(+), 24 deletions(-) create mode 100644 kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala diff --git a/build.sbt b/build.sbt index c5400bb7..0a9d45ae 100644 --- a/build.sbt +++ b/build.sbt @@ -228,7 +228,6 @@ lazy val kafka11 = project.in(file("kafka-0.11.x")) name := "monix-kafka-11", libraryDependencies ++= { if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.1" % "test" exclude ("log4j", "log4j")) - Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.0-kafka1.1-nosr" % Test) else Seq.empty[ModuleID] }, libraryDependencies += "org.apache.kafka" % "kafka-clients" % "0.11.0.3" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 1247f28b..b3cde65f 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -18,8 +18,8 @@ package monix.kafka import java.io.File import java.util.Properties - import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.internal.InternalApi import monix.kafka.config._ import scala.jdk.CollectionConverters._ @@ -247,7 +247,6 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - pollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -290,6 +289,14 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) + private[kafka] var pollHeartbeatRate: FiniteDuration = 100.millis + + @InternalApi + private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { + pollHeartbeatRate = interval + this + } + def toJavaMap: java.util.Map[String, Object] = toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava @@ -433,7 +440,6 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - pollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index f5f8e54f..fe0ef917 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -195,7 +195,6 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val consumerConfig = consumerCfg.copy( maxPollInterval = 200.millis, - pollHeartbeatRate = 10.millis ) val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -228,7 +227,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val delay = 200.millis val pollHeartbeat = 2.millis val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, pollHeartbeatRate = pollHeartbeat) + consumerCfg.copy(maxPollInterval = 200.millis).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala new file mode 100644 index 00000000..ac234b26 --- /dev/null +++ b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -0,0 +1,217 @@ +/* + * Copyright (c) 2014-2019 by its authors. Some rights reserved. + * See the project homepage at: https://github.com/monix/monix-kafka + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package monix.kafka + +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global +import monix.kafka.config.AutoOffsetReset +import monix.reactive.Observable +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.TopicPartition +import org.scalatest.FunSuite +import org.scalatest.concurrent.ScalaFutures + +import scala.concurrent.duration._ + +class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { + + val topicName = "monix-kafka-tests" + + val producerCfg = KafkaProducerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + clientId = "monix-kafka-1-0-producer-test" + ) + + val consumerCfg = KafkaConsumerConfig.default.copy( + bootstrapServers = List("127.0.0.1:6001"), + groupId = "kafka-tests", + clientId = "monix-kafka-1-0-consumer-test", + autoOffsetReset = AutoOffsetReset.Earliest + ) + + + test("auto committable consumer with slow processing doesn't cause rebalancing") { + withRunningKafka { + val count = 10000 + + val consumerConfig = consumerCfg.copy( + maxPollInterval = 200.millis, + heartbeatInterval = 10.millis, + maxPollRecords = 1 + ) + + val producer = KafkaProducerSink[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) + + val pushT = Observable + .range(0, count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .bufferIntrospective(1024) + .consumeWith(producer) + + val listT = consumer + .take(count) + .map(_.value()) + .bufferTumbling(count / 4) + .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) + .flatMap(Observable.fromIterable) + .toListL + + val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() + assert(result.map(_.toInt).sum === (0 until count).sum) + } + } + + test("slow committable downstream with small poll heartbeat does not cause rebalancing") { + withRunningKafka { + val totalRecords = 1000 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 40.millis + val pollHeartbeat = 1.millis + val maxPollInterval = 10.millis + val maxPollRecords = 1 + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < downstreamLatency) + assert(pollHeartbeat < maxPollInterval) + assert(maxPollInterval < downstreamLatency) + assert((1 to totalRecords).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === totalRecords) + assert(totalRecords === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + + //unhappy scenario + test("slow committable downstream with small `maxPollInterval` and high `pollHeartBeat` causes consumer rebalance") { + withRunningKafka { + val totalRecords = 200 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 2.seconds + val pollHeartbeat = 15.seconds + val maxPollInterval = 100.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .mapEvalF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + .as(committableMessage) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat > downstreamLatency) + assert(maxPollInterval < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture + + // todo - we check that value never returns, + // which is correct in this scenario since if the `maxPollInterval` + // is higher than `pollHeartBeat` and `downstreamLatency` + // on the other hand, it would be ideal to receive the following error message from kafka + // "the group has already rebalanced and assigned the partitions to another member" + // as it happens from kafka-client 1.1.0, see tests from kafka1x. + assert(f.value === None) + } + } + + test("super slow committable downstream causes consumer rebalance") { + withRunningKafka { + val totalRecords = 3 + val topicName = "monix-kafka-manual-commit-tests" + val downstreamLatency = 55.seconds + val pollHeartbeat = 5.seconds + val maxPollInterval = 4.seconds + // the downstreamLatency is higher than the `maxPollInterval` + // but smaller than `pollHeartBeat`, kafka will trigger rebalance + // and the consumer will be kicked out of the consumer group. + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to totalRecords) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + Task.sleep(downstreamLatency) *> manualCommit + } + .take(totalRecords) + .toListL + + assert(pollHeartbeat * 10 < downstreamLatency) + assert(maxPollInterval * 10 < downstreamLatency) + assert(fastPollHeartbeatConfig.pollHeartbeatRate === pollHeartbeat) + + implicit val patienceConfig: PatienceConfig = PatienceConfig(30.seconds, 100.milliseconds) + + val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture + + // todo - we check that value never returns, + // which is correct in this scenario since if the `maxPollInterval` + // is higher than `pollHeartBeat` and `downstreamLatency` + // on the other hand, it would be ideal to receive the following error message from kafka + // "the group has already rebalanced and assigned the partitions to another member" + // as it happens from kafka-client 1.1.0, see tests from kafka1x. + assert(f.value === None) + } + } + +} diff --git a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf index ea06b1b9..7c9ea91b 100644 --- a/kafka-0.11.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.11.x/src/main/resources/monix/kafka/default.conf @@ -79,6 +79,4 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. - monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 3aad12c5..d4c781d8 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -295,7 +295,7 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 10.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index f91c8bfe..b3ae8649 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -89,7 +89,8 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { val count = 10000 val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex).executeOn(io).take(count) + val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex) + .executeOn(io).take(count) val pushT = Observable .range(0, count) diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 7eeee574..f470c17a 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -18,8 +18,8 @@ package monix.kafka import java.io.File import java.util.Properties - import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.internal.InternalApi import monix.kafka.config._ import scala.jdk.CollectionConverters._ @@ -241,7 +241,6 @@ final case class KafkaConsumerConfig( observableCommitType: ObservableCommitType, observableCommitOrder: ObservableCommitOrder, observableSeekOnStart: ObservableSeekOnStart, - observablePollHeartbeatRate: FiniteDuration, properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( @@ -282,6 +281,14 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) + private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis + + @InternalApi + private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { + pollHeartbeatRate = interval + this + } + def toJavaMap: java.util.Map[String, Object] = toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava @@ -423,7 +430,6 @@ object KafkaConsumerConfig { observableCommitType = ObservableCommitType(config.getString("monix.observable.commit.type")), observableCommitOrder = ObservableCommitOrder(config.getString("monix.observable.commit.order")), observableSeekOnStart = ObservableSeekOnStart(config.getString("monix.observable.seek.onStart")), - observablePollHeartbeatRate = config.getInt("monix.observable.poll.heartbeat.rate.ms").millis, properties = Map.empty ) } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 1dda2737..6d0d08ee 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -19,7 +19,7 @@ package monix.kafka import cats.effect.Resource import monix.eval.Task import monix.execution.Ack.{Continue, Stop} -import monix.execution.{Ack, Callback, Cancelable} +import monix.execution.{Ack, Callback, Cancelable, Scheduler} import monix.kafka.config.ObservableCommitOrder import monix.reactive.Observable import monix.reactive.observers.Subscriber @@ -27,6 +27,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsume import scala.jdk.CollectionConverters._ import scala.concurrent.blocking +import scala.concurrent.duration._ /** Exposes an `Observable` that consumes a Kafka stream by * means of a Kafka Consumer client. @@ -107,18 +108,27 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { * * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] */ - private def pollHeartbeat(consumer: Consumer[K, V]): Task[Unit] = { - Task.sleep(config.observablePollHeartbeatRate) *> - Task.evalAsync( + private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { + Task.sleep(config.pollHeartbeatRate) >> + Task.eval { if (!isAcked) { consumer.synchronized { + // needed in order to ensure that the consumer assignment + // is paused, meaning that no messages will get lost. + val assignment = consumer.assignment().asScala.toList + consumer.pause(assignment: _*) val records = blocking(consumer.poll(0)) if (!records.isEmpty) { - throw new IllegalStateException(s"Received ${records.count()} unexpected messages.") + val errorMsg = s"Received ${records.count()} unexpected messages." + throw new IllegalStateException(errorMsg) } } - } else () - ) + } + } + .onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.seconds) + } } } diff --git a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala index 469d938f..3cc28c55 100644 --- a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala +++ b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala @@ -220,9 +220,7 @@ class MonixKafkaTest extends FunSuite { val count = 10000 val consumerConfig = consumerCfg.copy( - sessionTimeout = 200.millis, - observablePollHeartbeatRate = 100.millis - ) + sessionTimeout = 200.millis).withPollHeartBeatRate(100.millis) val producer = KafkaProducerSink[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index cb1b261f..bac8cadf 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -302,7 +302,7 @@ case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 10.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { From df39cd0818de707ecb5ebcd44a265376b492d28b Mon Sep 17 00:00:00 2001 From: Pau Date: Tue, 30 Mar 2021 21:33:30 +0200 Subject: [PATCH 41/61] Bit of clean up and updated readme --- README.md | 20 ++++----- benchmarks/src/main/resources/commands | 15 ------- kafka-0.10.x/src/main/resources/log4j2.xml | 16 ------- .../main/resources/monix/kafka/default.conf | 2 - .../scala/monix/kafka/PollHeartbeatTest.scala | 1 - .../monix/kafka/KafkaConsumerConfig.scala | 12 ------ .../monix/kafka/KafkaConsumerObservable.scala | 42 +------------------ .../scala/monix/kafka/MonixKafkaTest.scala | 2 +- .../main/resources/monix/kafka/default.conf | 2 - .../src/test/resources/log4j.properties | 18 -------- .../scala/monix/kafka/PollHeartbeatTest.scala | 5 --- 11 files changed, 12 insertions(+), 123 deletions(-) delete mode 100644 benchmarks/src/main/resources/commands delete mode 100644 kafka-0.10.x/src/main/resources/log4j2.xml delete mode 100644 kafka-1.0.x/src/test/resources/log4j.properties diff --git a/README.md b/README.md index cfc2457e..4a2f50e3 100644 --- a/README.md +++ b/README.md @@ -256,20 +256,18 @@ val observable = Enjoy! -### Caveats +### Internal poll heartbeat -[Issue#101](https://github.com/monix/monix-kafka/issues/101) -Starting from Kafka 0.10.1.0, there is `max.poll.interval.ms` setting: - - The maximum delay between invocations of poll() when using consumer group management. - This places an upper bound on the amount of time that the consumer can be idle before - fetching more records. If poll() is not called before expiration of this timeout, - then the consumer is considered failed and the group will rebalance in order +Starting from Kafka _0.10.1.0_, there is `max.poll.interval.ms` setting that defines the maximum delay between +invocations of poll(), if it is not called in that interval, then the consumer is considered failed and the group will rebalance in order to reassign the partitions to another member. -Since, monix-kafka backpressures until all records has been processed. -This could be a problem if processing takes time. -You can reduce `max.poll.records` if you are experiencing this issue. +This was an [issue](https://github.com/monix/monix-kafka/issues/101) in `monix-kafka`, +since poll is not called until all previous consumed ones were processed, so that slow downstream subscribers + were in risk of being kicked off the consumer group indefinitely. + +This has been resolved in `1.0.0-RC8` by introducing an internal poll heartbeat interval +that runs in the background keeping the consumer alive. ## How can I contribute to Monix-Kafka? diff --git a/benchmarks/src/main/resources/commands b/benchmarks/src/main/resources/commands deleted file mode 100644 index bc87b061..00000000 --- a/benchmarks/src/main/resources/commands +++ /dev/null @@ -1,15 +0,0 @@ - -# list topics -kafka-console-producer --broker-list broker:9092 --topic topic_1P_1RF - -# consume from beginning -docker exec -it broker kafka-console-consumer --bootstrap-server broker:9092 --topic topic2 --from-beginning -docker exec -it broker /bin/bash - -docker stop $(docker ps -a -q) - -# run producer benchmark -sbt 'benchmarks/jmh:run -o producer_results.txt monix.kafka.benchmarks.ProducerBenchmark' - -# run consumer benchmark -sbt 'benchmarks/jmh:run -o consumer_results.txt monix.kafka.benchmarks.ConsumerBenchmark' \ No newline at end of file diff --git a/kafka-0.10.x/src/main/resources/log4j2.xml b/kafka-0.10.x/src/main/resources/log4j2.xml deleted file mode 100644 index c3b6c46a..00000000 --- a/kafka-0.10.x/src/main/resources/log4j2.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n - - - - - - - - - - - - \ No newline at end of file diff --git a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf index dbbfebd3..182c8300 100644 --- a/kafka-0.10.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.10.x/src/main/resources/monix/kafka/default.conf @@ -77,6 +77,4 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. - monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index ac234b26..29902de2 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -44,7 +44,6 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { autoOffsetReset = AutoOffsetReset.Earliest ) - test("auto committable consumer with slow processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index f470c17a..7bcfa171 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -19,7 +19,6 @@ package monix.kafka import java.io.File import java.util.Properties import com.typesafe.config.{Config, ConfigFactory} -import monix.execution.internal.InternalApi import monix.kafka.config._ import scala.jdk.CollectionConverters._ @@ -194,9 +193,6 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. - * Specifies heartbeat time between KafkaConsumer#poll attempts. - * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -281,14 +277,6 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis - - @InternalApi - private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { - pollHeartbeatRate = interval - this - } - def toJavaMap: java.util.Map[String, Object] = toMap.filter(_._2 != null).map { case (a, b) => (a, b.asInstanceOf[AnyRef]) }.asJava diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 6d0d08ee..8f1d3e1e 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -19,7 +19,7 @@ package monix.kafka import cats.effect.Resource import monix.eval.Task import monix.execution.Ack.{Continue, Stop} -import monix.execution.{Ack, Callback, Cancelable, Scheduler} +import monix.execution.{Ack, Callback, Cancelable} import monix.kafka.config.ObservableCommitOrder import monix.reactive.Observable import monix.reactive.observers.Subscriber @@ -27,7 +27,6 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsume import scala.jdk.CollectionConverters._ import scala.concurrent.blocking -import scala.concurrent.duration._ /** Exposes an `Observable` that consumes a Kafka stream by * means of a Kafka Consumer client. @@ -76,14 +75,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd() else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning() // A task to execute on both cancellation and normal termination - pollHeartbeat(c) - // If polling fails the error is reported to the subscriber and - // wait 1sec as a rule of thumb leaving enough time for the consumer - // to recover and reassign partitions - .onErrorHandleWith(ex => Task(out.onError(ex))) - .loopForever - .startAndForget - .flatMap(_ => runLoop(c, out)) + runLoop(c, out) } startConsuming.runAsync(cb) } @@ -101,36 +93,6 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } - /** Returns task that constantly polls the `KafkaConsumer` in case subscriber - * is still processing last fed batch. - * This allows producer process commit calls and also keeps consumer alive even - * with long batch processing. - * - * @see [[https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread]] - */ - private def pollHeartbeat(consumer: Consumer[K, V])(implicit scheduler: Scheduler): Task[Unit] = { - Task.sleep(config.pollHeartbeatRate) >> - Task.eval { - if (!isAcked) { - consumer.synchronized { - // needed in order to ensure that the consumer assignment - // is paused, meaning that no messages will get lost. - val assignment = consumer.assignment().asScala.toList - consumer.pause(assignment: _*) - val records = blocking(consumer.poll(0)) - if (!records.isEmpty) { - val errorMsg = s"Received ${records.count()} unexpected messages." - throw new IllegalStateException(errorMsg) - } - } - } - } - .onErrorHandleWith { ex => - Task.now(scheduler.reportFailure(ex)) >> - Task.sleep(1.seconds) - } - } - } object KafkaConsumerObservable { diff --git a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala index 3cc28c55..c031512d 100644 --- a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala +++ b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala @@ -220,7 +220,7 @@ class MonixKafkaTest extends FunSuite { val count = 10000 val consumerConfig = consumerCfg.copy( - sessionTimeout = 200.millis).withPollHeartBeatRate(100.millis) + sessionTimeout = 200.millis) val producer = KafkaProducerSink[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) diff --git a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf index 59c2d7b9..91aefa9c 100644 --- a/kafka-1.0.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-1.0.x/src/main/resources/monix/kafka/default.conf @@ -78,6 +78,4 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. - monix.observable.poll.heartbeat.rate.ms = 100 } diff --git a/kafka-1.0.x/src/test/resources/log4j.properties b/kafka-1.0.x/src/test/resources/log4j.properties deleted file mode 100644 index 191ccb93..00000000 --- a/kafka-1.0.x/src/test/resources/log4j.properties +++ /dev/null @@ -1,18 +0,0 @@ -rootLogger.level = info -rootLogger.appenderRefs = stdout -rootLogger.appenderRef.stdout.ref = STDOUT - -appenders = stdout - -appender.stdout.name = STDOUT -appender.stdout.type = Console -appender.stdout.layout.type = PatternLayout -appender.stdout.layout.pattern =%d{yyyy-MM-dd HH:mm:ss.SSS} [%level] [%t] %c - %m%n - -loggers = kafka, kafka-consumer - -logger.kafka.name = org.apache.kafka -logger.kafka.level = warn - -logger.kafka-consumer.name = org.apache.kafka.clients.consumer -logger.kafka-consumer.level = info \ No newline at end of file diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 1f622c3d..171d88ee 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -146,11 +146,6 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { } } - /* Important Note: - * This test will throw throw an `IllegalStateException: Received 50 unexpected messages` - * at monix.kafka.KafkaConsumerObservable.$anonfun$pollHeartbeat$1(KafkaConsumerObservable.scala:112) - * This does not manifests after updating the `kafka-clients` dependency to `2.x` - */ test("super slow committable downstream causes consumer rebalance") { withRunningKafka { val totalRecords = 3 From 6d1ebbf77cbff17b006551bc3a6db0fa4b65593a Mon Sep 17 00:00:00 2001 From: Pau Date: Wed, 31 Mar 2021 10:28:48 +0200 Subject: [PATCH 42/61] Update changelog and trigger pipeline --- CHANGES.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 8860e9a8..9d467e39 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,22 @@ + +## Version 1.0.0-RC6 (January 23, 2021) + +Depends on Monix 3.3.0 + +Main changes: + +- PR #248: Enable '-Xfatal-warnings' flag (internal) +- PR #245: Use Kafka Producer Consumer interfaces + +## Version 1.0.0-RC7 (April 18, 2020) + +Depends on Monix 3.3.0 + +Main changes: + +- PR #248: Enable '-Xfatal-warnings flag +- PR #245: Use Kafka Producer Consumer interfaces + ## Version 1.0.0-RC6 (April 18, 2020) Depends on Monix 3.x From 38f7d4768321ead4a89ed136020be3f58101285d Mon Sep 17 00:00:00 2001 From: Pau Date: Wed, 31 Mar 2021 12:24:08 +0200 Subject: [PATCH 43/61] Test indentation fix --- .../src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index fe0ef917..60ec4976 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -193,9 +193,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { withRunningKafka { val count = 10000 - val consumerConfig = consumerCfg.copy( - maxPollInterval = 200.millis, - ) + val consumerConfig = consumerCfg.copy(maxPollInterval = 200.millis) val producer = KafkaProducerSink[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) From bac31756d1311670752a48f28aaacfba33edfc92 Mon Sep 17 00:00:00 2001 From: Pau Date: Wed, 31 Mar 2021 22:56:02 +0200 Subject: [PATCH 44/61] Add logback in test --- kafka-0.10.x/src/main/resources/logback.xml | 25 +++++++++++++++++ kafka-0.11.x/src/main/resources/log4j2.xml | 30 --------------------- kafka-0.11.x/src/main/resources/logback.xml | 25 +++++++++++++++++ kafka-0.9.x/src/test/resources/logback.xml | 28 ++++++++++++++----- kafka-1.0.x/src/main/resources/logback.xml | 25 +++++++++++++++++ 5 files changed, 96 insertions(+), 37 deletions(-) create mode 100644 kafka-0.10.x/src/main/resources/logback.xml delete mode 100644 kafka-0.11.x/src/main/resources/log4j2.xml create mode 100644 kafka-0.11.x/src/main/resources/logback.xml create mode 100644 kafka-1.0.x/src/main/resources/logback.xml diff --git a/kafka-0.10.x/src/main/resources/logback.xml b/kafka-0.10.x/src/main/resources/logback.xml new file mode 100644 index 00000000..327dc7d7 --- /dev/null +++ b/kafka-0.10.x/src/main/resources/logback.xml @@ -0,0 +1,25 @@ + + + + + + %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-0.11.x/src/main/resources/log4j2.xml b/kafka-0.11.x/src/main/resources/log4j2.xml deleted file mode 100644 index bafe0424..00000000 --- a/kafka-0.11.x/src/main/resources/log4j2.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - logs - logs/archive - %-5level %d [%t] %c:%M(%L): %m%n - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/kafka-0.11.x/src/main/resources/logback.xml b/kafka-0.11.x/src/main/resources/logback.xml new file mode 100644 index 00000000..327dc7d7 --- /dev/null +++ b/kafka-0.11.x/src/main/resources/logback.xml @@ -0,0 +1,25 @@ + + + + + + %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-0.9.x/src/test/resources/logback.xml b/kafka-0.9.x/src/test/resources/logback.xml index 5a3e8a64..327dc7d7 100644 --- a/kafka-0.9.x/src/test/resources/logback.xml +++ b/kafka-0.9.x/src/test/resources/logback.xml @@ -1,11 +1,25 @@ - - - %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n - + + + + + %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + + - - + + + + + + + + + + + + - + + \ No newline at end of file diff --git a/kafka-1.0.x/src/main/resources/logback.xml b/kafka-1.0.x/src/main/resources/logback.xml new file mode 100644 index 00000000..327dc7d7 --- /dev/null +++ b/kafka-1.0.x/src/main/resources/logback.xml @@ -0,0 +1,25 @@ + + + + + + %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + + + + + + + + + + + + + + + + + + + \ No newline at end of file From 444c2a2fa74f274ff3b6809847fc761c5f80bce9 Mon Sep 17 00:00:00 2001 From: Pau Date: Thu, 1 Apr 2021 09:44:28 +0200 Subject: [PATCH 45/61] Add unchanged topic partition test case --- benchmarks/results/consumer.md | 28 +++++--- benchmarks/src/main/resources/log4j2.xml | 16 ----- benchmarks/src/main/resources/logback.xml | 25 +++++++ .../kafka/benchmarks/ConsumerBenchmark.scala | 34 ++++++++-- .../monix/kafka/MonixKafkaTopicListTest.scala | 67 ------------------- .../scala/monix/kafka/PollHeartbeatTest.scala | 37 ++++++++++ .../scala/monix/kafka/PollHeartbeatTest.scala | 40 ++++++++++- .../scala/monix/kafka/PollHeartbeatTest.scala | 38 +++++++++++ 8 files changed, 186 insertions(+), 99 deletions(-) delete mode 100644 benchmarks/src/main/resources/log4j2.xml create mode 100644 benchmarks/src/main/resources/logback.xml diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 41bcc231..9932a5eb 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -16,16 +16,24 @@ ConsumerBenchmark.monix_manual_commit thrpt 10 16.319 ± 1.465 ops/s ### 1fork 1thread --- Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit10ms thrpt 10 11.090 ± 1.883 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 0.993 ± 0.002 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 4.792 ± 0.017 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 8.249 ± 0.305 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 10.038 ± 0.433 ops/s +ConsumerBenchmark.monixAutoCommitHeartbeat10ms thrpt 7 4.865 ± 1.044 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat15ms thrpt 7 13.454 ± 2.680 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat1ms thrpt 7 14.281 ± 1.591 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat50ms thrpt 7 11.900 ± 0.698 ops/s --- ### 1 fork 3 threads Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit10ms thrpt 10 17.266 ± 2.231 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1000ms thrpt 10 2.971 ± 0.009 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat100ms thrpt 10 9.477 ± 0.064 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat10ms thrpt 10 14.710 ± 1.660 ops/s -ConsumerBenchmark.monix_manual_commit_heartbeat1ms thrpt 10 15.494 ± 4.163 ops/s +ConsumerBenchmark.monixAutoCommitHeartbeat10ms thrpt 7 4.865 ± 1.044 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat15ms thrpt 7 13.454 ± 2.680 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat1ms thrpt 7 14.281 ± 1.591 ops/s +ConsumerBenchmark.monixManualCommitHeartbeat50ms thrpt 7 11.900 ± 0.698 ops/s + +```sbt +sbt 'benchmarks/jmh:run -i 5 -wi 1 -f1 -t1 monix.kafka.benchmarks.ConsumerBenchmark.*' +``` \ No newline at end of file diff --git a/benchmarks/src/main/resources/log4j2.xml b/benchmarks/src/main/resources/log4j2.xml deleted file mode 100644 index c3b6c46a..00000000 --- a/benchmarks/src/main/resources/log4j2.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n - - - - - - - - - - - - \ No newline at end of file diff --git a/benchmarks/src/main/resources/logback.xml b/benchmarks/src/main/resources/logback.xml new file mode 100644 index 00000000..327dc7d7 --- /dev/null +++ b/benchmarks/src/main/resources/logback.xml @@ -0,0 +1,25 @@ + + + + + + %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 1910d799..5e640c00 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -35,7 +35,7 @@ class ConsumerBenchmark extends MonixFixture { } @Benchmark - def monix_auto_commit10ms(): Unit = { + def monixAutoCommitHeartbeat10ms(): Unit = { val conf = consumerConf.value().copy( maxPollRecords = maxPollRecords, observableCommitType = ObservableCommitType.Async) @@ -49,7 +49,7 @@ class ConsumerBenchmark extends MonixFixture { @Benchmark - def monix_manual_commit_heartbeat1ms(): Unit = { + def monixManualCommitHeartbeat1ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(1.millis) @@ -61,7 +61,7 @@ class ConsumerBenchmark extends MonixFixture { } @Benchmark - def monix_manual_commit_heartbeat10ms(): Unit = { + def monixManualCommitHeartbeat10ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(10.millis) @@ -73,7 +73,31 @@ class ConsumerBenchmark extends MonixFixture { } @Benchmark - def monix_manual_commit_heartbeat100ms(): Unit = { + def monixManualCommitHeartbeat15ms(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(15.millis) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(consumedRecords) + .headL + .runSyncUnsafe() + } + + @Benchmark + def monixManualCommitHeartbeat50ms(): Unit = { + val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(50.millis) + + KafkaConsumerObservable.manualCommit[Integer, Integer](conf, List(monixTopic)) + .mapEvalF(_.committableOffset.commitAsync()) + .take(consumedRecords) + .headL + .runSyncUnsafe() + } + + @Benchmark + def monixManualCommitHeartbeat100ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(100.millis) @@ -85,7 +109,7 @@ class ConsumerBenchmark extends MonixFixture { } @Benchmark - def monix_manual_commit_heartbeat1000ms(): Unit = { + def monixManualCommitHeartbeat1000ms(): Unit = { val conf = consumerConf.value().copy(maxPollRecords = maxPollRecords) .withPollHeartBeatRate(1000.millis) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 60ec4976..02bd40fe 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -189,71 +189,4 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("slow batches autocommit processing doesn't cause rebalancing") { - withRunningKafka { - val count = 10000 - - val consumerConfig = consumerCfg.copy(maxPollInterval = 200.millis) - - val producer = KafkaProducerSink[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) - - val pushT = Observable - .range(0, count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferIntrospective(1024) - .consumeWith(producer) - - val listT = consumer - .take(count) - .map(_.value()) - .bufferTumbling(count / 4) - .mapEval(s => Task.sleep(2.second) >> Task.delay(s)) - .flatMap(Observable.fromIterable) - .toListL - - val (result, _) = Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 60.seconds) - assert(result.map(_.toInt).sum === (0 until count).sum) - } - } - - test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { - withRunningKafka { - - val count = 250 - val topicName = "monix-kafka-manual-commit-tests" - val delay = 200.millis - val pollHeartbeat = 2.millis - val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis).withPollHeartBeatRate(pollHeartbeat) - - val producer = KafkaProducer[String, String](producerCfg, io) - val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) - - val pushT = Observable - .fromIterable(1 to count) - .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .mapEval(producer.send) - .lastL - - val listT = consumer - .executeOn(io) - .doOnNextF { committableMessage => - Task.sleep(delay) *> - CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync - } - .take(count) - .toListL - - val (committableMessages, _) = - Await.result(Task.parZip2(listT.executeAsync, pushT.executeAsync).runToFuture, 100.seconds) - val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last - assert(pollHeartbeat * 10 < delay) - assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) - assert(lastRecord.value().toInt === count) - assert(count === lastCommittableOffset.offset) - assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) - } - } - } diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index ac234b26..d111a6b7 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -44,6 +44,43 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { autoOffsetReset = AutoOffsetReset.Earliest ) + test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + withRunningKafka { + + val count = 250 + val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + } + .take(count) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe(100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } test("auto committable consumer with slow processing doesn't cause rebalancing") { withRunningKafka { diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 29902de2..d255090c 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -44,6 +44,44 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { autoOffsetReset = AutoOffsetReset.Earliest ) + test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + withRunningKafka { + + val count = 250 + val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + } + .take(count) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe(100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + test("auto committable consumer with slow processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 @@ -71,7 +109,7 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { .flatMap(Observable.fromIterable) .toListL - val (result, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second).executeAsync).runSyncUnsafe() + val result = Task.parZip2(listT.executeAsync, pushT.delayExecution(1.second)).map(_._1).runSyncUnsafe() assert(result.map(_.toInt).sum === (0 until count).sum) } } diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 171d88ee..a0e647f4 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -30,6 +30,44 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { autoOffsetReset = AutoOffsetReset.Earliest ) + test("slow downstream with small poll heartbeat and manual async commit keeps the consumer assignment") { + withRunningKafka { + + val count = 250 + val topicName = "monix-kafka-manual-commit-tests" + val delay = 200.millis + val pollHeartbeat = 2.millis + val fastPollHeartbeatConfig = + consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + + val producer = KafkaProducer[String, String](producerCfg, io) + val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) + + val pushT = Observable + .fromIterable(1 to count) + .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) + .mapEval(producer.send) + .lastL + + val listT = consumer + .executeOn(io) + .doOnNextF { committableMessage => + Task.sleep(delay) *> + CommittableOffsetBatch(Seq(committableMessage.committableOffset)).commitAsync().executeAsync + } + .take(count) + .toListL + + val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.executeAsync).runSyncUnsafe(100.seconds) + val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last + assert(pollHeartbeat * 10 < delay) + assert((1 to count).sum === committableMessages.map(_.record.value().toInt).sum) + assert(lastRecord.value().toInt === count) + assert(count === lastCommittableOffset.offset) + assert(new TopicPartition(topicName, 0) === lastCommittableOffset.topicPartition) + } + } + test("auto committable consumer with slow processing doesn't cause rebalancing") { withRunningKafka { val count = 10000 From b0316b61e7b1c75cfd8667bf6ab16c0cfff9473f Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 00:05:14 +0200 Subject: [PATCH 46/61] Kafka tests running on github actions --- .github/workflow/build.yml | 48 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 .github/workflow/build.yml diff --git a/.github/workflow/build.yml b/.github/workflow/build.yml new file mode 100644 index 00000000..f93c574b --- /dev/null +++ b/.github/workflow/build.yml @@ -0,0 +1,48 @@ +name: build + +on: [push, pull_request] + +jobs: + + tests: + name: scala-${{ matrix.scala }} jdk-${{ matrix.java }} tests + runs-on: ubuntu-latest + + strategy: + fail-fast: true + matrix: + java: [8] + scala: [2.11.12, 2.12.10] + + steps: + - uses: actions/checkout@v2 + - uses: olafurpg/setup-scala@v7 + with: + java-version: "adopt@1.${{ matrix.java }}" + + - name: Cache SBT Coursier directory + uses: actions/cache@v1 + with: + path: ~/.cache/coursier/v1 + key: ${{ runner.os }}-coursier-${{ hashFiles('**/*.sbt') }} + restore-keys: | + ${{ runner.os }}-coursier- + - name: Cache SBT directory + uses: actions/cache@v1 + with: + path: ~/.sbt + key: | + ${{ runner.os }}-sbt-${{ hashFiles('project/build.properties') }}-${{ hashFiles('project/plugins.sbt') }} + restore-keys: ${{ runner.os }}-sbt- + + - name: Run Tests for Kafka 0.9.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} + run: sbt -J-Xmx6144m kafka9/test + + - name: Run Tests for Kafka 0.10.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} + run: sbt -J-Xmx6144m kafka10/test + + - name: Run Tests for Kafka 0.11.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} + run: sbt -J-Xmx6144m kafka11/test + + - name: Run Tests for Kafka 1.x.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} + run: sbt -J-Xmx6144m kafka1x/test From 38b31c7f41f02bcb03bd3b548ed1d82ac39ab49a Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 00:39:27 +0200 Subject: [PATCH 47/61] Renames github workflows folder --- .github/{workflow => workflows}/build.yml | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename .github/{workflow => workflows}/build.yml (100%) diff --git a/.github/workflow/build.yml b/.github/workflows/build.yml similarity index 100% rename from .github/workflow/build.yml rename to .github/workflows/build.yml From c61857f1320e0b50fadef405166d9aedb8330a4c Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 00:44:12 +0200 Subject: [PATCH 48/61] Updates scala action --- .github/workflows/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index f93c574b..8f3b243c 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -16,7 +16,7 @@ jobs: steps: - uses: actions/checkout@v2 - - uses: olafurpg/setup-scala@v7 + - uses: olafurpg/setup-scala@v10 with: java-version: "adopt@1.${{ matrix.java }}" From edeb1795648d56de674dad14cfd89f61d278abf6 Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 09:37:41 +0200 Subject: [PATCH 49/61] Removes tests for kafka 0.9.x Revmoves travis-ci --- .github/workflows/build.yml | 3 --- .travis.yml | 37 ------------------------------------- 2 files changed, 40 deletions(-) delete mode 100644 .travis.yml diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 8f3b243c..16039333 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -35,9 +35,6 @@ jobs: ${{ runner.os }}-sbt-${{ hashFiles('project/build.properties') }}-${{ hashFiles('project/plugins.sbt') }} restore-keys: ${{ runner.os }}-sbt- - - name: Run Tests for Kafka 0.9.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} - run: sbt -J-Xmx6144m kafka9/test - - name: Run Tests for Kafka 0.10.x, Java ${{ matrix.java }} and Scala ${{ matrix.scala }} run: sbt -J-Xmx6144m kafka10/test diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index d284396e..00000000 --- a/.travis.yml +++ /dev/null @@ -1,37 +0,0 @@ -language: scala -sudo: required -dist: trusty -group: edge - -matrix: - include: - # Scala 2.11 - - jdk: oraclejdk8 - scala: 2.11.12 - env: COMMAND=ci - # Scala 2.12 - - jdk: oraclejdk8 - scala: 2.12.13 - env: COMMAND=ci - -env: - global: - - TRAVIS_NODE_VERSION="8.9" # LTS - -# http://austinpray.com/ops/2015/09/20/change-travis-node-version.html -install: -- rm -rf ~/.nvm && git clone https://github.com/creationix/nvm.git ~/.nvm && (cd ~/.nvm && git checkout `git describe --abbrev=0 --tags`) && source ~/.nvm/nvm.sh && nvm install $TRAVIS_NODE_VERSION - -script: -- export SBT_PROFILE=$COVERAGE -- sbt -J-Xmx6144m ++$TRAVIS_SCALA_VERSION $COMMAND - -cache: - directories: - - $HOME/.ivy2/cache - - $HOME/.coursier/cache - - $HOME/.sbt - -before_cache: -- find $HOME/.sbt -name "*.lock" -type f -delete -- find $HOME/.ivy2/cache -name "ivydata-*.properties" -type f -delete \ No newline at end of file From 8c09b6ac8dc5eb1e7e2615db663b6b28407754c9 Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 10:59:07 +0200 Subject: [PATCH 50/61] Add benchmark with autocomit async and sync scenarios --- benchmarks/auto-consumer-benchmark.txt | 139 ++++++++++++++++++ benchmarks/auto-consumer-benchmark7.txt | 69 +++++++++ benchmarks/results/consumer.md | 9 +- .../kafka/benchmarks/ConsumerBenchmark.scala | 21 ++- .../scala/monix/kafka/SerializationTest.scala | 2 +- .../scala/monix/kafka/SerializationTest.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../scala/monix/kafka/SerializationTest.scala | 2 +- 8 files changed, 236 insertions(+), 10 deletions(-) create mode 100644 benchmarks/auto-consumer-benchmark.txt create mode 100644 benchmarks/auto-consumer-benchmark7.txt diff --git a/benchmarks/auto-consumer-benchmark.txt b/benchmarks/auto-consumer-benchmark.txt new file mode 100644 index 00000000..d007047c --- /dev/null +++ b/benchmarks/auto-consumer-benchmark.txt @@ -0,0 +1,139 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms + +# Run progress: 0.00% complete, ETA 00:05:30 +# Fork: 1 of 1 +# Warmup Iteration 1: 6.111 ops/s +Iteration 1: 5.604 ops/s +Iteration 2: 4.208 ops/s +Iteration 3: 4.286 ops/s +Iteration 4: 4.115 ops/s +Iteration 5: 4.269 ops/s +Iteration 6: 4.145 ops/s +Iteration 7: 4.114 ops/s +Iteration 8: 3.722 ops/s +Iteration 9: 4.439 ops/s +Iteration 10: 4.627 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms": + 4.353 ±(99.9%) 0.753 ops/s [Average] + (min, avg, max) = (3.722, 4.353, 5.604), stdev = 0.498 + CI (99.9%): [3.600, 5.106] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixManualCommitHeartbeat10ms + +# Run progress: 33.33% complete, ETA 00:04:16 +# Fork: 1 of 1 +# Warmup Iteration 1: + +java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:78) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:78) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + +java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_thrpt_jmhStub(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:119) + at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:83) + at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) + at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) + at java.lang.Thread.run(Thread.java:748) + + + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 10 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 3 threads, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms + +# Run progress: 66.67% complete, ETA 00:01:06 +# Fork: 1 of 1 +# Warmup Iteration 1: 14.815 ops/s +Iteration 1: 15.612 ops/s +Iteration 2: 14.008 ops/s +Iteration 3: 15.498 ops/s +Iteration 4: 15.792 ops/s +Iteration 5: 16.849 ops/s +Iteration 6: 14.517 ops/s +Iteration 7: 16.839 ops/s +Iteration 8: 16.595 ops/s +Iteration 9: 17.230 ops/s +Iteration 10: 17.387 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms": + 16.033 ±(99.9%) 1.732 ops/s [Average] + (min, avg, max) = (14.008, 16.033, 17.387), stdev = 1.146 + CI (99.9%): [14.300, 17.765] (assumes normal distribution) + + +# Run complete. Total time: 00:04:08 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 10 4.353 ± 0.753 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 10 16.033 ± 1.732 ops/s diff --git a/benchmarks/auto-consumer-benchmark7.txt b/benchmarks/auto-consumer-benchmark7.txt new file mode 100644 index 00000000..42ce3a66 --- /dev/null +++ b/benchmarks/auto-consumer-benchmark7.txt @@ -0,0 +1,69 @@ +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 7 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms + +# Run progress: 0.00% complete, ETA 00:02:40 +# Fork: 1 of 1 +# Warmup Iteration 1: 5.707 ops/s +Iteration 1: 4.243 ops/s +Iteration 2: 3.605 ops/s +Iteration 3: 3.262 ops/s +Iteration 4: 3.308 ops/s +Iteration 5: 3.326 ops/s +Iteration 6: 3.435 ops/s +Iteration 7: 3.617 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms": + 3.542 ±(99.9%) 0.765 ops/s [Average] + (min, avg, max) = (3.262, 3.542, 4.243), stdev = 0.340 + CI (99.9%): [2.777, 4.308] (assumes normal distribution) + + +# JMH version: 1.21 +# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 +# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java +# VM options: +# Warmup: 1 iterations, 10 s each +# Measurement: 7 iterations, 10 s each +# Timeout: 10 min per iteration +# Threads: 1 thread, will synchronize iterations +# Benchmark mode: Throughput, ops/time +# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms + +# Run progress: 50.00% complete, ETA 00:01:24 +# Fork: 1 of 1 +# Warmup Iteration 1: 10.047 ops/s +Iteration 1: 11.621 ops/s +Iteration 2: 12.044 ops/s +Iteration 3: 12.034 ops/s +Iteration 4: 11.083 ops/s +Iteration 5: 10.559 ops/s +Iteration 6: 10.934 ops/s +Iteration 7: 11.266 ops/s + + +Result "monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms": + 11.363 ±(99.9%) 1.267 ops/s [Average] + (min, avg, max) = (10.559, 11.363, 12.044), stdev = 0.563 + CI (99.9%): [10.096, 12.630] (assumes normal distribution) + + +# Run complete. Total time: 00:02:47 + +REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on +why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial +experiments, perform baseline and negative tests that provide experimental control, make sure +the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. +Do not assume the numbers tell you what you want them to tell. + +Benchmark Mode Cnt Score Error Units +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 3.542 ± 0.765 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 11.363 ± 1.267 ops/s diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 9932a5eb..8d6e902d 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -3,7 +3,8 @@ ### 1fork 1thread Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monix_auto_commit thrpt 10 11.912 ± 0.617 ops/s +ConsumerBenchmark.monix_auto_commit_async thrpt 7 13.097 ± 0.827 ops/s +ConsumerBenchmark.monix_auto_commit_sync thrpt 7 12.486 ± 1.087 ops/s ConsumerBenchmark.monix_manual_commit thrpt 10 11.519 ± 2.247 ops/s ### 1 fork 3 threads @@ -16,7 +17,8 @@ ConsumerBenchmark.monix_manual_commit thrpt 10 16.319 ± 1.465 ops/s ### 1fork 1thread --- Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAutoCommitHeartbeat10ms thrpt 7 4.865 ± 1.044 ops/s +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 3.542 ± 0.765 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 11.363 ± 1.267 ops/s ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s @@ -26,7 +28,8 @@ ConsumerBenchmark.monixManualCommitHeartbeat50ms thrpt 7 11.900 ± 0.698 --- ### 1 fork 3 threads Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAutoCommitHeartbeat10ms thrpt 7 4.865 ± 1.044 ops/s +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 4.353 ± 0.753 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 16.033 ± 1.732 ops/s ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 5e640c00..2907dc46 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -1,5 +1,6 @@ package monix.kafka.benchmarks +import monix.kafka.config.ObservableCommitOrder.BeforeAck import monix.kafka.config.ObservableCommitType import java.util.concurrent.TimeUnit @@ -35,11 +36,12 @@ class ConsumerBenchmark extends MonixFixture { } @Benchmark - def monixAutoCommitHeartbeat10ms(): Unit = { + def monixAsyncAutoCommitHeartbeat15ms(): Unit = { val conf = consumerConf.value().copy( maxPollRecords = maxPollRecords, - observableCommitType = ObservableCommitType.Async) - .withPollHeartBeatRate(10.millis) + observableCommitType = ObservableCommitType.Async, + observableCommitOrder = BeforeAck) + .withPollHeartBeatRate(15.millis) KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) .take(consumedRecords) @@ -47,6 +49,19 @@ class ConsumerBenchmark extends MonixFixture { .runSyncUnsafe() } + @Benchmark + def monixSyncAutoCommitHeartbeat15ms(): Unit = { + val conf = consumerConf.value().copy( + maxPollRecords = maxPollRecords, + observableCommitType = ObservableCommitType.Sync, + observableCommitOrder = BeforeAck) + .withPollHeartBeatRate(15.millis) + + KafkaConsumerObservable[Integer, Integer](conf, List(monixTopic)) + .take(consumedRecords) + .headL + .runSyncUnsafe() + } @Benchmark def monixManualCommitHeartbeat1ms(): Unit = { diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/SerializationTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/SerializationTest.scala index e1941115..1a80c91d 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/SerializationTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/SerializationTest.scala @@ -96,7 +96,7 @@ class SerializationTest extends FunSuite with KafkaTestKit { .map(_.value()) .toListL - val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 70.seconds) assert(result.map(_.value.toInt).sum === (0 until count).filter(_ % 2 == 0).sum) } } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/SerializationTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/SerializationTest.scala index e1941115..1a80c91d 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/SerializationTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/SerializationTest.scala @@ -96,7 +96,7 @@ class SerializationTest extends FunSuite with KafkaTestKit { .map(_.value()) .toListL - val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 70.seconds) assert(result.map(_.value.toInt).sum === (0 until count).filter(_ % 2 == 0).sum) } } diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 81630129..d2a556c5 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -88,7 +88,7 @@ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( try consumer.synchronized { val assignment = consumer.assignment() consumer.resume(assignment) - val next = blocking(consumer.poll(0)) + val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment) val result = next.asScala.map { record => CommittableMessage( diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/SerializationTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/SerializationTest.scala index e1941115..1a80c91d 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/SerializationTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/SerializationTest.scala @@ -96,7 +96,7 @@ class SerializationTest extends FunSuite with KafkaTestKit { .map(_.value()) .toListL - val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 60.seconds) + val (result, _) = Await.result(Task.parZip2(listT, pushT).runToFuture, 70.seconds) assert(result.map(_.value.toInt).sum === (0 until count).filter(_ % 2 == 0).sum) } } From d54ed10f46387b1f90c1cc028937c29be1a13641 Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 2 Apr 2021 11:00:21 +0200 Subject: [PATCH 51/61] Removed benchmark results file --- benchmarks/auto-consumer-benchmark.txt | 139 ------------------------ benchmarks/auto-consumer-benchmark7.txt | 69 ------------ 2 files changed, 208 deletions(-) delete mode 100644 benchmarks/auto-consumer-benchmark.txt delete mode 100644 benchmarks/auto-consumer-benchmark7.txt diff --git a/benchmarks/auto-consumer-benchmark.txt b/benchmarks/auto-consumer-benchmark.txt deleted file mode 100644 index d007047c..00000000 --- a/benchmarks/auto-consumer-benchmark.txt +++ /dev/null @@ -1,139 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms - -# Run progress: 0.00% complete, ETA 00:05:30 -# Fork: 1 of 1 -# Warmup Iteration 1: 6.111 ops/s -Iteration 1: 5.604 ops/s -Iteration 2: 4.208 ops/s -Iteration 3: 4.286 ops/s -Iteration 4: 4.115 ops/s -Iteration 5: 4.269 ops/s -Iteration 6: 4.145 ops/s -Iteration 7: 4.114 ops/s -Iteration 8: 3.722 ops/s -Iteration 9: 4.439 ops/s -Iteration 10: 4.627 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms": - 4.353 ±(99.9%) 0.753 ops/s [Average] - (min, avg, max) = (3.722, 4.353, 5.604), stdev = 0.498 - CI (99.9%): [3.600, 5.106] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixManualCommitHeartbeat10ms - -# Run progress: 33.33% complete, ETA 00:04:16 -# Fork: 1 of 1 -# Warmup Iteration 1: - -java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:78) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:78) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - -java.lang.NoSuchMethodError: monix.kafka.benchmarks.generated.ConsumerBenchmark_jmhType.monixManualCommitHeartbeat10ms()V - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_thrpt_jmhStub(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:119) - at monix.kafka.benchmarks.generated.ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.monixManualCommitHeartbeat10ms_Throughput(ConsumerBenchmark_monixManualCommitHeartbeat10ms_jmhTest.java:83) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453) - at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) - - - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 10 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 3 threads, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms - -# Run progress: 66.67% complete, ETA 00:01:06 -# Fork: 1 of 1 -# Warmup Iteration 1: 14.815 ops/s -Iteration 1: 15.612 ops/s -Iteration 2: 14.008 ops/s -Iteration 3: 15.498 ops/s -Iteration 4: 15.792 ops/s -Iteration 5: 16.849 ops/s -Iteration 6: 14.517 ops/s -Iteration 7: 16.839 ops/s -Iteration 8: 16.595 ops/s -Iteration 9: 17.230 ops/s -Iteration 10: 17.387 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms": - 16.033 ±(99.9%) 1.732 ops/s [Average] - (min, avg, max) = (14.008, 16.033, 17.387), stdev = 1.146 - CI (99.9%): [14.300, 17.765] (assumes normal distribution) - - -# Run complete. Total time: 00:04:08 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 10 4.353 ± 0.753 ops/s -ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 10 16.033 ± 1.732 ops/s diff --git a/benchmarks/auto-consumer-benchmark7.txt b/benchmarks/auto-consumer-benchmark7.txt deleted file mode 100644 index 42ce3a66..00000000 --- a/benchmarks/auto-consumer-benchmark7.txt +++ /dev/null @@ -1,69 +0,0 @@ -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 7 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms - -# Run progress: 0.00% complete, ETA 00:02:40 -# Fork: 1 of 1 -# Warmup Iteration 1: 5.707 ops/s -Iteration 1: 4.243 ops/s -Iteration 2: 3.605 ops/s -Iteration 3: 3.262 ops/s -Iteration 4: 3.308 ops/s -Iteration 5: 3.326 ops/s -Iteration 6: 3.435 ops/s -Iteration 7: 3.617 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms": - 3.542 ±(99.9%) 0.765 ops/s [Average] - (min, avg, max) = (3.262, 3.542, 4.243), stdev = 0.340 - CI (99.9%): [2.777, 4.308] (assumes normal distribution) - - -# JMH version: 1.21 -# VM version: JDK 1.8.0_221, Java HotSpot(TM) 64-Bit Server VM, 25.221-b11 -# VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_221.jdk/Contents/Home/jre/bin/java -# VM options: -# Warmup: 1 iterations, 10 s each -# Measurement: 7 iterations, 10 s each -# Timeout: 10 min per iteration -# Threads: 1 thread, will synchronize iterations -# Benchmark mode: Throughput, ops/time -# Benchmark: monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms - -# Run progress: 50.00% complete, ETA 00:01:24 -# Fork: 1 of 1 -# Warmup Iteration 1: 10.047 ops/s -Iteration 1: 11.621 ops/s -Iteration 2: 12.044 ops/s -Iteration 3: 12.034 ops/s -Iteration 4: 11.083 ops/s -Iteration 5: 10.559 ops/s -Iteration 6: 10.934 ops/s -Iteration 7: 11.266 ops/s - - -Result "monix.kafka.benchmarks.ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms": - 11.363 ±(99.9%) 1.267 ops/s [Average] - (min, avg, max) = (10.559, 11.363, 12.044), stdev = 0.563 - CI (99.9%): [10.096, 12.630] (assumes normal distribution) - - -# Run complete. Total time: 00:02:47 - -REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on -why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial -experiments, perform baseline and negative tests that provide experimental control, make sure -the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. -Do not assume the numbers tell you what you want them to tell. - -Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 3.542 ± 0.765 ops/s -ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 11.363 ± 1.267 ops/s From 9bcd8ac8147df27e582dfe11440e039120f1d5df Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 10:00:32 +0200 Subject: [PATCH 52/61] Some clean up --- .../monix/kafka/KafkaConsumerConfig.scala | 2 +- .../monix/kafka/KafkaConsumerObservable.scala | 4 ++-- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../monix/kafka/KafkaConsumerConfig.scala | 5 +---- .../monix/kafka/KafkaConsumerObservable.scala | 4 ++-- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- kafka-1.0.x/src/main/resources/logback.xml | 19 ++++++++----------- .../monix/kafka/KafkaConsumerConfig.scala | 5 +---- .../monix/kafka/KafkaConsumerObservable.scala | 4 ++-- .../KafkaConsumerObservableAutoCommit.scala | 2 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- 13 files changed, 23 insertions(+), 32 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index b3cde65f..58c8f5ba 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -289,7 +289,7 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 100.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 15.millis @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 9038dad5..ec28524b 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -39,7 +39,7 @@ import scala.util.matching.Regex */ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumerTask: Task[Consumer[K, V]] + protected def consumerT: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -66,7 +66,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - consumerTask.bracket { c => + consumerT.bracket { c => // Skipping all available messages on all partitions if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index d74de333..911bd97a 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 8c1e494a..a8c188d1 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index d4c781d8..7dc0e606 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -202,9 +202,6 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. - * Specifies heartbeat time between KafkaConsumer#poll attempts. - * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -295,7 +292,7 @@ final case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 15.millis @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index fec4b0f2..6e2a8502 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -40,7 +40,7 @@ import scala.util.matching.Regex trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { protected def config: KafkaConsumerConfig - protected def consumerTask: Task[Consumer[K, V]] + protected def consumerT: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -68,7 +68,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - consumerTask.bracket { c => + consumerT.bracket { c => // Skipping all available messages on all partitions if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index d74de333..911bd97a 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index bbf697b0..00cc3231 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles diff --git a/kafka-1.0.x/src/main/resources/logback.xml b/kafka-1.0.x/src/main/resources/logback.xml index 327dc7d7..dd6cb2d2 100644 --- a/kafka-1.0.x/src/main/resources/logback.xml +++ b/kafka-1.0.x/src/main/resources/logback.xml @@ -1,13 +1,14 @@ - - - - - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n - - + + + %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n + + + + + @@ -18,8 +19,4 @@ - - - - \ No newline at end of file diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index bac8cadf..b1254626 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -207,9 +207,6 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param observablePollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. - * Specifies heartbeat time between KafkaConsumer#poll attempts. - * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate @@ -302,7 +299,7 @@ case class KafkaConsumerConfig( "retry.backoff.ms" -> retryBackoffTime.toMillis.toString ) - private[kafka] var pollHeartbeatRate: FiniteDuration = 50.millis + private[kafka] var pollHeartbeatRate: FiniteDuration = 15.millis @InternalApi private[kafka] def withPollHeartBeatRate(interval: FiniteDuration): KafkaConsumerConfig = { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index ce04ad8d..5598fc71 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -40,7 +40,7 @@ import scala.concurrent.duration._ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogging { protected def config: KafkaConsumerConfig - protected def consumerTask: Task[Consumer[K, V]] + protected def consumerT: Task[Consumer[K, V]] @volatile protected var isAcked = true @@ -68,7 +68,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg Task.create { (scheduler, cb) => implicit val s = scheduler val startConsuming = - consumerTask.bracket { c => + consumerT.bracket { c => // Skipping all available messages on all partitions if (config.observableSeekOnStart.isSeekEnd) c.seekToEnd(Nil.asJavaCollection) else if (config.observableSeekOnStart.isSeekBeginning) c.seekToBeginning(Nil.asJavaCollection) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index 719863f0..e60554ec 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success} */ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] { /* Based on the [[KafkaConsumerConfig.observableCommitType]] it diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index d2a556c5..4f313c60 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -37,7 +37,7 @@ import scala.jdk.CollectionConverters._ */ final class KafkaConsumerObservableManualCommit[K, V] private[kafka] ( override protected val config: KafkaConsumerConfig, - override protected val consumerTask: Task[Consumer[K, V]]) + override protected val consumerT: Task[Consumer[K, V]]) extends KafkaConsumerObservable[K, V, CommittableMessage[K, V]] { // Caching value to save CPU cycles From 6f2fa50a44962e4642d0cf33d5db93e69bff394e Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 10:56:04 +0200 Subject: [PATCH 53/61] Use pollTimeout and updated benchmark results --- benchmarks/results/consumer.md | 8 ++++---- build.sbt | 2 +- .../monix/kafka/KafkaConsumerObservableAutoCommit.scala | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 8d6e902d..284548b1 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -17,8 +17,8 @@ ConsumerBenchmark.monix_manual_commit thrpt 10 16.319 ± 1.465 ops/s ### 1fork 1thread --- Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 3.542 ± 0.765 ops/s -ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 11.363 ± 1.267 ops/s +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat15ms thrpt 7 13.126 ± 1.737 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat15ms thrpt 7 12.102 ± 1.214 ops/s ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s @@ -28,8 +28,8 @@ ConsumerBenchmark.monixManualCommitHeartbeat50ms thrpt 7 11.900 ± 0.698 --- ### 1 fork 3 threads Benchmark Mode Cnt Score Error Units -ConsumerBenchmark.monixAsyncAutoCommitHeartbeat10ms thrpt 7 4.353 ± 0.753 ops/s -ConsumerBenchmark.monixSyncAutoCommitHeartbeat10ms thrpt 7 16.033 ± 1.732 ops/s +ConsumerBenchmark.monixAsyncAutoCommitHeartbeat15ms thrpt 7 16.966 ± 2.659 ops/s +ConsumerBenchmark.monixSyncAutoCommitHeartbeat15ms thrpt 7 15.083 ± 4.242 ops/s ConsumerBenchmark.monixManualCommitHeartbeat1000ms thrpt 7 2.978 ± 0.006 ops/s ConsumerBenchmark.monixManualCommitHeartbeat100ms thrpt 7 9.961 ± 1.317 ops/s ConsumerBenchmark.monixManualCommitHeartbeat10ms thrpt 7 13.346 ± 0.716 ops/s diff --git a/build.sbt b/build.sbt index 9ffe65af..e17931e4 100644 --- a/build.sbt +++ b/build.sbt @@ -217,7 +217,7 @@ lazy val kafka1x = project.in(file("kafka-1.0.x")) if (!(scalaVersion.value startsWith "2.13")) Seq("net.manub" %% "scalatest-embedded-kafka" % "1.1.1" % "test" exclude ("log4j", "log4j")) else Seq.empty[ModuleID] }, - libraryDependencies += "org.apache.kafka" % "kafka-clients" % "1.1.1" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") + libraryDependencies += "org.apache.kafka" % "kafka-clients" % "1.0.2" exclude("org.slf4j","slf4j-log4j12") exclude("log4j", "log4j") ) lazy val kafka11 = project.in(file("kafka-0.11.x")) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala index e60554ec..911bd97a 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableAutoCommit.scala @@ -72,7 +72,7 @@ final class KafkaConsumerObservableAutoCommit[K, V] private[kafka] ( if (cancelable.isCanceled) Stop else { consumer.resume(assignment) - val next = blocking(consumer.poll(0)) + val next = blocking(consumer.poll(pollTimeoutMillis)) consumer.pause(assignment) if (shouldCommitBefore) consumerCommit(consumer) // Feeding the observer happens on the Subscriber's scheduler From 50c3d9d046fbf3efb92798f98655e7ca9c2fc1c7 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 11:05:54 +0200 Subject: [PATCH 54/61] ScalafmtAll --- .../monix/kafka/KafkaConsumerConfig.scala | 85 +++++++++---------- .../monix/kafka/KafkaConsumerObservable.scala | 7 +- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../kafka/MonixKafkaTopicRegexTest.scala | 2 +- .../scala/monix/kafka/PollHeartbeatTest.scala | 15 ++-- .../monix/kafka/KafkaConsumerObservable.scala | 39 +++++---- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../kafka/MonixKafkaTopicRegexTest.scala | 3 +- .../scala/monix/kafka/PollHeartbeatTest.scala | 15 ++-- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../scala/monix/kafka/MonixKafkaTest.scala | 3 +- .../monix/kafka/KafkaConsumerObservable.scala | 42 ++++----- .../KafkaConsumerObservableManualCommit.scala | 2 +- .../monix/kafka/MonixKafkaTopicListTest.scala | 2 +- .../kafka/MonixKafkaTopicRegexTest.scala | 2 +- .../scala/monix/kafka/PollHeartbeatTest.scala | 15 ++-- 16 files changed, 124 insertions(+), 114 deletions(-) diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala index 58c8f5ba..007c694a 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerConfig.scala @@ -198,56 +198,53 @@ import scala.concurrent.duration._ * Specifies when the commit should happen, like before we receive the * acknowledgement from downstream, or afterwards. * - * @param pollHeartbeatRate is the `monix.observable.poll.heartbeat.rate.ms` setting. - * Specifies heartbeat time between KafkaConsumer#poll attempts. - * * @param properties map of other properties that will be passed to * the underlying kafka client. Any properties not explicitly handled * by this object can be set via the map, but in case of a duplicate * a value set on the case class will overwrite value set via properties. */ final case class KafkaConsumerConfig( - bootstrapServers: List[String], - fetchMinBytes: Int, - groupId: String, - heartbeatInterval: FiniteDuration, - maxPartitionFetchBytes: Int, - sessionTimeout: FiniteDuration, - sslKeyPassword: Option[String], - sslKeyStorePassword: Option[String], - sslKeyStoreLocation: Option[String], - sslTrustStoreLocation: Option[String], - sslTrustStorePassword: Option[String], - autoOffsetReset: AutoOffsetReset, - connectionsMaxIdleTime: FiniteDuration, - enableAutoCommit: Boolean, - excludeInternalTopics: Boolean, - maxPollRecords: Int, - maxPollInterval: FiniteDuration, - receiveBufferInBytes: Int, - requestTimeout: FiniteDuration, - saslKerberosServiceName: Option[String], - saslMechanism: String, - securityProtocol: SecurityProtocol, - sendBufferInBytes: Int, - sslEnabledProtocols: List[SSLProtocol], - sslKeystoreType: String, - sslProtocol: SSLProtocol, - sslProvider: Option[String], - sslTruststoreType: String, - checkCRCs: Boolean, - clientId: String, - fetchMaxWaitTime: FiniteDuration, - metadataMaxAge: FiniteDuration, - metricReporters: List[String], - metricsNumSamples: Int, - metricsSampleWindow: FiniteDuration, - reconnectBackoffTime: FiniteDuration, - retryBackoffTime: FiniteDuration, - observableCommitType: ObservableCommitType, - observableCommitOrder: ObservableCommitOrder, - observableSeekOnStart: ObservableSeekOnStart, - properties: Map[String, String]) { + bootstrapServers: List[String], + fetchMinBytes: Int, + groupId: String, + heartbeatInterval: FiniteDuration, + maxPartitionFetchBytes: Int, + sessionTimeout: FiniteDuration, + sslKeyPassword: Option[String], + sslKeyStorePassword: Option[String], + sslKeyStoreLocation: Option[String], + sslTrustStoreLocation: Option[String], + sslTrustStorePassword: Option[String], + autoOffsetReset: AutoOffsetReset, + connectionsMaxIdleTime: FiniteDuration, + enableAutoCommit: Boolean, + excludeInternalTopics: Boolean, + maxPollRecords: Int, + maxPollInterval: FiniteDuration, + receiveBufferInBytes: Int, + requestTimeout: FiniteDuration, + saslKerberosServiceName: Option[String], + saslMechanism: String, + securityProtocol: SecurityProtocol, + sendBufferInBytes: Int, + sslEnabledProtocols: List[SSLProtocol], + sslKeystoreType: String, + sslProtocol: SSLProtocol, + sslProvider: Option[String], + sslTruststoreType: String, + checkCRCs: Boolean, + clientId: String, + fetchMaxWaitTime: FiniteDuration, + metadataMaxAge: FiniteDuration, + metricReporters: List[String], + metricsNumSamples: Int, + metricsSampleWindow: FiniteDuration, + reconnectBackoffTime: FiniteDuration, + retryBackoffTime: FiniteDuration, + observableCommitType: ObservableCommitType, + observableCommitOrder: ObservableCommitOrder, + observableSeekOnStart: ObservableSeekOnStart, + properties: Map[String, String]) { def toMap: Map[String, String] = properties ++ Map( "bootstrap.servers" -> bootstrapServers.mkString(","), diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index ec28524b..6926de87 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -116,11 +116,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } } + }.onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.second) } - .onErrorHandleWith { ex => - Task.now(scheduler.reportFailure(ex)) >> - Task.sleep(1.second) - } } } diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index a8c188d1..1f924b96 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{Future, blocking} +import scala.concurrent.{blocking, Future} import scala.util.control.NonFatal import scala.util.{Failure, Success} diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index b4943dda..f09c6a1b 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -105,4 +105,4 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } -} \ No newline at end of file +} diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index d111a6b7..6fa5bc85 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -123,7 +123,9 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val maxPollInterval = 10.millis val maxPollRecords = 1 val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + consumerCfg + .copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -137,14 +139,16 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } .take(totalRecords) .toListL - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val (committableMessages, _) = + Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert(pollHeartbeat < maxPollInterval) @@ -179,7 +183,8 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } @@ -213,7 +218,7 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { // but smaller than `pollHeartBeat`, kafka will trigger rebalance // and the consumer will be kicked out of the consumer group. val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 6e2a8502..772edc94 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -118,11 +118,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { } } } + }.onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.seconds) } - .onErrorHandleWith { ex => - Task.now(scheduler.reportFailure(ex)) >> - Task.sleep(1.seconds) - } } } @@ -138,8 +137,8 @@ object KafkaConsumerObservable { * instance to use for consuming from Kafka */ def apply[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = new KafkaConsumerObservableAutoCommit[K, V](cfg, consumer) /** Builds a [[KafkaConsumerObservable]] instance. @@ -150,8 +149,8 @@ object KafkaConsumerObservable { * @param topics is the list of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) apply(cfg, consumer) @@ -165,8 +164,8 @@ object KafkaConsumerObservable { * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) apply(cfg, consumer) @@ -195,8 +194,8 @@ object KafkaConsumerObservable { * instance to use for consuming from Kafka */ def manualCommit[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val manualCommitConfig = cfg.copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) new KafkaConsumerObservableManualCommit[K, V](manualCommitConfig, consumer) @@ -223,8 +222,8 @@ object KafkaConsumerObservable { * @param topics is the list of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) manualCommit(cfg, consumer) @@ -251,8 +250,8 @@ object KafkaConsumerObservable { * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) manualCommit(cfg, consumer) @@ -260,8 +259,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given list of topics. */ def createConsumer[K, V](config: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap @@ -275,8 +274,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given topics regex. */ def createConsumer[K, V](config: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap blocking { diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 00cc3231..11331b2e 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{Future, blocking} +import scala.concurrent.{blocking, Future} import scala.util.control.NonFatal import scala.util.{Failure, Success} diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index b3ae8649..17e2b090 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -90,7 +90,8 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { val producer = KafkaProducerSink[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerCfg, topicsRegex) - .executeOn(io).take(count) + .executeOn(io) + .take(count) val pushT = Observable .range(0, count) diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index d255090c..3ba70e4f 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -123,7 +123,9 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val maxPollInterval = 10.millis val maxPollRecords = 1 val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + consumerCfg + .copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -137,14 +139,16 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } .take(totalRecords) .toListL - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val (committableMessages, _) = + Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert(pollHeartbeat < maxPollInterval) @@ -179,7 +183,8 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } @@ -213,7 +218,7 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { // but smaller than `pollHeartBeat`, kafka will trigger rebalance // and the consumer will be kicked out of the consumer group. val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 981c6d87..2ad32cd8 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import java.util import scala.jdk.CollectionConverters._ -import scala.concurrent.{Future, blocking} +import scala.concurrent.{blocking, Future} import scala.util.control.NonFatal import scala.util.{Failure, Success} diff --git a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala index c031512d..213a854b 100644 --- a/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala +++ b/kafka-0.9.x/src/test/scala/monix/kafka/MonixKafkaTest.scala @@ -219,8 +219,7 @@ class MonixKafkaTest extends FunSuite { test("slow batches processing doesn't cause rebalancing") { val count = 10000 - val consumerConfig = consumerCfg.copy( - sessionTimeout = 200.millis) + val consumerConfig = consumerCfg.copy(sessionTimeout = 200.millis) val producer = KafkaProducerSink[String, String](producerCfg, io) val consumer = KafkaConsumerObservable[String, String](consumerConfig, List(topicName)).executeOn(io) diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 5598fc71..779b27d6 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -118,11 +118,10 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogg } } } + }.onErrorHandleWith { ex => + Task.now(scheduler.reportFailure(ex)) >> + Task.sleep(1.seconds) } - .onErrorHandleWith { ex => - Task.now(scheduler.reportFailure(ex)) >> - Task.sleep(1.seconds) - } } } @@ -139,8 +138,8 @@ object KafkaConsumerObservable { * instance to use for consuming from Kafka */ def apply[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = new KafkaConsumerObservableAutoCommit[K, V](cfg, consumer) /** Builds a [[KafkaConsumerObservable]] instance. @@ -151,8 +150,8 @@ object KafkaConsumerObservable { * @param topics is the list of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) apply(cfg, consumer) @@ -166,8 +165,8 @@ object KafkaConsumerObservable { * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def apply[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, ConsumerRecord[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) apply(cfg, consumer) @@ -196,10 +195,11 @@ object KafkaConsumerObservable { * instance to use for consuming from Kafka */ def manualCommit[K, V]( - cfg: KafkaConsumerConfig, - consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + cfg: KafkaConsumerConfig, + consumer: Task[Consumer[K, V]]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { - val manualCommitConfig = cfg.copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) + val manualCommitConfig = cfg + .copy(observableCommitOrder = ObservableCommitOrder.NoAck, enableAutoCommit = false) .withPollHeartBeatRate(cfg.pollHeartbeatRate) new KafkaConsumerObservableManualCommit[K, V](manualCommitConfig, consumer) } @@ -225,8 +225,8 @@ object KafkaConsumerObservable { * @param topics is the list of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topics) manualCommit(cfg, consumer) @@ -253,8 +253,8 @@ object KafkaConsumerObservable { * @param topicsRegex is the pattern of Kafka topics to subscribe to. */ def manualCommit[K, V](cfg: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): KafkaConsumerObservable[K, V, CommittableMessage[K, V]] = { val consumer = createConsumer[K, V](cfg, topicsRegex) manualCommit(cfg, consumer) @@ -262,8 +262,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given list of topics. */ def createConsumer[K, V](config: KafkaConsumerConfig, topics: List[String])(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap @@ -277,8 +277,8 @@ object KafkaConsumerObservable { /** Returns a `Task` for creating a consumer instance given topics regex. */ def createConsumer[K, V](config: KafkaConsumerConfig, topicsRegex: Regex)(implicit - K: Deserializer[K], - V: Deserializer[V]): Task[Consumer[K, V]] = { + K: Deserializer[K], + V: Deserializer[V]): Task[Consumer[K, V]] = { Task.evalAsync { val configMap = config.toJavaMap blocking { diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala index 4f313c60..4f83a455 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservableManualCommit.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.{Consumer, OffsetAndMetadata, OffsetCom import org.apache.kafka.common.TopicPartition import java.util -import scala.concurrent.{Future, blocking} +import scala.concurrent.{blocking, Future} import scala.util.control.NonFatal import scala.util.{Failure, Success} import scala.jdk.CollectionConverters._ diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 3ead708e..314c5a86 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -154,4 +154,4 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { assert(first.isDefined && second.isRight && third.isEmpty) } } -} \ No newline at end of file +} diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala index ff567170..d6f2335e 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/MonixKafkaTopicRegexTest.scala @@ -104,4 +104,4 @@ class MonixKafkaTopicRegexTest extends FunSuite with KafkaTestKit { assert(result.map(_.toInt).sum === (0 until count).sum) } } -} \ No newline at end of file +} diff --git a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index a0e647f4..2be9232e 100644 --- a/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-1.0.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -12,7 +12,7 @@ import org.scalatest.concurrent.ScalaFutures import scala.concurrent.duration._ -class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { +class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val topicName = "monix-kafka-tests" @@ -109,7 +109,9 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val maxPollInterval = 10.millis val maxPollRecords = 1 val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords).withPollHeartBeatRate(pollHeartbeat) + consumerCfg + .copy(maxPollInterval = 200.millis, maxPollRecords = maxPollRecords) + .withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -123,14 +125,16 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } .take(totalRecords) .toListL - val (committableMessages, _) = Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() + val (committableMessages, _) = + Task.parZip2(listT.executeAsync, pushT.delayExecution(100.millis).executeAsync).runSyncUnsafe() val CommittableMessage(lastRecord, lastCommittableOffset) = committableMessages.last assert(pollHeartbeat * 10 < downstreamLatency) assert(pollHeartbeat < maxPollInterval) @@ -165,7 +169,8 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val listT = consumer .executeOn(io) .mapEvalF { committableMessage => - val manualCommit = Task.defer(committableMessage.committableOffset.commitAsync()) + val manualCommit = Task + .defer(committableMessage.committableOffset.commitAsync()) .as(committableMessage) Task.sleep(downstreamLatency) *> manualCommit } From f7991ca78af03f2eda10114c6bb33125f00a535b Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 11:09:47 +0200 Subject: [PATCH 55/61] Ignore Ds_store file --- .gitignore | 2 ++ .../src/test/scala/monix/kafka/.DS_Store | Bin 6148 -> 0 bytes 2 files changed, 2 insertions(+) delete mode 100644 kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store diff --git a/.gitignore b/.gitignore index 5b5bab1b..c4af8e55 100644 --- a/.gitignore +++ b/.gitignore @@ -18,3 +18,5 @@ project/plugins/project/ .idea .bsp/sbt.json + +.DS_Store diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store b/kafka-0.11.x/src/test/scala/monix/kafka/.DS_Store deleted file mode 100644 index 5008ddfcf53c02e82d7eee2e57c38e5672ef89f6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeH~Jr2S!425mzP>H1@V-^m;4Wg<&0T*E43hX&L&p$$qDprKhvt+--jT7}7np#A3 zem<@ulZcFPQ@L2!n>{z**++&mCkOWA81W14cNZlEfg7;MkzE(HCqgga^y>{tEnwC%0;vJ&^%eQ zLs35+`xjp>T0 Date: Sat, 3 Apr 2021 11:17:53 +0200 Subject: [PATCH 56/61] Removes logback file from main --- kafka-0.10.x/src/test/resources/logback.xml | 11 +++++++ kafka-0.11.x/src/main/resources/logback.xml | 25 ---------------- .../src/test/resources/log4j.properties | 0 kafka-0.11.x/src/test/resources/log4j2.xml | 30 ------------------- .../src/test}/resources/logback.xml | 0 kafka-1.0.x/src/main/resources/logback.xml | 22 -------------- .../src/test/resources/logback-test.xml | 11 ++++++- 7 files changed, 21 insertions(+), 78 deletions(-) delete mode 100644 kafka-0.11.x/src/main/resources/logback.xml delete mode 100644 kafka-0.11.x/src/test/resources/log4j.properties delete mode 100644 kafka-0.11.x/src/test/resources/log4j2.xml rename {kafka-0.10.x/src/main => kafka-0.11.x/src/test}/resources/logback.xml (100%) delete mode 100644 kafka-1.0.x/src/main/resources/logback.xml diff --git a/kafka-0.10.x/src/test/resources/logback.xml b/kafka-0.10.x/src/test/resources/logback.xml index 2beb83c9..6b2cd9ed 100644 --- a/kafka-0.10.x/src/test/resources/logback.xml +++ b/kafka-0.10.x/src/test/resources/logback.xml @@ -8,4 +8,15 @@ + + + + + + + + + + + diff --git a/kafka-0.11.x/src/main/resources/logback.xml b/kafka-0.11.x/src/main/resources/logback.xml deleted file mode 100644 index 327dc7d7..00000000 --- a/kafka-0.11.x/src/main/resources/logback.xml +++ /dev/null @@ -1,25 +0,0 @@ - - - - - - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/kafka-0.11.x/src/test/resources/log4j.properties b/kafka-0.11.x/src/test/resources/log4j.properties deleted file mode 100644 index e69de29b..00000000 diff --git a/kafka-0.11.x/src/test/resources/log4j2.xml b/kafka-0.11.x/src/test/resources/log4j2.xml deleted file mode 100644 index bafe0424..00000000 --- a/kafka-0.11.x/src/test/resources/log4j2.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - logs - logs/archive - %-5level %d [%t] %c:%M(%L): %m%n - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/kafka-0.10.x/src/main/resources/logback.xml b/kafka-0.11.x/src/test/resources/logback.xml similarity index 100% rename from kafka-0.10.x/src/main/resources/logback.xml rename to kafka-0.11.x/src/test/resources/logback.xml diff --git a/kafka-1.0.x/src/main/resources/logback.xml b/kafka-1.0.x/src/main/resources/logback.xml deleted file mode 100644 index dd6cb2d2..00000000 --- a/kafka-1.0.x/src/main/resources/logback.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - %d{yyyyMMdd-HH:mm:ss.SSSZ} [%thread] %-5level %logger - %msg%n - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/kafka-1.0.x/src/test/resources/logback-test.xml b/kafka-1.0.x/src/test/resources/logback-test.xml index cc97f771..1e022eb2 100644 --- a/kafka-1.0.x/src/test/resources/logback-test.xml +++ b/kafka-1.0.x/src/test/resources/logback-test.xml @@ -9,8 +9,17 @@ - + + + + + + + + + + From f05abe6b5e9c22fba1f9d2781970fe230675a4ad Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 15:59:19 +0200 Subject: [PATCH 57/61] Remove unnecessary stuff --- .../scala/monix/kafka/PollHeartbeatTest.scala | 26 +++++++++-------- .../scala/monix/kafka/PollHeartbeatTest.scala | 28 ++++++++++--------- .../main/resources/monix/kafka/default.conf | 2 -- .../monix/kafka/KafkaConsumerObservable.scala | 5 ++-- .../monix/kafka/KafkaProducerConfig.scala | 2 +- 5 files changed, 32 insertions(+), 31 deletions(-) diff --git a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 6fa5bc85..2d4ab8fb 100644 --- a/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.10.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -197,12 +197,13 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture - // todo - we check that value never returns, - // which is correct in this scenario since if the `maxPollInterval` - // is higher than `pollHeartBeat` and `downstreamLatency` - // on the other hand, it would be ideal to receive the following error message from kafka - // "the group has already rebalanced and assigned the partitions to another member" - // as it happens from kafka-client 1.1.0, see tests from kafka1x. + /* checks that value never returns, + * which is correct in this scenario since if the `maxPollInterval` + * is higher than `pollHeartBeat` and `downstreamLatency` + * on the other hand, it would be ideal to receive the following error message from kafka + * "the group has already rebalanced and assigned the partitions to another member" + * as it happens from kafka-client 1.1.0, see tests from kafka1x. + */ assert(f.value === None) } } @@ -246,12 +247,13 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture - // todo - we check that value never returns, - // which is correct in this scenario since if the `maxPollInterval` - // is higher than `pollHeartBeat` and `downstreamLatency` - // on the other hand, it would be ideal to receive the following error message from kafka - // "the group has already rebalanced and assigned the partitions to another member" - // as it happens from kafka-client 1.1.0, see tests from kafka1x. + /* checks that value never returns, + * which is correct in this scenario since if the `maxPollInterval` + * is higher than `pollHeartBeat` and `downstreamLatency` + * on the other hand, it would be ideal to receive the following error message from kafka + * "the group has already rebalanced and assigned the partitions to another member" + * as it happens from kafka-client 1.1.0, see tests from kafka1x. + */ assert(f.value === None) } } diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala index 3ba70e4f..7db73855 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/PollHeartbeatTest.scala @@ -197,12 +197,13 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture - // todo - we check that value never returns, - // which is correct in this scenario since if the `maxPollInterval` - // is higher than `pollHeartBeat` and `downstreamLatency` - // on the other hand, it would be ideal to receive the following error message from kafka - // "the group has already rebalanced and assigned the partitions to another member" - // as it happens from kafka-client 1.1.0, see tests from kafka1x. + /* checks that value never returns, + * which is correct in this scenario since if the `maxPollInterval` + * is higher than `pollHeartBeat` and `downstreamLatency` + * on the other hand, it would be ideal to receive the following error message from kafka + * "the group has already rebalanced and assigned the partitions to another member" + * as it happens from kafka-client 1.1.0, see tests from kafka1x. + */ assert(f.value === None) } } @@ -218,7 +219,7 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { // but smaller than `pollHeartBeat`, kafka will trigger rebalance // and the consumer will be kicked out of the consumer group. val fastPollHeartbeatConfig = - consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) + consumerCfg.copy(maxPollInterval = maxPollInterval, maxPollRecords = 1).withPollHeartBeatRate(pollHeartbeat) val producer = KafkaProducer[String, String](producerCfg, io) val consumer = KafkaConsumerObservable.manualCommit[String, String](fastPollHeartbeatConfig, List(topicName)) @@ -246,12 +247,13 @@ class PollHeartbeatTest extends FunSuite with KafkaTestKit with ScalaFutures { val f = Task.parZip2(listT.executeAsync, pushT.executeAsync).map(_._1).delayResult(50.seconds).runToFuture - // todo - we check that value never returns, - // which is correct in this scenario since if the `maxPollInterval` - // is higher than `pollHeartBeat` and `downstreamLatency` - // on the other hand, it would be ideal to receive the following error message from kafka - // "the group has already rebalanced and assigned the partitions to another member" - // as it happens from kafka-client 1.1.0, see tests from kafka1x. + /* checks that value never returns, + * which is correct in this scenario since if the `maxPollInterval` + * is higher than `pollHeartBeat` and `downstreamLatency` + * on the other hand, it would be ideal to receive the following error message from kafka + * "the group has already rebalanced and assigned the partitions to another member" + * as it happens from kafka-client 1.1.0, see tests from kafka1x. + */ assert(f.value === None) } } diff --git a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf index 27a6f7bb..b995b27d 100644 --- a/kafka-0.9.x/src/main/resources/monix/kafka/default.conf +++ b/kafka-0.9.x/src/main/resources/monix/kafka/default.conf @@ -72,6 +72,4 @@ kafka { monix.observable.commit.type = "sync" # Possible values: before-ack, after-ack or no-ack monix.observable.commit.order = "after-ack" - # Internal kafka heartbeat that avoids partition reasignment when the upstream subscribers is slow. - monix.observable.poll.heartbeat.rate.ms = 100 } \ No newline at end of file diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 779b27d6..7a880be3 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -16,7 +16,6 @@ package monix.kafka -import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.execution.Ack.{Continue, Stop} import monix.execution.{Ack, Callback, Cancelable, Scheduler} @@ -37,9 +36,9 @@ import scala.concurrent.duration._ * [[KafkaConsumerConfig]] needed and see `monix/kafka/default.conf`, * (in the resource files) that is exposing all default values. */ -trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] with StrictLogging { - protected def config: KafkaConsumerConfig +trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { + protected def config: KafkaConsumerConfig protected def consumerT: Task[Consumer[K, V]] @volatile diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala index de704730..20f87485 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaProducerConfig.scala @@ -194,7 +194,7 @@ import scala.concurrent.duration._ * by this object can be set via the map, but in case of a duplicate * a value set on the case class will overwrite value set via properties. */ -case class KafkaProducerConfig( +final case class KafkaProducerConfig( bootstrapServers: List[String], acks: Acks, bufferMemoryInBytes: Int, From 12e75a8f5e2c317b0692112448bd53a033275e08 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 3 Apr 2021 16:15:43 +0200 Subject: [PATCH 58/61] Remove unused import --- .../scala/monix/kafka/benchmarks/ConsumerBenchmark.scala | 1 - .../test/scala/monix/kafka/MonixKafkaTopicListTest.scala | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index 2907dc46..c38155d0 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -5,7 +5,6 @@ import monix.kafka.config.ObservableCommitType import java.util.concurrent.TimeUnit import monix.kafka.{KafkaConsumerObservable, KafkaProducerSink} -//import monix.kafka.config.ObservableCommitType import monix.reactive.Observable import org.apache.kafka.clients.producer.ProducerRecord import org.openjdk.jmh.annotations.{BenchmarkMode, Fork, Measurement, Mode, OutputTimeUnit, Scope, State, Threads, Warmup, _} diff --git a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala index 64a6b4c4..8d44c672 100644 --- a/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala +++ b/kafka-0.11.x/src/test/scala/monix/kafka/MonixKafkaTopicListTest.scala @@ -106,9 +106,9 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { } } - test("manual commit consumer test when subscribed to topics list") { + test("manual commitSync consumer test when subscribed to topics list") { withRunningKafka { - val count = 1000 + val count = 10000 val topicName = "monix-kafka-manual-commit-tests" val producer = KafkaProducerSink[String, String](producerCfg, io) @@ -117,7 +117,7 @@ class MonixKafkaTopicListTest extends FunSuite with KafkaTestKit { val pushT = Observable .range(0, count) .map(msg => new ProducerRecord(topicName, "obs", msg.toString)) - .bufferTumbling(10) + .bufferIntrospective(1024) .consumeWith(producer) val listT = consumer From 10cbf9f09c9f5b95545b1f48f447f894d8dcdf17 Mon Sep 17 00:00:00 2001 From: Pau Date: Fri, 9 Apr 2021 20:35:47 +0200 Subject: [PATCH 59/61] Benchmarks readme --- benchmarks/results/consumer.md | 9 +++++++-- benchmarks/results/producer.md | 3 +++ benchmarks/results/readme.md | 25 ------------------------- 3 files changed, 10 insertions(+), 27 deletions(-) delete mode 100644 benchmarks/results/readme.md diff --git a/benchmarks/results/consumer.md b/benchmarks/results/consumer.md index 284548b1..a4c84e47 100644 --- a/benchmarks/results/consumer.md +++ b/benchmarks/results/consumer.md @@ -1,6 +1,12 @@ -## RC7 +## Consumer Benchmarks + +The consumer benchmark covers the *manual* and *auto commit* consumer implementations of the different libraries. +The manual commit will also cover producing committing back the consumed offsets. +It also runs on different range `pollHeartbeatRate` [1, 10, 15, 100, 1000], which is an important configuration + implemented in this `monix-kafka` library. +## RC7 ### 1fork 1thread Benchmark Mode Cnt Score Error Units ConsumerBenchmark.monix_auto_commit_async thrpt 7 13.097 ± 0.827 ops/s @@ -12,7 +18,6 @@ Benchmark Mode Cnt Score Error Units ConsumerBenchmark.monix_auto_commit thrpt 10 16.186 ± 0.920 ops/s ConsumerBenchmark.monix_manual_commit thrpt 10 16.319 ± 1.465 ops/s - ## RC8 - (Introduces PollHeartbeatRate) ### 1fork 1thread --- diff --git a/benchmarks/results/producer.md b/benchmarks/results/producer.md index 04e68cf0..bf9602b1 100644 --- a/benchmarks/results/producer.md +++ b/benchmarks/results/producer.md @@ -1,3 +1,6 @@ +## Producer benchmarks + +This section includes benchmarks for single and sink producers. ## RC7 diff --git a/benchmarks/results/readme.md b/benchmarks/results/readme.md deleted file mode 100644 index 6f2b612d..00000000 --- a/benchmarks/results/readme.md +++ /dev/null @@ -1,25 +0,0 @@ -###Monix Kafka Benchmarks - -This document explains the approach followed to benchmark monix-kafka. - - Ideally, a Kafka performance benchmark should happen under some long stress test in a real Kafka cluster, -although, our hardware limitations we have to stick to running simpler basic tests that proves application performance on - a docker container. - -The benchmark will focus on the most basic `consumer` and `procucer` scenarios. - -Although Kafka is very configurable by nature, our benchmark will use the kafka default properties for [consumer](https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html) -and [producer](https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html). - -It also assumes that all the used topics will have 2 partitions, and 1 replication factor. - -## Producer benchmarks - -This section includes benchmarks for single and sink producers. -Although some libraries like `alpakka-kafka` do not expose methods for producing single record, but only for sink. - - -## Consumer benchmark -The consumer benchmark covers the manual and auto commit consumer implementations of the different libraries. -The manual commit will also cover producing committing back the consumed offsets. -In the `monix` performance test scenarios, compares with a different range `pollHeartbeatRate` [10, 100, 1000, 3000]. \ No newline at end of file From 32c5bd7a307f4b923b3fdf39831da31a46c01127 Mon Sep 17 00:00:00 2001 From: Pau Date: Sat, 10 Apr 2021 23:08:29 +0200 Subject: [PATCH 60/61] Small correction --- README.md | 4 ++-- .../scala/monix/kafka/benchmarks/BaseFixture.scala | 10 +++------- .../monix/kafka/benchmarks/ConsumerBenchmark.scala | 4 ++-- 3 files changed, 7 insertions(+), 11 deletions(-) diff --git a/README.md b/README.md index 4a2f50e3..5026496d 100644 --- a/README.md +++ b/README.md @@ -266,8 +266,8 @@ This was an [issue](https://github.com/monix/monix-kafka/issues/101) in `monix-k since poll is not called until all previous consumed ones were processed, so that slow downstream subscribers were in risk of being kicked off the consumer group indefinitely. -This has been resolved in `1.0.0-RC8` by introducing an internal poll heartbeat interval -that runs in the background keeping the consumer alive. +It is resolved in `1.0.0-RC8` by introducing an internal poll heartbeat interval +that runs in the background keeping the consumer alive. ## How can I contribute to Monix-Kafka? diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala index 932eb48b..5ac3f2d2 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/BaseFixture.scala @@ -5,11 +5,7 @@ import monix.eval.Coeval import scala.util.Random trait BaseFixture { - - val brokerUrl = "127.0.0.1:9092" - val randomId: Coeval[String] = Coeval(Random.alphanumeric.filter(_.isLetter).take(20).mkString) - - // topic names - val monixTopic = "monix_topic" - + protected val brokerUrl = "127.0.0.1:9092" + protected val randomId: Coeval[String] = Coeval(Random.alphanumeric.filter(_.isLetter).take(20).mkString) + protected val monixTopic = "monix_topic" } diff --git a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala index c38155d0..ad8dfee3 100644 --- a/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala +++ b/benchmarks/src/main/scala/monix/kafka/benchmarks/ConsumerBenchmark.scala @@ -19,13 +19,13 @@ import scala.concurrent.duration._ @Threads(3) class ConsumerBenchmark extends MonixFixture { - var totalRecords: Int = 1500 + var totalRecords: Int = 1100 val consumedRecords = 1000 var maxPollRecords: Int = 1 - // preparing test data @Setup def setup(): Unit = { + // preparing test data Observable .from(0 to totalRecords) .map(i => new ProducerRecord[Integer, Integer](monixTopic, i)) From 15267029e210142dbde9a10a98376a6a7965b58b Mon Sep 17 00:00:00 2001 From: Pau Date: Sun, 11 Apr 2021 19:27:38 +0200 Subject: [PATCH 61/61] MR Feedback --- CHANGES.md | 11 +---------- benchmarks/results/producer.md | 4 ++++ .../scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- .../scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- .../scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- .../scala/monix/kafka/KafkaConsumerObservable.scala | 2 +- 6 files changed, 9 insertions(+), 14 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 9d467e39..c9b58030 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,5 @@ -## Version 1.0.0-RC6 (January 23, 2021) +## Version 1.0.0-RC7 (January 23, 2021) Depends on Monix 3.3.0 @@ -8,15 +8,6 @@ Main changes: - PR #248: Enable '-Xfatal-warnings' flag (internal) - PR #245: Use Kafka Producer Consumer interfaces -## Version 1.0.0-RC7 (April 18, 2020) - -Depends on Monix 3.3.0 - -Main changes: - -- PR #248: Enable '-Xfatal-warnings flag -- PR #245: Use Kafka Producer Consumer interfaces - ## Version 1.0.0-RC6 (April 18, 2020) Depends on Monix 3.x diff --git a/benchmarks/results/producer.md b/benchmarks/results/producer.md index bf9602b1..38aec700 100644 --- a/benchmarks/results/producer.md +++ b/benchmarks/results/producer.md @@ -11,6 +11,10 @@ ProducerBenchmark.monix_sink_producer thrpt 3 1.861 ± 5.475 ops/s ## RC8 +### 10iterations 1fork 1thread +ProducerBenchmark.monix_single_producer thrpt 10 0.473 ± 0.070 ops/s +ProducerBenchmark.monix_sink_producer thrpt 10 2.009 ± 0.277 ops/s + ### 10iterations 1fork 3threads Benchmark Mode Cnt Score Error Units ProducerBenchmark.monix_single_producer thrpt 10 0.981 ± 0.202 ops/s diff --git a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 6926de87..dea323ef 100644 --- a/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.10.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -73,7 +73,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void } { consumer => // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))) } startConsuming.runAsync(cb) } diff --git a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 772edc94..2e7db3f5 100644 --- a/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.11.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -75,7 +75,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void } { consumer => // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))) } startConsuming.runAsync(cb) } diff --git a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 8f1d3e1e..d131e181 100644 --- a/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-0.9.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -68,7 +68,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Resource .make(consumer) { c => // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(c.close()))).memoizeOnSuccess + Task.evalAsync(consumer.synchronized(blocking(c.close()))) } .use { c => // Skipping all available messages on all partitions diff --git a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala index 7a880be3..2b38df73 100644 --- a/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala +++ b/kafka-1.0.x/src/main/scala/monix/kafka/KafkaConsumerObservable.scala @@ -74,7 +74,7 @@ trait KafkaConsumerObservable[K, V, Out] extends Observable[Out] { Task.race(runLoop(c, out), pollHeartbeat(c).loopForever).void } { consumer => // Forced asynchronous boundary - Task.evalAsync(consumer.synchronized(blocking(consumer.close()))).memoizeOnSuccess + Task.evalAsync(consumer.synchronized(blocking(consumer.close()))) } startConsuming.runAsync(cb) }