From d8d9665d2cdabf2ffeaff0c9a3d69b28810e7bff Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 8 Sep 2019 22:46:02 +0200 Subject: [PATCH 01/31] WIP: Add wrapper for Metadata requests #497 --- .../akka/kafka/javadsl/MetadataClient.scala | 47 +++++++++++ .../akka/kafka/scaladsl/MetadataClient.scala | 49 ++++++++++++ .../main/scala/akka/kafka/KafkaPorts.scala | 1 + .../java/docs/javadsl/MetadataClientTest.java | 80 +++++++++++++++++++ .../kafka/scaladsl/MetadataClientSpec.scala | 45 +++++++++++ 5 files changed, 222 insertions(+) create mode 100644 core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala create mode 100644 core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala create mode 100644 tests/src/test/java/docs/javadsl/MetadataClientTest.java create mode 100644 tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala new file mode 100644 index 000000000..bc87d00d5 --- /dev/null +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 - 2016 Softwaremill + * Copyright (C) 2016 - 2019 Lightbend Inc. + */ + +package akka.kafka.javadsl + +import java.util.concurrent.{CompletionStage, Executor} + +import akka.actor.ActorSystem +import akka.kafka.ConsumerSettings +import akka.util.Timeout +import org.apache.kafka.common.TopicPartition + +import scala.concurrent.ExecutionContext +import scala.compat.java8.FutureConverters._ +import scala.collection.JavaConverters._ + +object MetadataClient { + + def getBeginningOffsets[K, V]( + consumerSettings: ConsumerSettings[K, V], + partitions: java.util.Set[TopicPartition], + timeout: Timeout, + system: ActorSystem, + executor: Executor + ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { + val ec = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getBeginningOffsets(consumerSettings, partitions.asScala.toSet, timeout)(system, ec) + .map { beginningOffsets => + val scalaMapWithJavaValues = beginningOffsets.mapValues(long2Long) + scalaMapWithJavaValues.asJava + }(ec) + .toJava + } + + def getBeginningOffsetForPartition[K, V]( + consumerSettings: ConsumerSettings[K, V], + partition: TopicPartition, + timeout: Timeout, + system: ActorSystem, + executor: Executor + ): CompletionStage[java.lang.Long] = + getBeginningOffsets(consumerSettings, Set(partition).asJava, timeout, system, executor) + .thenApply(_.get(partition)) +} diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala new file mode 100644 index 000000000..dadccd895 --- /dev/null +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2014 - 2016 Softwaremill + * Copyright (C) 2016 - 2019 Lightbend Inc. + */ + +package akka.kafka.scaladsl +import akka.actor.{ActorRef, ActorSystem} +import akka.kafka.{ConsumerSettings, KafkaConsumerActor} +import akka.kafka.Metadata.{BeginningOffsets, GetBeginningOffsets} +import akka.pattern.ask +import akka.util.Timeout +import org.apache.kafka.common.TopicPartition + +import scala.concurrent.{ExecutionContext, Future} + +object MetadataClient { + + private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() + + def getBeginningOffsets[K, V]( + consumerSettings: ConsumerSettings[K, V], + partitions: Set[TopicPartition], + timeout: Timeout + )(implicit system: ActorSystem, ec: ExecutionContext): Future[Map[TopicPartition, Long]] = { + val consumerActor = getConsumerActor(consumerSettings) + (consumerActor ? GetBeginningOffsets(partitions))(timeout) + .mapTo[BeginningOffsets] + .map(_.response.get) + } + + def getBeginningOffsetForPartition[K, V]( + consumerSettings: ConsumerSettings[K, V], + partition: TopicPartition, + timeout: Timeout + )(implicit system: ActorSystem, ec: ExecutionContext): Future[Long] = + getBeginningOffsets(consumerSettings, Set(partition), timeout) + .map(beginningOffsets => beginningOffsets(partition)) + + private def getConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V])(implicit system: ActorSystem) = { + val consumerActor = consumerActors.get(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]]) + if (consumerActor.isEmpty) { + val newConsumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + consumerActors.put(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]], newConsumerActor) + newConsumerActor + } else { + consumerActor.get + } + } +} diff --git a/tests/src/main/scala/akka/kafka/KafkaPorts.scala b/tests/src/main/scala/akka/kafka/KafkaPorts.scala index d9bb08091..2cdcf5a0d 100644 --- a/tests/src/main/scala/akka/kafka/KafkaPorts.scala +++ b/tests/src/main/scala/akka/kafka/KafkaPorts.scala @@ -25,4 +25,5 @@ object KafkaPorts { val ProducerExamplesTest = 9112 val KafkaConnectionCheckerTest = 9122 val PartitionAssignmentHandlerSpec = 9132 + val MetadataClientTest = 9142 } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java new file mode 100644 index 000000000..2548d4b3a --- /dev/null +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2014 - 2016 Softwaremill + * Copyright (C) 2016 - 2019 Lightbend Inc. + */ + +package docs.javadsl; + +import akka.actor.ActorSystem; +import akka.kafka.ConsumerSettings; +import akka.kafka.KafkaPorts; +import akka.kafka.javadsl.MetadataClient; +import akka.kafka.testkit.javadsl.EmbeddedKafkaJunit4Test; +import akka.stream.ActorMaterializer; +import akka.stream.Materializer; +import akka.testkit.javadsl.TestKit; +import akka.util.Timeout; +import org.apache.kafka.common.TopicPartition; +import org.junit.AfterClass; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; + +public class MetadataClientTest extends EmbeddedKafkaJunit4Test { + + private static final ActorSystem sys = ActorSystem.create("MetadataClientTest"); + private static final Materializer mat = ActorMaterializer.create(sys); + private static final Executor ec = Executors.newSingleThreadExecutor(); + + public MetadataClientTest() { + super(sys, mat, KafkaPorts.MetadataClientTest()); + } + + @Test + public void shouldFetchBeginningOffsetsForGivenPartitionsTest() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition0 = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Set partitions = Collections.singleton(partition0); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + + final CompletionStage> response = + MetadataClient.getBeginningOffsets(consumerSettings, partitions, timeout, sys, ec); + final Map beginningOffsets = response.toCompletableFuture().join(); + + assertThat(beginningOffsets.get(partition0), is(0L)); + } + + @Test + public void shouldFetchBeginningOffsetForGivenPartitionTest() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition0 = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + + final CompletionStage response = + MetadataClient.getBeginningOffsetForPartition( + consumerSettings, partition0, timeout, sys, ec); + final Long beginningOffset = response.toCompletableFuture().join(); + + assertThat(beginningOffset, is(0L)); + } + + @AfterClass + public static void afterClass() { + TestKit.shutdownActorSystem(sys); + } +} diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala new file mode 100644 index 000000000..22163e18a --- /dev/null +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 - 2016 Softwaremill + * Copyright (C) 2016 - 2019 Lightbend Inc. + */ + +package akka.kafka.scaladsl + +import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike +import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped +import org.apache.kafka.common.TopicPartition + +import scala.concurrent.Await +import scala.language.postfixOps +import scala.concurrent.duration._ + +class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { + + "MetadataClient" must { + "fetch beginning offsets for given partitions" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + + val beginningOffsetsFuture = MetadataClient + .getBeginningOffsets(consumerSettings, Set(partition0), 1 seconds) + val beginningOffsets = Await.result(beginningOffsetsFuture, 1 seconds) + + beginningOffsets(partition0) shouldBe 0 + } + + "fetch beginning offset for given partition" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + + val beginningOffsetFuture = MetadataClient + .getBeginningOffsetForPartition(consumerSettings, partition0, 1 seconds) + val beginningOffset = Await.result(beginningOffsetFuture, 1 seconds) + + beginningOffset shouldBe 0 + } + } +} From 0509d603fccf4dee1f570856490cf98d37ee7c5a Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sat, 14 Sep 2019 14:42:51 +0200 Subject: [PATCH 02/31] Code review fixes. --- .../akka/kafka/javadsl/MetadataClient.scala | 17 ++++------ .../akka/kafka/scaladsl/MetadataClient.scala | 33 +++++-------------- .../java/docs/javadsl/MetadataClientTest.java | 9 +++-- .../kafka/scaladsl/MetadataClientSpec.scala | 7 ++-- 4 files changed, 27 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index bc87d00d5..0c9d2391a 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -7,8 +7,7 @@ package akka.kafka.javadsl import java.util.concurrent.{CompletionStage, Executor} -import akka.actor.ActorSystem -import akka.kafka.ConsumerSettings +import akka.actor.ActorRef import akka.util.Timeout import org.apache.kafka.common.TopicPartition @@ -18,16 +17,15 @@ import scala.collection.JavaConverters._ object MetadataClient { - def getBeginningOffsets[K, V]( - consumerSettings: ConsumerSettings[K, V], + def getBeginningOffsets( + consumerActor: ActorRef, partitions: java.util.Set[TopicPartition], timeout: Timeout, - system: ActorSystem, executor: Executor ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { val ec = ExecutionContext.fromExecutor(executor) akka.kafka.scaladsl.MetadataClient - .getBeginningOffsets(consumerSettings, partitions.asScala.toSet, timeout)(system, ec) + .getBeginningOffsets(consumerActor, partitions.asScala.toSet, timeout)(ec) .map { beginningOffsets => val scalaMapWithJavaValues = beginningOffsets.mapValues(long2Long) scalaMapWithJavaValues.asJava @@ -35,13 +33,12 @@ object MetadataClient { .toJava } - def getBeginningOffsetForPartition[K, V]( - consumerSettings: ConsumerSettings[K, V], + def getBeginningOffsetForPartition( + consumerActor: ActorRef, partition: TopicPartition, timeout: Timeout, - system: ActorSystem, executor: Executor ): CompletionStage[java.lang.Long] = - getBeginningOffsets(consumerSettings, Set(partition).asJava, timeout, system, executor) + getBeginningOffsets(consumerActor, Set(partition).asJava, timeout, executor) .thenApply(_.get(partition)) } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index dadccd895..5f649cfd0 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -4,8 +4,8 @@ */ package akka.kafka.scaladsl -import akka.actor.{ActorRef, ActorSystem} -import akka.kafka.{ConsumerSettings, KafkaConsumerActor} + +import akka.actor.ActorRef import akka.kafka.Metadata.{BeginningOffsets, GetBeginningOffsets} import akka.pattern.ask import akka.util.Timeout @@ -15,35 +15,20 @@ import scala.concurrent.{ExecutionContext, Future} object MetadataClient { - private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() - - def getBeginningOffsets[K, V]( - consumerSettings: ConsumerSettings[K, V], + def getBeginningOffsets( + consumerActor: ActorRef, partitions: Set[TopicPartition], timeout: Timeout - )(implicit system: ActorSystem, ec: ExecutionContext): Future[Map[TopicPartition, Long]] = { - val consumerActor = getConsumerActor(consumerSettings) + )(implicit ec: ExecutionContext): Future[Map[TopicPartition, Long]] = (consumerActor ? GetBeginningOffsets(partitions))(timeout) .mapTo[BeginningOffsets] .map(_.response.get) - } - def getBeginningOffsetForPartition[K, V]( - consumerSettings: ConsumerSettings[K, V], + def getBeginningOffsetForPartition( + consumerActor: ActorRef, partition: TopicPartition, timeout: Timeout - )(implicit system: ActorSystem, ec: ExecutionContext): Future[Long] = - getBeginningOffsets(consumerSettings, Set(partition), timeout) + )(implicit ec: ExecutionContext): Future[Long] = + getBeginningOffsets(consumerActor, Set(partition), timeout) .map(beginningOffsets => beginningOffsets(partition)) - - private def getConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V])(implicit system: ActorSystem) = { - val consumerActor = consumerActors.get(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]]) - if (consumerActor.isEmpty) { - val newConsumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - consumerActors.put(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]], newConsumerActor) - newConsumerActor - } else { - consumerActor.get - } - } } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 2548d4b3a..4b106f7d0 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -5,8 +5,10 @@ package docs.javadsl; +import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.kafka.ConsumerSettings; +import akka.kafka.KafkaConsumerActor; import akka.kafka.KafkaPorts; import akka.kafka.javadsl.MetadataClient; import akka.kafka.testkit.javadsl.EmbeddedKafkaJunit4Test; @@ -48,9 +50,10 @@ public void shouldFetchBeginningOffsetsForGivenPartitionsTest() { consumerDefaults().withGroupId(group1); final Set partitions = Collections.singleton(partition0); final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); final CompletionStage> response = - MetadataClient.getBeginningOffsets(consumerSettings, partitions, timeout, sys, ec); + MetadataClient.getBeginningOffsets(consumerActor, partitions, timeout, ec); final Map beginningOffsets = response.toCompletableFuture().join(); assertThat(beginningOffsets.get(partition0), is(0L)); @@ -64,10 +67,10 @@ public void shouldFetchBeginningOffsetForGivenPartitionTest() { final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); final CompletionStage response = - MetadataClient.getBeginningOffsetForPartition( - consumerSettings, partition0, timeout, sys, ec); + MetadataClient.getBeginningOffsetForPartition(consumerActor, partition0, timeout, ec); final Long beginningOffset = response.toCompletableFuture().join(); assertThat(beginningOffset, is(0L)); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 22163e18a..f75f5673f 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -5,6 +5,7 @@ package akka.kafka.scaladsl +import akka.kafka.KafkaConsumerActor import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import org.apache.kafka.common.TopicPartition @@ -21,9 +22,10 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val group1 = createGroupId(1) val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) val beginningOffsetsFuture = MetadataClient - .getBeginningOffsets(consumerSettings, Set(partition0), 1 seconds) + .getBeginningOffsets(consumerActor, Set(partition0), 1 seconds) val beginningOffsets = Await.result(beginningOffsetsFuture, 1 seconds) beginningOffsets(partition0) shouldBe 0 @@ -34,9 +36,10 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val group1 = createGroupId(1) val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) val beginningOffsetFuture = MetadataClient - .getBeginningOffsetForPartition(consumerSettings, partition0, 1 seconds) + .getBeginningOffsetForPartition(consumerActor, partition0, 1 seconds) val beginningOffset = Await.result(beginningOffsetFuture, 1 seconds) beginningOffset shouldBe 0 From abeba926a8ce2fb0717087470bace2204faaf074 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 16 Sep 2019 22:57:27 +0200 Subject: [PATCH 03/31] Fix CI compilation error. --- core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 0c9d2391a..152751866 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -40,5 +40,5 @@ object MetadataClient { executor: Executor ): CompletionStage[java.lang.Long] = getBeginningOffsets(consumerActor, Set(partition).asJava, timeout, executor) - .thenApply(_.get(partition)) + .thenApply(beginningOffsets => beginningOffsets.get(partition)) } From fc162707214d54fb2701ae5f915ea99e60075222 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 16 Sep 2019 23:03:46 +0200 Subject: [PATCH 04/31] Fix CI compilation error. --- core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 152751866..f4942a27e 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -40,5 +40,5 @@ object MetadataClient { executor: Executor ): CompletionStage[java.lang.Long] = getBeginningOffsets(consumerActor, Set(partition).asJava, timeout, executor) - .thenApply(beginningOffsets => beginningOffsets.get(partition)) + .thenApply((beginningOffsets: java.util.Map[TopicPartition, java.lang.Long]) => beginningOffsets.get(partition)) } From 0e6d4a9d607244a8a808ed782150aa5200cde9d7 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Tue, 17 Sep 2019 09:33:59 +0200 Subject: [PATCH 05/31] Fix CI compilation error. --- .../main/scala/akka/kafka/javadsl/MetadataClient.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index f4942a27e..033759848 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -38,7 +38,11 @@ object MetadataClient { partition: TopicPartition, timeout: Timeout, executor: Executor - ): CompletionStage[java.lang.Long] = - getBeginningOffsets(consumerActor, Set(partition).asJava, timeout, executor) - .thenApply((beginningOffsets: java.util.Map[TopicPartition, java.lang.Long]) => beginningOffsets.get(partition)) + ): CompletionStage[java.lang.Long] = { + val ec = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getBeginningOffsetForPartition(consumerActor, partition, timeout)(ec) + .map(long2Long)(ec) + .toJava + } } From 887c267a726eaa3418fefb19bb9dd1ff88e26ec4 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Tue, 17 Sep 2019 10:13:03 +0200 Subject: [PATCH 06/31] Fix CI compilation error. --- core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 033759848..3d069dd3d 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -31,6 +31,7 @@ object MetadataClient { scalaMapWithJavaValues.asJava }(ec) .toJava + .asInstanceOf[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] } def getBeginningOffsetForPartition( From 47a0c471d140983cd81df9494c48ce18b29674fe Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Wed, 18 Sep 2019 21:51:36 +0200 Subject: [PATCH 07/31] MetadataClientTest extends TestcontainersKafkaJunit4Test instead of EmbeddedKafkaJunit4Test. --- tests/src/main/scala/akka/kafka/KafkaPorts.scala | 1 - tests/src/test/java/docs/javadsl/MetadataClientTest.java | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/src/main/scala/akka/kafka/KafkaPorts.scala b/tests/src/main/scala/akka/kafka/KafkaPorts.scala index 2cdcf5a0d..d9bb08091 100644 --- a/tests/src/main/scala/akka/kafka/KafkaPorts.scala +++ b/tests/src/main/scala/akka/kafka/KafkaPorts.scala @@ -25,5 +25,4 @@ object KafkaPorts { val ProducerExamplesTest = 9112 val KafkaConnectionCheckerTest = 9122 val PartitionAssignmentHandlerSpec = 9132 - val MetadataClientTest = 9142 } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 4b106f7d0..9336b0123 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -9,9 +9,8 @@ import akka.actor.ActorSystem; import akka.kafka.ConsumerSettings; import akka.kafka.KafkaConsumerActor; -import akka.kafka.KafkaPorts; import akka.kafka.javadsl.MetadataClient; -import akka.kafka.testkit.javadsl.EmbeddedKafkaJunit4Test; +import akka.kafka.testkit.javadsl.TestcontainersKafkaJunit4Test; import akka.stream.ActorMaterializer; import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; @@ -31,14 +30,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; -public class MetadataClientTest extends EmbeddedKafkaJunit4Test { +public class MetadataClientTest extends TestcontainersKafkaJunit4Test { private static final ActorSystem sys = ActorSystem.create("MetadataClientTest"); private static final Materializer mat = ActorMaterializer.create(sys); private static final Executor ec = Executors.newSingleThreadExecutor(); public MetadataClientTest() { - super(sys, mat, KafkaPorts.MetadataClientTest()); + super(sys, mat); } @Test From 9ad60c0e6eff8078c8e8b21d2ae1aad02a92fa68 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 23 Sep 2019 23:26:15 +0200 Subject: [PATCH 08/31] Add stopping consumer actor in tests. --- tests/src/test/java/docs/javadsl/MetadataClientTest.java | 4 ++++ .../test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 9336b0123..669ad50fe 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -56,6 +56,8 @@ public void shouldFetchBeginningOffsetsForGivenPartitionsTest() { final Map beginningOffsets = response.toCompletableFuture().join(); assertThat(beginningOffsets.get(partition0), is(0L)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } @Test @@ -73,6 +75,8 @@ public void shouldFetchBeginningOffsetForGivenPartitionTest() { final Long beginningOffset = response.toCompletableFuture().join(); assertThat(beginningOffset, is(0L)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } @AfterClass diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index f75f5673f..1ba1586db 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -29,6 +29,8 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val beginningOffsets = Await.result(beginningOffsetsFuture, 1 seconds) beginningOffsets(partition0) shouldBe 0 + + consumerActor ! KafkaConsumerActor.Stop } "fetch beginning offset for given partition" in assertAllStagesStopped { @@ -43,6 +45,8 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val beginningOffset = Await.result(beginningOffsetFuture, 1 seconds) beginningOffset shouldBe 0 + + consumerActor ! KafkaConsumerActor.Stop } } } From 64858bbaeb7b5b4fe2dbdd5b1300eccaec47325d Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Wed, 25 Sep 2019 15:11:42 +0200 Subject: [PATCH 09/31] Add failing cases to MetadataClientTest and MetadataClientSpec. --- .../java/docs/javadsl/MetadataClientTest.java | 53 ++++++++++++++++++- .../kafka/scaladsl/MetadataClientSpec.scala | 37 +++++++++++-- 2 files changed, 83 insertions(+), 7 deletions(-) diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 669ad50fe..d33567e86 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -16,12 +16,16 @@ import akka.testkit.javadsl.TestKit; import akka.util.Timeout; import org.apache.kafka.common.TopicPartition; +import org.hamcrest.core.IsInstanceOf; import org.junit.AfterClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.util.Collections; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; import java.util.concurrent.Executor; import java.util.concurrent.Executors; @@ -36,12 +40,14 @@ public class MetadataClientTest extends TestcontainersKafkaJunit4Test { private static final Materializer mat = ActorMaterializer.create(sys); private static final Executor ec = Executors.newSingleThreadExecutor(); + @Rule public ExpectedException expectedException = ExpectedException.none(); + public MetadataClientTest() { super(sys, mat); } @Test - public void shouldFetchBeginningOffsetsForGivenPartitionsTest() { + public void shouldFetchBeginningOffsetsForGivenPartitions() { final String topic1 = createTopic(); final String group1 = createGroupId(); final TopicPartition partition0 = new TopicPartition(topic1, 0); @@ -61,7 +67,29 @@ public void shouldFetchBeginningOffsetsForGivenPartitionsTest() { } @Test - public void shouldFetchBeginningOffsetForGivenPartitionTest() { + public void shouldFailInCaseOfAnExceptionDuringFetchOffsetsForNonExistingTopics() { + expectedException.expect(CompletionException.class); + expectedException.expectCause( + IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); + + final String group1 = createGroupId(); + final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Set partitions = Collections.singleton(nonExistingPartition); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + final CompletionStage> response = + MetadataClient.getBeginningOffsets(consumerActor, partitions, timeout, ec); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + + response.toCompletableFuture().join(); + } + + @Test + public void shouldFetchBeginningOffsetForGivenPartition() { final String topic1 = createTopic(); final String group1 = createGroupId(); final TopicPartition partition0 = new TopicPartition(topic1, 0); @@ -79,6 +107,27 @@ public void shouldFetchBeginningOffsetForGivenPartitionTest() { consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } + @Test + public void shouldFailInCaseOfAnExceptionDuringFetchOffsetForNonExistingTopic() { + expectedException.expect(CompletionException.class); + expectedException.expectCause( + IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); + + final String group1 = createGroupId(); + final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + final CompletionStage response = + MetadataClient.getBeginningOffsetForPartition(consumerActor, nonExistingPartition, timeout, ec); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + + response.toCompletableFuture().join(); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 1ba1586db..0ae0ac70e 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -10,7 +10,6 @@ import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import org.apache.kafka.common.TopicPartition -import scala.concurrent.Await import scala.language.postfixOps import scala.concurrent.duration._ @@ -24,15 +23,29 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val consumerSettings = consumerDefaults.withGroupId(group1) val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - val beginningOffsetsFuture = MetadataClient + val beginningOffsets = MetadataClient .getBeginningOffsets(consumerActor, Set(partition0), 1 seconds) - val beginningOffsets = Await.result(beginningOffsetsFuture, 1 seconds) + .futureValue beginningOffsets(partition0) shouldBe 0 consumerActor ! KafkaConsumerActor.Stop } + "fail in case of an exception during fetch offsets for non-existing topics" in assertAllStagesStopped { + val group1 = createGroupId(1) + val nonExistingPartition = new TopicPartition("non-existing topic", 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + val beginningOffsetsFuture = MetadataClient + .getBeginningOffsets(consumerActor, Set(nonExistingPartition), 1 seconds) + + beginningOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] + + consumerActor ! KafkaConsumerActor.Stop + } + "fetch beginning offset for given partition" in assertAllStagesStopped { val topic1 = createTopic(1) val group1 = createGroupId(1) @@ -40,13 +53,27 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val consumerSettings = consumerDefaults.withGroupId(group1) val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - val beginningOffsetFuture = MetadataClient + val beginningOffset = MetadataClient .getBeginningOffsetForPartition(consumerActor, partition0, 1 seconds) - val beginningOffset = Await.result(beginningOffsetFuture, 1 seconds) + .futureValue beginningOffset shouldBe 0 consumerActor ! KafkaConsumerActor.Stop } + + "fail in case of an exception during fetch offset for non-existing topic" in assertAllStagesStopped { + val group1 = createGroupId(1) + val nonExistingPartition = new TopicPartition("non-existing topic", 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + val beginningOffsetFuture = MetadataClient + .getBeginningOffsetForPartition(consumerActor, nonExistingPartition, 1 seconds) + + beginningOffsetFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] + + consumerActor ! KafkaConsumerActor.Stop + } } } From 5d9c31cfc3b7612034463a52a8991b4a4afe81db Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 11:45:43 +0200 Subject: [PATCH 10/31] Fix compilation errors. --- .../akka/kafka/javadsl/MetadataClient.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 3d069dd3d..93d0418ed 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -11,8 +11,9 @@ import akka.actor.ActorRef import akka.util.Timeout import org.apache.kafka.common.TopicPartition -import scala.concurrent.ExecutionContext +import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.compat.java8.FutureConverters._ +import scala.collection.compat._ import scala.collection.JavaConverters._ object MetadataClient { @@ -23,15 +24,13 @@ object MetadataClient { timeout: Timeout, executor: Executor ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { - val ec = ExecutionContext.fromExecutor(executor) + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) akka.kafka.scaladsl.MetadataClient - .getBeginningOffsets(consumerActor, partitions.asScala.toSet, timeout)(ec) + .getBeginningOffsets(consumerActor, partitions.asScala.toSet, timeout) .map { beginningOffsets => - val scalaMapWithJavaValues = beginningOffsets.mapValues(long2Long) - scalaMapWithJavaValues.asJava - }(ec) + beginningOffsets.view.mapValues(Long.box).toMap.asJava + } .toJava - .asInstanceOf[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] } def getBeginningOffsetForPartition( @@ -40,10 +39,10 @@ object MetadataClient { timeout: Timeout, executor: Executor ): CompletionStage[java.lang.Long] = { - val ec = ExecutionContext.fromExecutor(executor) + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) akka.kafka.scaladsl.MetadataClient - .getBeginningOffsetForPartition(consumerActor, partition, timeout)(ec) - .map(long2Long)(ec) + .getBeginningOffsetForPartition(consumerActor, partition, timeout) + .map(Long.box) .toJava } } From 9fdac7b50407fd6d1201a04bad084ac7a570e025 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 15:03:59 +0200 Subject: [PATCH 11/31] Add MetadataClient.getEndOffsets and MetadataClient.getEndOffsetForPartition --- .../akka/kafka/javadsl/MetadataClient.scala | 28 ++++++ .../akka/kafka/scaladsl/MetadataClient.scala | 19 +++- .../java/docs/javadsl/MetadataClientTest.java | 90 ++++++++++++++++++- .../kafka/scaladsl/MetadataClientSpec.scala | 72 ++++++++++++++- 4 files changed, 204 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 93d0418ed..ab96183b1 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -45,4 +45,32 @@ object MetadataClient { .map(Long.box) .toJava } + + def getEndOffsets( + consumerActor: ActorRef, + partitions: java.util.Set[TopicPartition], + timeout: Timeout, + executor: Executor + ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getEndOffsets(consumerActor, partitions.asScala.toSet, timeout) + .map { endOffsets => + endOffsets.view.mapValues(Long.box).toMap.asJava + } + .toJava + } + + def getEndOffsetForPartition( + consumerActor: ActorRef, + partition: TopicPartition, + timeout: Timeout, + executor: Executor + ): CompletionStage[java.lang.Long] = { + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getEndOffsetForPartition(consumerActor, partition, timeout) + .map(Long.box) + .toJava + } } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 5f649cfd0..0391da74e 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -6,7 +6,7 @@ package akka.kafka.scaladsl import akka.actor.ActorRef -import akka.kafka.Metadata.{BeginningOffsets, GetBeginningOffsets} +import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets} import akka.pattern.ask import akka.util.Timeout import org.apache.kafka.common.TopicPartition @@ -31,4 +31,21 @@ object MetadataClient { )(implicit ec: ExecutionContext): Future[Long] = getBeginningOffsets(consumerActor, Set(partition), timeout) .map(beginningOffsets => beginningOffsets(partition)) + + def getEndOffsets( + consumerActor: ActorRef, + partitions: Set[TopicPartition], + timeout: Timeout + )(implicit ec: ExecutionContext): Future[Map[TopicPartition, Long]] = + (consumerActor ? GetEndOffsets(partitions))(timeout) + .mapTo[EndOffsets] + .map(_.response.get) + + def getEndOffsetForPartition( + consumerActor: ActorRef, + partition: TopicPartition, + timeout: Timeout + )(implicit ec: ExecutionContext): Future[Long] = + getEndOffsets(consumerActor, Set(partition), timeout) + .map(endOffsets => endOffsets(partition)) } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index d33567e86..7ab5cbd5d 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -67,7 +67,7 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { } @Test - public void shouldFailInCaseOfAnExceptionDuringFetchOffsetsForNonExistingTopics() { + public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetsForNonExistingTopics() { expectedException.expect(CompletionException.class); expectedException.expectCause( IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); @@ -108,7 +108,7 @@ public void shouldFetchBeginningOffsetForGivenPartition() { } @Test - public void shouldFailInCaseOfAnExceptionDuringFetchOffsetForNonExistingTopic() { + public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetForNonExistingTopic() { expectedException.expect(CompletionException.class); expectedException.expectCause( IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); @@ -128,6 +128,92 @@ public void shouldFailInCaseOfAnExceptionDuringFetchOffsetForNonExistingTopic() response.toCompletableFuture().join(); } + @Test + public void shouldFetchEndOffsetsForGivenPartitions() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition0 = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Set partitions = Collections.singleton(partition0); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + produceString(topic1, 10, partition0.partition()).toCompletableFuture().join(); + + final CompletionStage> response = + MetadataClient.getEndOffsets(consumerActor, partitions, timeout, ec); + final Map endOffsets = response.toCompletableFuture().join(); + + assertThat(endOffsets.get(partition0), is(10L)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + } + + @Test + public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetsForNonExistingTopic() { + expectedException.expect(CompletionException.class); + expectedException.expectCause( + IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); + + final String group1 = createGroupId(); + final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); + final Set partitions = Collections.singleton(nonExistingPartition); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + final CompletionStage> response = + MetadataClient.getEndOffsets(consumerActor, partitions, timeout, ec); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + + response.toCompletableFuture().join(); + } + + @Test + public void shouldFetchEndOffsetForGivenPartition() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition0 = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + produceString(topic1, 10, partition0.partition()).toCompletableFuture().join(); + + final CompletionStage response = + MetadataClient.getEndOffsetForPartition(consumerActor, partition0, timeout, ec); + final Long endOffset = response.toCompletableFuture().join(); + + assertThat(endOffset, is(10L)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + } + + @Test + public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetForNonExistingTopic() { + expectedException.expect(CompletionException.class); + expectedException.expectCause( + IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); + + final String group1 = createGroupId(); + final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + final CompletionStage response = + MetadataClient.getEndOffsetForPartition(consumerActor, nonExistingPartition, timeout, ec); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + + response.toCompletableFuture().join(); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 0ae0ac70e..020881c75 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -23,6 +23,8 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val consumerSettings = consumerDefaults.withGroupId(group1) val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + awaitProduce(produce(topic1, 1 to 10)) + val beginningOffsets = MetadataClient .getBeginningOffsets(consumerActor, Set(partition0), 1 seconds) .futureValue @@ -32,7 +34,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } - "fail in case of an exception during fetch offsets for non-existing topics" in assertAllStagesStopped { + "fail in case of an exception during fetch beginning offsets for non-existing topics" in assertAllStagesStopped { val group1 = createGroupId(1) val nonExistingPartition = new TopicPartition("non-existing topic", 0) val consumerSettings = consumerDefaults.withGroupId(group1) @@ -53,6 +55,8 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val consumerSettings = consumerDefaults.withGroupId(group1) val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + awaitProduce(produce(topic1, 1 to 10)) + val beginningOffset = MetadataClient .getBeginningOffsetForPartition(consumerActor, partition0, 1 seconds) .futureValue @@ -62,7 +66,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } - "fail in case of an exception during fetch offset for non-existing topic" in assertAllStagesStopped { + "fail in case of an exception during fetch beginning offset for non-existing topic" in assertAllStagesStopped { val group1 = createGroupId(1) val nonExistingPartition = new TopicPartition("non-existing topic", 0) val consumerSettings = consumerDefaults.withGroupId(group1) @@ -75,5 +79,69 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } + + "fetch end offsets for given partitions" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + awaitProduce(produce(topic1, 1 to 10)) + + val endOffsets = MetadataClient + .getEndOffsets(consumerActor, Set(partition0), 1 seconds) + .futureValue + + endOffsets(partition0) shouldBe 10 + + consumerActor ! KafkaConsumerActor.Stop + } + + "fail in case of an exception during fetch end offsets for non-existing topics" in assertAllStagesStopped { + val group1 = createGroupId(1) + val nonExistingPartition = new TopicPartition("non-existing topic", 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + val endOffsetsFuture = MetadataClient + .getEndOffsets(consumerActor, Set(nonExistingPartition), 1 seconds) + + endOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] + + consumerActor ! KafkaConsumerActor.Stop + } + + "fetch end offset for given partition" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + awaitProduce(produce(topic1, 1 to 10)) + + val endOffset = MetadataClient + .getEndOffsetForPartition(consumerActor, partition0, 1 seconds) + .futureValue + + endOffset shouldBe 10 + + consumerActor ! KafkaConsumerActor.Stop + } + + "fail in case of an exception during fetch end offset for non-existing topic" in assertAllStagesStopped { + val group1 = createGroupId(1) + val nonExistingPartition = new TopicPartition("non-existing topic", 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + val endOffsetFuture = MetadataClient + .getEndOffsetForPartition(consumerActor, nonExistingPartition, 1 seconds) + + endOffsetFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] + + consumerActor ! KafkaConsumerActor.Stop + } } } From bb6f27ec2fcbcbdbd19cd5e418cc24b46f5b196f Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 18:54:29 +0200 Subject: [PATCH 12/31] Add scaladsl.MetadataClient.getListTopics --- .../akka/kafka/scaladsl/MetadataClient.scala | 12 +++++++-- .../kafka/scaladsl/MetadataClientSpec.scala | 27 ++++++++++++++++++- 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 0391da74e..d343075e1 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -6,10 +6,10 @@ package akka.kafka.scaladsl import akka.actor.ActorRef -import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets} +import akka.kafka.Metadata.{GetBeginningOffsets, BeginningOffsets, GetEndOffsets, EndOffsets, ListTopics, Topics} import akka.pattern.ask import akka.util.Timeout -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.concurrent.{ExecutionContext, Future} @@ -48,4 +48,12 @@ object MetadataClient { )(implicit ec: ExecutionContext): Future[Long] = getEndOffsets(consumerActor, Set(partition), timeout) .map(endOffsets => endOffsets(partition)) + + def getListTopics( + consumerActor: ActorRef, + timeout: Timeout + )(implicit ec: ExecutionContext): Future[Map[String, List[PartitionInfo]]] = + (consumerActor ? ListTopics)(timeout) + .mapTo[Topics] + .map(_.response.get) } diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 020881c75..fe985a747 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -8,7 +8,7 @@ package akka.kafka.scaladsl import akka.kafka.KafkaConsumerActor import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.language.postfixOps import scala.concurrent.duration._ @@ -143,5 +143,30 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } + + "fetch list of topics" in assertAllStagesStopped { + val group = createGroupId(1) + val topic1 = createTopic(suffix = 1, partitions = 2) + val topic2 = createTopic(suffix = 2, partitions = 1) + val consumerSettings = consumerDefaults.withGroupId(group) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + awaitProduce(produce(topic1, 1 to 10, partition = 0)) + awaitProduce(produce(topic1, 1 to 10, partition = 1)) + awaitProduce(produce(topic2, 1 to 10, partition = 0)) + + val topics = MetadataClient + .getListTopics(consumerActor, 1 second) + .futureValue + + val expectedPartitionsForTopic1 = (topic1, 0) :: (topic1, 1) :: Nil + val expectedPartitionsForTopic2 = (topic2, 0) :: Nil + + topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 + topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 + } } + + private val mapToTopicPartition = (p: PartitionInfo) => (p.topic(), p.partition()) + } From 2be13e6630906e9f2c521434cff2b2789cf30b3f Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 19:26:38 +0200 Subject: [PATCH 13/31] Add javadsl.MetadataClient.getListTopics --- .../akka/kafka/javadsl/MetadataClient.scala | 16 ++++++++- .../java/docs/javadsl/MetadataClientTest.java | 33 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index ab96183b1..7fde88fdf 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -9,7 +9,7 @@ import java.util.concurrent.{CompletionStage, Executor} import akka.actor.ActorRef import akka.util.Timeout -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.compat.java8.FutureConverters._ @@ -73,4 +73,18 @@ object MetadataClient { .map(Long.box) .toJava } + + def getListTopics( + consumerActor: ActorRef, + timeout: Timeout, + executor: Executor + ): CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]] = { + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getListTopics(consumerActor, timeout) + .map { topics => + topics.view.mapValues(partitionsInfo => partitionsInfo.asJava).toMap.asJava + } + .toJava + } } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 7ab5cbd5d..ab87c559d 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -15,6 +15,7 @@ import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; import akka.util.Timeout; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.hamcrest.core.IsInstanceOf; import org.junit.AfterClass; @@ -23,6 +24,7 @@ import org.junit.rules.ExpectedException; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletionException; @@ -31,7 +33,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import static java.util.stream.Collectors.toSet; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.core.Is.is; public class MetadataClientTest extends TestcontainersKafkaJunit4Test { @@ -214,6 +218,35 @@ public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetForNonExistingTopic response.toCompletableFuture().join(); } + @Test + public void shouldFetchTopicList() { + final String group = createGroupId(); + final String topic1 = createTopic(1, 2); + final String topic2 = createTopic(2, 1); + final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + produceString(topic1, 10, 0).toCompletableFuture().join(); + produceString(topic1, 10, 1).toCompletableFuture().join(); + produceString(topic2, 10, 0).toCompletableFuture().join(); + + final CompletionStage>> response = + MetadataClient.getListTopics(consumerActor, timeout, ec); + final Map> topics = response.toCompletableFuture().join(); + + final Set partitionsForTopic1 = + topics.get(topic1).stream().map(PartitionInfo::partition).collect(toSet()); + + final Set partitionsForTopic2 = + topics.get(topic2).stream().map(PartitionInfo::partition).collect(toSet()); + + assertThat(partitionsForTopic1, containsInAnyOrder(0, 1)); + assertThat(partitionsForTopic2, containsInAnyOrder(0)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); From 42c916d3176019c43ca7f2d80615d88ade73d9a7 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 19:29:01 +0200 Subject: [PATCH 14/31] Remove unnecessary tests --- .../java/docs/javadsl/MetadataClientTest.java | 42 ------------------- .../kafka/scaladsl/MetadataClientSpec.scala | 28 ------------- 2 files changed, 70 deletions(-) diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index ab87c559d..cf17bf962 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -111,27 +111,6 @@ public void shouldFetchBeginningOffsetForGivenPartition() { consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } - @Test - public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetForNonExistingTopic() { - expectedException.expect(CompletionException.class); - expectedException.expectCause( - IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); - - final String group1 = createGroupId(); - final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); - final ConsumerSettings consumerSettings = - consumerDefaults().withGroupId(group1); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - final CompletionStage response = - MetadataClient.getBeginningOffsetForPartition(consumerActor, nonExistingPartition, timeout, ec); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - - response.toCompletableFuture().join(); - } - @Test public void shouldFetchEndOffsetsForGivenPartitions() { final String topic1 = createTopic(); @@ -197,27 +176,6 @@ public void shouldFetchEndOffsetForGivenPartition() { consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } - @Test - public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetForNonExistingTopic() { - expectedException.expect(CompletionException.class); - expectedException.expectCause( - IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); - - final String group1 = createGroupId(); - final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); - final ConsumerSettings consumerSettings = - consumerDefaults().withGroupId(group1); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - final CompletionStage response = - MetadataClient.getEndOffsetForPartition(consumerActor, nonExistingPartition, timeout, ec); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - - response.toCompletableFuture().join(); - } - @Test public void shouldFetchTopicList() { final String group = createGroupId(); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index fe985a747..4a016d588 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -66,20 +66,6 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } - "fail in case of an exception during fetch beginning offset for non-existing topic" in assertAllStagesStopped { - val group1 = createGroupId(1) - val nonExistingPartition = new TopicPartition("non-existing topic", 0) - val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - val beginningOffsetFuture = MetadataClient - .getBeginningOffsetForPartition(consumerActor, nonExistingPartition, 1 seconds) - - beginningOffsetFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - - consumerActor ! KafkaConsumerActor.Stop - } - "fetch end offsets for given partitions" in assertAllStagesStopped { val topic1 = createTopic(1) val group1 = createGroupId(1) @@ -130,20 +116,6 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { consumerActor ! KafkaConsumerActor.Stop } - "fail in case of an exception during fetch end offset for non-existing topic" in assertAllStagesStopped { - val group1 = createGroupId(1) - val nonExistingPartition = new TopicPartition("non-existing topic", 0) - val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - val endOffsetFuture = MetadataClient - .getEndOffsetForPartition(consumerActor, nonExistingPartition, 1 seconds) - - endOffsetFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - - consumerActor ! KafkaConsumerActor.Stop - } - "fetch list of topics" in assertAllStagesStopped { val group = createGroupId(1) val topic1 = createTopic(suffix = 1, partitions = 2) From dd2c27cefe26ad51b454976abd00f0299384fcc6 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 20:22:47 +0200 Subject: [PATCH 15/31] Rename method getListTopics to listTopics --- core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala | 4 ++-- core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala | 4 ++-- tests/src/test/java/docs/javadsl/MetadataClientTest.java | 2 +- .../test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 7fde88fdf..a200e0ef7 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -74,14 +74,14 @@ object MetadataClient { .toJava } - def getListTopics( + def listTopics( consumerActor: ActorRef, timeout: Timeout, executor: Executor ): CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]] = { implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) akka.kafka.scaladsl.MetadataClient - .getListTopics(consumerActor, timeout) + .listTopics(consumerActor, timeout) .map { topics => topics.view.mapValues(partitionsInfo => partitionsInfo.asJava).toMap.asJava } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index d343075e1..799e62610 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -6,7 +6,7 @@ package akka.kafka.scaladsl import akka.actor.ActorRef -import akka.kafka.Metadata.{GetBeginningOffsets, BeginningOffsets, GetEndOffsets, EndOffsets, ListTopics, Topics} +import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets, ListTopics, Topics} import akka.pattern.ask import akka.util.Timeout import org.apache.kafka.common.{PartitionInfo, TopicPartition} @@ -49,7 +49,7 @@ object MetadataClient { getEndOffsets(consumerActor, Set(partition), timeout) .map(endOffsets => endOffsets(partition)) - def getListTopics( + def listTopics( consumerActor: ActorRef, timeout: Timeout )(implicit ec: ExecutionContext): Future[Map[String, List[PartitionInfo]]] = diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index cf17bf962..23698a483 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -190,7 +190,7 @@ public void shouldFetchTopicList() { produceString(topic2, 10, 0).toCompletableFuture().join(); final CompletionStage>> response = - MetadataClient.getListTopics(consumerActor, timeout, ec); + MetadataClient.listTopics(consumerActor, timeout, ec); final Map> topics = response.toCompletableFuture().join(); final Set partitionsForTopic1 = diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 4a016d588..6d378b357 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -128,7 +128,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { awaitProduce(produce(topic2, 1 to 10, partition = 0)) val topics = MetadataClient - .getListTopics(consumerActor, 1 second) + .listTopics(consumerActor, 1 second) .futureValue val expectedPartitionsForTopic1 = (topic1, 0) :: (topic1, 1) :: Nil From 0740da79c77b85e7e86ce49ca15f0e40590fd95e Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 21:01:09 +0200 Subject: [PATCH 16/31] Add scaladsl.MetadataClient.getPartitionsFor --- .../akka/kafka/scaladsl/MetadataClient.scala | 20 ++++++++++++++++++- .../kafka/scaladsl/MetadataClientSpec.scala | 16 +++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 799e62610..aefb00f74 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -6,7 +6,16 @@ package akka.kafka.scaladsl import akka.actor.ActorRef -import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets, ListTopics, Topics} +import akka.kafka.Metadata.{ + BeginningOffsets, + EndOffsets, + GetBeginningOffsets, + GetEndOffsets, + GetPartitionsFor, + ListTopics, + PartitionsFor, + Topics +} import akka.pattern.ask import akka.util.Timeout import org.apache.kafka.common.{PartitionInfo, TopicPartition} @@ -56,4 +65,13 @@ object MetadataClient { (consumerActor ? ListTopics)(timeout) .mapTo[Topics] .map(_.response.get) + + def getPartitionsFor( + consumerActor: ActorRef, + topic: String, + timeout: Timeout + )(implicit ec: ExecutionContext): Future[List[PartitionInfo]] = + (consumerActor ? GetPartitionsFor(topic))(timeout) + .mapTo[PartitionsFor] + .map(_.response.get) } diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 6d378b357..04ebe5713 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -137,6 +137,22 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 } + + "fetch partitions of given topic" in assertAllStagesStopped { + val group = createGroupId(1) + val topic = createTopic(suffix = 1, partitions = 2) + val consumerSettings = consumerDefaults.withGroupId(group) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + awaitProduce(produce(topic, 1 to 10, partition = 0)) + awaitProduce(produce(topic, 1 to 10, partition = 1)) + + val partitionsInfo = MetadataClient + .getPartitionsFor(consumerActor, topic, 1 second) + .futureValue + + partitionsInfo.leftSideValue.map(_.partition()) shouldBe List(0, 1) + } } private val mapToTopicPartition = (p: PartitionInfo) => (p.topic(), p.partition()) From eadc7ba8cdf7f7acdc009070b6baa8da80c3d2d7 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 29 Sep 2019 23:53:24 +0200 Subject: [PATCH 17/31] Add javadsl.MetadataClient.getPartitionsFor --- .../akka/kafka/javadsl/MetadataClient.scala | 15 ++++++++++++ .../java/docs/javadsl/MetadataClientTest.java | 23 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index a200e0ef7..059b0fe3f 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -87,4 +87,19 @@ object MetadataClient { } .toJava } + + def getPartitionsFor( + consumerActor: ActorRef, + topic: java.lang.String, + timeout: Timeout, + executor: Executor + ): CompletionStage[java.util.List[PartitionInfo]] = { + implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) + akka.kafka.scaladsl.MetadataClient + .getPartitionsFor(consumerActor, topic, timeout) + .map { partitionsInfo => + partitionsInfo.asJava + } + .toJava + } } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 23698a483..20425607d 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -205,6 +205,29 @@ public void shouldFetchTopicList() { consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } + @Test + public void shouldFetchPartitionsInfoForGivenTopic() { + final String group = createGroupId(); + final String topic = createTopic(1, 2); + final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + + produceString(topic, 10, 0).toCompletableFuture().join(); + produceString(topic, 10, 1).toCompletableFuture().join(); + + final CompletionStage> response = + MetadataClient.getPartitionsFor(consumerActor, topic, timeout, ec); + final List partitionInfos = response.toCompletableFuture().join(); + + final Set partitions = + partitionInfos.stream().map(PartitionInfo::partition).collect(toSet()); + + assertThat(partitions, containsInAnyOrder(0, 1)); + + consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); From 74e30938bf4f351a3d95da34920887b5e831e8f2 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 7 Oct 2019 16:55:56 +0200 Subject: [PATCH 18/31] WIP: Proposal of MetadataClient as a class. --- .../akka/kafka/javadsl/MetadataClient.scala | 111 ++++--------- .../akka/kafka/scaladsl/MetadataClient.scala | 104 +++++------- .../java/docs/javadsl/MetadataClientTest.java | 150 ++---------------- .../kafka/scaladsl/MetadataClientSpec.scala | 121 ++------------ 4 files changed, 102 insertions(+), 384 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 059b0fe3f..a44069629 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -3,103 +3,50 @@ * Copyright (C) 2016 - 2019 Lightbend Inc. */ +/* + * Copyright (C) 2014 - 2016 Softwaremill + * Copyright (C) 2016 - 2019 Lightbend Inc. + */ + package akka.kafka.javadsl -import java.util.concurrent.{CompletionStage, Executor} +import java.util.concurrent.CompletionStage -import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.dispatch.ExecutionContexts +import akka.kafka.ConsumerSettings import akka.util.Timeout -import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.apache.kafka.common.TopicPartition -import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.compat.java8.FutureConverters._ import scala.collection.compat._ import scala.collection.JavaConverters._ -object MetadataClient { +class MetadataClient(actorSystem: ActorSystem, timeout: Timeout) { + + private val metadataClient = new akka.kafka.scaladsl.MetadataClient(actorSystem, timeout) - def getBeginningOffsets( - consumerActor: ActorRef, - partitions: java.util.Set[TopicPartition], - timeout: Timeout, - executor: Executor - ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .getBeginningOffsets(consumerActor, partitions.asScala.toSet, timeout) + def getBeginningOffsets[K, V]( + consumerSettings: ConsumerSettings[K, V], + partitions: java.util.Set[TopicPartition] + ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = + metadataClient + .getBeginningOffsets(consumerSettings, partitions.asScala.toSet) .map { beginningOffsets => beginningOffsets.view.mapValues(Long.box).toMap.asJava - } + }(ExecutionContexts.sameThreadExecutionContext) .toJava - } - def getBeginningOffsetForPartition( - consumerActor: ActorRef, - partition: TopicPartition, - timeout: Timeout, - executor: Executor - ): CompletionStage[java.lang.Long] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .getBeginningOffsetForPartition(consumerActor, partition, timeout) - .map(Long.box) + def getBeginningOffsetForPartition[K, V]( + consumerSettings: ConsumerSettings[K, V], + partition: TopicPartition + ): CompletionStage[java.lang.Long] = + metadataClient + .getBeginningOffsetForPartition(consumerSettings, partition) + .map(Long.box)(ExecutionContexts.sameThreadExecutionContext) .toJava - } - def getEndOffsets( - consumerActor: ActorRef, - partitions: java.util.Set[TopicPartition], - timeout: Timeout, - executor: Executor - ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .getEndOffsets(consumerActor, partitions.asScala.toSet, timeout) - .map { endOffsets => - endOffsets.view.mapValues(Long.box).toMap.asJava - } - .toJava - } - - def getEndOffsetForPartition( - consumerActor: ActorRef, - partition: TopicPartition, - timeout: Timeout, - executor: Executor - ): CompletionStage[java.lang.Long] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .getEndOffsetForPartition(consumerActor, partition, timeout) - .map(Long.box) - .toJava - } + def stopConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V]): Unit = + metadataClient.stopConsumerActor(consumerSettings) - def listTopics( - consumerActor: ActorRef, - timeout: Timeout, - executor: Executor - ): CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .listTopics(consumerActor, timeout) - .map { topics => - topics.view.mapValues(partitionsInfo => partitionsInfo.asJava).toMap.asJava - } - .toJava - } - - def getPartitionsFor( - consumerActor: ActorRef, - topic: java.lang.String, - timeout: Timeout, - executor: Executor - ): CompletionStage[java.util.List[PartitionInfo]] = { - implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(executor) - akka.kafka.scaladsl.MetadataClient - .getPartitionsFor(consumerActor, topic, timeout) - .map { partitionsInfo => - partitionsInfo.asJava - } - .toJava - } } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index aefb00f74..c5497e21a 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -5,73 +5,57 @@ package akka.kafka.scaladsl -import akka.actor.ActorRef -import akka.kafka.Metadata.{ - BeginningOffsets, - EndOffsets, - GetBeginningOffsets, - GetEndOffsets, - GetPartitionsFor, - ListTopics, - PartitionsFor, - Topics -} +import akka.actor.{ActorRef, ActorSystem} +import akka.dispatch.ExecutionContexts +import akka.kafka.{ConsumerSettings, KafkaConsumerActor} +import akka.kafka.Metadata.{BeginningOffsets, GetBeginningOffsets} import akka.pattern.ask import akka.util.Timeout -import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.apache.kafka.common.TopicPartition + +import scala.concurrent.{ExecutionContextExecutor, Future} +import scala.util.{Failure, Success} -import scala.concurrent.{ExecutionContext, Future} +class MetadataClient(actorSystem: ActorSystem, timeout: Timeout) { -object MetadataClient { + private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() + private implicit val system: ActorSystem = actorSystem + private implicit val ec: ExecutionContextExecutor = actorSystem.dispatcher - def getBeginningOffsets( - consumerActor: ActorRef, - partitions: Set[TopicPartition], - timeout: Timeout - )(implicit ec: ExecutionContext): Future[Map[TopicPartition, Long]] = + def getBeginningOffsets[K, V]( + consumerSettings: ConsumerSettings[K, V], + partitions: Set[TopicPartition] + ): Future[Map[TopicPartition, Long]] = { + val consumerActor: ActorRef = getConsumerActor(consumerSettings) (consumerActor ? GetBeginningOffsets(partitions))(timeout) .mapTo[BeginningOffsets] - .map(_.response.get) - - def getBeginningOffsetForPartition( - consumerActor: ActorRef, - partition: TopicPartition, - timeout: Timeout - )(implicit ec: ExecutionContext): Future[Long] = - getBeginningOffsets(consumerActor, Set(partition), timeout) + .map(_.response) + .flatMap { + case Success(res) => Future.successful(res) + case Failure(e) => Future.failed(e) + }(ExecutionContexts.sameThreadExecutionContext) + } + + def getBeginningOffsetForPartition[K, V]( + consumerSettings: ConsumerSettings[K, V], + partition: TopicPartition + ): Future[Long] = + getBeginningOffsets(consumerSettings, Set(partition)) .map(beginningOffsets => beginningOffsets(partition)) - def getEndOffsets( - consumerActor: ActorRef, - partitions: Set[TopicPartition], - timeout: Timeout - )(implicit ec: ExecutionContext): Future[Map[TopicPartition, Long]] = - (consumerActor ? GetEndOffsets(partitions))(timeout) - .mapTo[EndOffsets] - .map(_.response.get) - - def getEndOffsetForPartition( - consumerActor: ActorRef, - partition: TopicPartition, - timeout: Timeout - )(implicit ec: ExecutionContext): Future[Long] = - getEndOffsets(consumerActor, Set(partition), timeout) - .map(endOffsets => endOffsets(partition)) - - def listTopics( - consumerActor: ActorRef, - timeout: Timeout - )(implicit ec: ExecutionContext): Future[Map[String, List[PartitionInfo]]] = - (consumerActor ? ListTopics)(timeout) - .mapTo[Topics] - .map(_.response.get) - - def getPartitionsFor( - consumerActor: ActorRef, - topic: String, - timeout: Timeout - )(implicit ec: ExecutionContext): Future[List[PartitionInfo]] = - (consumerActor ? GetPartitionsFor(topic))(timeout) - .mapTo[PartitionsFor] - .map(_.response.get) + def stopConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V]): Unit = { + val consumerActor = getConsumerActor(consumerSettings) + consumerActor ! KafkaConsumerActor.Stop + } + + private def getConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V])(implicit system: ActorSystem) = { + val consumerActor = consumerActors.get(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]]) + if (consumerActor.isEmpty) { + val newConsumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + consumerActors.put(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]], newConsumerActor) + newConsumerActor + } else { + consumerActor.get + } + } } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 20425607d..1376cf44f 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -3,19 +3,21 @@ * Copyright (C) 2016 - 2019 Lightbend Inc. */ +/// * +// * Copyright (C) 2014 - 2016 Softwaremill +// * Copyright (C) 2016 - 2019 Lightbend Inc. +// */ +// package docs.javadsl; -import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.kafka.ConsumerSettings; -import akka.kafka.KafkaConsumerActor; import akka.kafka.javadsl.MetadataClient; import akka.kafka.testkit.javadsl.TestcontainersKafkaJunit4Test; import akka.stream.ActorMaterializer; import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; import akka.util.Timeout; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.hamcrest.core.IsInstanceOf; import org.junit.AfterClass; @@ -24,25 +26,18 @@ import org.junit.rules.ExpectedException; import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import static java.util.stream.Collectors.toSet; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.core.Is.is; public class MetadataClientTest extends TestcontainersKafkaJunit4Test { - private static final ActorSystem sys = ActorSystem.create("MetadataClientTest"); private static final Materializer mat = ActorMaterializer.create(sys); - private static final Executor ec = Executors.newSingleThreadExecutor(); @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -59,15 +54,13 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { consumerDefaults().withGroupId(group1); final Set partitions = Collections.singleton(partition0); final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + final MetadataClient metadataClient = new MetadataClient(sys, timeout); final CompletionStage> response = - MetadataClient.getBeginningOffsets(consumerActor, partitions, timeout, ec); + metadataClient.getBeginningOffsets(consumerSettings, partitions); final Map beginningOffsets = response.toCompletableFuture().join(); assertThat(beginningOffsets.get(partition0), is(0L)); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); } @Test @@ -82,12 +75,12 @@ public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetsForNonExisti consumerDefaults().withGroupId(group1); final Set partitions = Collections.singleton(nonExistingPartition); final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + final MetadataClient metadataClient = new MetadataClient(sys, timeout); final CompletionStage> response = - MetadataClient.getBeginningOffsets(consumerActor, partitions, timeout, ec); + metadataClient.getBeginningOffsets(consumerSettings, partitions); - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + metadataClient.stopConsumerActor(consumerSettings); response.toCompletableFuture().join(); } @@ -100,132 +93,15 @@ public void shouldFetchBeginningOffsetForGivenPartition() { final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); + final MetadataClient metadataClient = new MetadataClient(sys, timeout); final CompletionStage response = - MetadataClient.getBeginningOffsetForPartition(consumerActor, partition0, timeout, ec); + metadataClient.getBeginningOffsetForPartition(consumerSettings, partition0); final Long beginningOffset = response.toCompletableFuture().join(); assertThat(beginningOffset, is(0L)); - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - } - - @Test - public void shouldFetchEndOffsetsForGivenPartitions() { - final String topic1 = createTopic(); - final String group1 = createGroupId(); - final TopicPartition partition0 = new TopicPartition(topic1, 0); - final ConsumerSettings consumerSettings = - consumerDefaults().withGroupId(group1); - final Set partitions = Collections.singleton(partition0); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - produceString(topic1, 10, partition0.partition()).toCompletableFuture().join(); - - final CompletionStage> response = - MetadataClient.getEndOffsets(consumerActor, partitions, timeout, ec); - final Map endOffsets = response.toCompletableFuture().join(); - - assertThat(endOffsets.get(partition0), is(10L)); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - } - - @Test - public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetsForNonExistingTopic() { - expectedException.expect(CompletionException.class); - expectedException.expectCause( - IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); - - final String group1 = createGroupId(); - final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); - final Set partitions = Collections.singleton(nonExistingPartition); - final ConsumerSettings consumerSettings = - consumerDefaults().withGroupId(group1); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - final CompletionStage> response = - MetadataClient.getEndOffsets(consumerActor, partitions, timeout, ec); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - - response.toCompletableFuture().join(); - } - - @Test - public void shouldFetchEndOffsetForGivenPartition() { - final String topic1 = createTopic(); - final String group1 = createGroupId(); - final TopicPartition partition0 = new TopicPartition(topic1, 0); - final ConsumerSettings consumerSettings = - consumerDefaults().withGroupId(group1); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - produceString(topic1, 10, partition0.partition()).toCompletableFuture().join(); - - final CompletionStage response = - MetadataClient.getEndOffsetForPartition(consumerActor, partition0, timeout, ec); - final Long endOffset = response.toCompletableFuture().join(); - - assertThat(endOffset, is(10L)); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - } - - @Test - public void shouldFetchTopicList() { - final String group = createGroupId(); - final String topic1 = createTopic(1, 2); - final String topic2 = createTopic(2, 1); - final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - produceString(topic1, 10, 0).toCompletableFuture().join(); - produceString(topic1, 10, 1).toCompletableFuture().join(); - produceString(topic2, 10, 0).toCompletableFuture().join(); - - final CompletionStage>> response = - MetadataClient.listTopics(consumerActor, timeout, ec); - final Map> topics = response.toCompletableFuture().join(); - - final Set partitionsForTopic1 = - topics.get(topic1).stream().map(PartitionInfo::partition).collect(toSet()); - - final Set partitionsForTopic2 = - topics.get(topic2).stream().map(PartitionInfo::partition).collect(toSet()); - - assertThat(partitionsForTopic1, containsInAnyOrder(0, 1)); - assertThat(partitionsForTopic2, containsInAnyOrder(0)); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); - } - - @Test - public void shouldFetchPartitionsInfoForGivenTopic() { - final String group = createGroupId(); - final String topic = createTopic(1, 2); - final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final ActorRef consumerActor = system().actorOf(KafkaConsumerActor.props(consumerSettings)); - - produceString(topic, 10, 0).toCompletableFuture().join(); - produceString(topic, 10, 1).toCompletableFuture().join(); - - final CompletionStage> response = - MetadataClient.getPartitionsFor(consumerActor, topic, timeout, ec); - final List partitionInfos = response.toCompletableFuture().join(); - - final Set partitions = - partitionInfos.stream().map(PartitionInfo::partition).collect(toSet()); - - assertThat(partitions, containsInAnyOrder(0, 1)); - - consumerActor.tell(KafkaConsumerActor.stop(), ActorRef.noSender()); + metadataClient.stopConsumerActor(consumerSettings); } @AfterClass diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 04ebe5713..727f2cc76 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -5,10 +5,9 @@ package akka.kafka.scaladsl -import akka.kafka.KafkaConsumerActor import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped -import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.apache.kafka.common.TopicPartition import scala.language.postfixOps import scala.concurrent.duration._ @@ -21,31 +20,33 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val group1 = createGroupId(1) val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + + val metadataClient = new MetadataClient(system, 1 seconds) awaitProduce(produce(topic1, 1 to 10)) - val beginningOffsets = MetadataClient - .getBeginningOffsets(consumerActor, Set(partition0), 1 seconds) + val beginningOffsets = metadataClient + .getBeginningOffsets(consumerSettings, Set(partition0)) .futureValue beginningOffsets(partition0) shouldBe 0 - consumerActor ! KafkaConsumerActor.Stop + metadataClient.stopConsumerActor(consumerSettings) } "fail in case of an exception during fetch beginning offsets for non-existing topics" in assertAllStagesStopped { val group1 = createGroupId(1) val nonExistingPartition = new TopicPartition("non-existing topic", 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - val beginningOffsetsFuture = MetadataClient - .getBeginningOffsets(consumerActor, Set(nonExistingPartition), 1 seconds) + val metadataClient = new MetadataClient(system, 1 second) + + val beginningOffsetsFuture = metadataClient + .getBeginningOffsets(consumerSettings, Set(nonExistingPartition)) beginningOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - consumerActor ! KafkaConsumerActor.Stop + metadataClient.stopConsumerActor(consumerSettings) } "fetch beginning offset for given partition" in assertAllStagesStopped { @@ -53,108 +54,18 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val group1 = createGroupId(1) val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - awaitProduce(produce(topic1, 1 to 10)) - - val beginningOffset = MetadataClient - .getBeginningOffsetForPartition(consumerActor, partition0, 1 seconds) - .futureValue - - beginningOffset shouldBe 0 - - consumerActor ! KafkaConsumerActor.Stop - } - - "fetch end offsets for given partitions" in assertAllStagesStopped { - val topic1 = createTopic(1) - val group1 = createGroupId(1) - val partition0 = new TopicPartition(topic1, 0) - val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - awaitProduce(produce(topic1, 1 to 10)) - - val endOffsets = MetadataClient - .getEndOffsets(consumerActor, Set(partition0), 1 seconds) - .futureValue - - endOffsets(partition0) shouldBe 10 - - consumerActor ! KafkaConsumerActor.Stop - } - - "fail in case of an exception during fetch end offsets for non-existing topics" in assertAllStagesStopped { - val group1 = createGroupId(1) - val nonExistingPartition = new TopicPartition("non-existing topic", 0) - val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - val endOffsetsFuture = MetadataClient - .getEndOffsets(consumerActor, Set(nonExistingPartition), 1 seconds) - endOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - - consumerActor ! KafkaConsumerActor.Stop - } - - "fetch end offset for given partition" in assertAllStagesStopped { - val topic1 = createTopic(1) - val group1 = createGroupId(1) - val partition0 = new TopicPartition(topic1, 0) - val consumerSettings = consumerDefaults.withGroupId(group1) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) + val metadataClient = new MetadataClient(system, 1 second) awaitProduce(produce(topic1, 1 to 10)) - val endOffset = MetadataClient - .getEndOffsetForPartition(consumerActor, partition0, 1 seconds) + val beginningOffset = metadataClient + .getBeginningOffsetForPartition(consumerSettings, partition0) .futureValue - endOffset shouldBe 10 - - consumerActor ! KafkaConsumerActor.Stop - } - - "fetch list of topics" in assertAllStagesStopped { - val group = createGroupId(1) - val topic1 = createTopic(suffix = 1, partitions = 2) - val topic2 = createTopic(suffix = 2, partitions = 1) - val consumerSettings = consumerDefaults.withGroupId(group) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - awaitProduce(produce(topic1, 1 to 10, partition = 0)) - awaitProduce(produce(topic1, 1 to 10, partition = 1)) - awaitProduce(produce(topic2, 1 to 10, partition = 0)) - - val topics = MetadataClient - .listTopics(consumerActor, 1 second) - .futureValue - - val expectedPartitionsForTopic1 = (topic1, 0) :: (topic1, 1) :: Nil - val expectedPartitionsForTopic2 = (topic2, 0) :: Nil - - topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 - topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 - } - - "fetch partitions of given topic" in assertAllStagesStopped { - val group = createGroupId(1) - val topic = createTopic(suffix = 1, partitions = 2) - val consumerSettings = consumerDefaults.withGroupId(group) - val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - - awaitProduce(produce(topic, 1 to 10, partition = 0)) - awaitProduce(produce(topic, 1 to 10, partition = 1)) - - val partitionsInfo = MetadataClient - .getPartitionsFor(consumerActor, topic, 1 second) - .futureValue + beginningOffset shouldBe 0 - partitionsInfo.leftSideValue.map(_.partition()) shouldBe List(0, 1) + metadataClient.stopConsumerActor(consumerSettings) } } - - private val mapToTopicPartition = (p: PartitionInfo) => (p.topic(), p.partition()) - } From ba80d6dab51874b0d9677122ad43f271503e5c9a Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Tue, 8 Oct 2019 15:14:25 +0200 Subject: [PATCH 19/31] WIP: Proposal of MetadataClient as a companion object. --- .../akka/kafka/javadsl/MetadataClient.scala | 40 +++++++++++------- .../akka/kafka/scaladsl/MetadataClient.scala | 42 ++++++++++--------- .../java/docs/javadsl/MetadataClientTest.java | 28 ++++++++----- .../kafka/scaladsl/MetadataClientSpec.scala | 18 ++++---- 4 files changed, 74 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index a44069629..16e6771ca 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -10,9 +10,9 @@ package akka.kafka.javadsl -import java.util.concurrent.CompletionStage +import java.util.concurrent.{CompletionStage, Executor} -import akka.actor.ActorSystem +import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts import akka.kafka.ConsumerSettings import akka.util.Timeout @@ -21,32 +21,44 @@ import org.apache.kafka.common.TopicPartition import scala.compat.java8.FutureConverters._ import scala.collection.compat._ import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionContextExecutor -class MetadataClient(actorSystem: ActorSystem, timeout: Timeout) { - - private val metadataClient = new akka.kafka.scaladsl.MetadataClient(actorSystem, timeout) +class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient) { def getBeginningOffsets[K, V]( - consumerSettings: ConsumerSettings[K, V], partitions: java.util.Set[TopicPartition] ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = metadataClient - .getBeginningOffsets(consumerSettings, partitions.asScala.toSet) + .getBeginningOffsets(partitions.asScala.toSet) .map { beginningOffsets => beginningOffsets.view.mapValues(Long.box).toMap.asJava }(ExecutionContexts.sameThreadExecutionContext) .toJava - def getBeginningOffsetForPartition[K, V]( - consumerSettings: ConsumerSettings[K, V], - partition: TopicPartition - ): CompletionStage[java.lang.Long] = + def getBeginningOffsetForPartition[K, V](partition: TopicPartition): CompletionStage[java.lang.Long] = metadataClient - .getBeginningOffsetForPartition(consumerSettings, partition) + .getBeginningOffsetForPartition(partition) .map(Long.box)(ExecutionContexts.sameThreadExecutionContext) .toJava - def stopConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V]): Unit = - metadataClient.stopConsumerActor(consumerSettings) + def stop(): Unit = + metadataClient.stop() +} + +object MetadataClient { + + def create(consumerActor: ActorRef, timeout: Timeout, executor: Executor): MetadataClient = { + implicit val ec: ExecutionContextExecutor = ExecutionContexts.fromExecutor(executor) + val metadataClient = akka.kafka.scaladsl.MetadataClient.create(consumerActor, timeout) + new MetadataClient(metadataClient) + } + def create[K, V](consumerSettings: ConsumerSettings[K, V], + timeout: Timeout, + system: ActorSystem, + executor: Executor): MetadataClient = { + val metadataClient = akka.kafka.scaladsl.MetadataClient + .create(consumerSettings, timeout)(system, ExecutionContexts.fromExecutor(executor)) + new MetadataClient(metadataClient) + } } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index c5497e21a..76044916e 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -13,20 +13,12 @@ import akka.pattern.ask import akka.util.Timeout import org.apache.kafka.common.TopicPartition -import scala.concurrent.{ExecutionContextExecutor, Future} +import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} -class MetadataClient(actorSystem: ActorSystem, timeout: Timeout) { +class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext) { - private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() - private implicit val system: ActorSystem = actorSystem - private implicit val ec: ExecutionContextExecutor = actorSystem.dispatcher - - def getBeginningOffsets[K, V]( - consumerSettings: ConsumerSettings[K, V], - partitions: Set[TopicPartition] - ): Future[Map[TopicPartition, Long]] = { - val consumerActor: ActorRef = getConsumerActor(consumerSettings) + def getBeginningOffsets(partitions: Set[TopicPartition]): Future[Map[TopicPartition, Long]] = (consumerActor ? GetBeginningOffsets(partitions))(timeout) .mapTo[BeginningOffsets] .map(_.response) @@ -34,21 +26,31 @@ class MetadataClient(actorSystem: ActorSystem, timeout: Timeout) { case Success(res) => Future.successful(res) case Failure(e) => Future.failed(e) }(ExecutionContexts.sameThreadExecutionContext) - } - def getBeginningOffsetForPartition[K, V]( - consumerSettings: ConsumerSettings[K, V], - partition: TopicPartition - ): Future[Long] = - getBeginningOffsets(consumerSettings, Set(partition)) + def getBeginningOffsetForPartition(partition: TopicPartition): Future[Long] = + getBeginningOffsets(Set(partition)) .map(beginningOffsets => beginningOffsets(partition)) - def stopConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V]): Unit = { - val consumerActor = getConsumerActor(consumerSettings) + def stop(): Unit = consumerActor ! KafkaConsumerActor.Stop +} + +object MetadataClient { + + private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() + + def create(consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext): MetadataClient = + new MetadataClient(consumerActor, timeout) + + def create[K, V]( + consumerSettings: ConsumerSettings[K, V], + timeout: Timeout + )(implicit system: ActorSystem, ec: ExecutionContext): MetadataClient = { + val consumerActor = getConsumerActor(system, consumerSettings) + new MetadataClient(consumerActor, timeout) } - private def getConsumerActor[K, V](consumerSettings: ConsumerSettings[K, V])(implicit system: ActorSystem) = { + private def getConsumerActor[K, V](system: ActorSystem, consumerSettings: ConsumerSettings[K, V]) = { val consumerActor = consumerActors.get(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]]) if (consumerActor.isEmpty) { val newConsumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 1376cf44f..2cbd7c1c4 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -30,6 +30,8 @@ import java.util.Set; import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import static org.hamcrest.MatcherAssert.assertThat; @@ -38,6 +40,8 @@ public class MetadataClientTest extends TestcontainersKafkaJunit4Test { private static final ActorSystem sys = ActorSystem.create("MetadataClientTest"); private static final Materializer mat = ActorMaterializer.create(sys); + private static final Executor executor = Executors.newSingleThreadExecutor(); + private static final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -53,14 +57,16 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); final Set partitions = Collections.singleton(partition0); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final MetadataClient metadataClient = new MetadataClient(sys, timeout); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); final CompletionStage> response = - metadataClient.getBeginningOffsets(consumerSettings, partitions); + metadataClient.getBeginningOffsets(partitions); final Map beginningOffsets = response.toCompletableFuture().join(); assertThat(beginningOffsets.get(partition0), is(0L)); + + metadataClient.stop(); } @Test @@ -74,13 +80,13 @@ public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetsForNonExisti final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); final Set partitions = Collections.singleton(nonExistingPartition); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final MetadataClient metadataClient = new MetadataClient(sys, timeout); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); final CompletionStage> response = - metadataClient.getBeginningOffsets(consumerSettings, partitions); + metadataClient.getBeginningOffsets(partitions); - metadataClient.stopConsumerActor(consumerSettings); + metadataClient.stop(); response.toCompletableFuture().join(); } @@ -92,16 +98,16 @@ public void shouldFetchBeginningOffsetForGivenPartition() { final TopicPartition partition0 = new TopicPartition(topic1, 0); final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); - final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); - final MetadataClient metadataClient = new MetadataClient(sys, timeout); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); final CompletionStage response = - metadataClient.getBeginningOffsetForPartition(consumerSettings, partition0); + metadataClient.getBeginningOffsetForPartition(partition0); final Long beginningOffset = response.toCompletableFuture().join(); assertThat(beginningOffset, is(0L)); - metadataClient.stopConsumerActor(consumerSettings); + metadataClient.stop(); } @AfterClass diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 727f2cc76..baf4964df 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -21,17 +21,17 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val metadataClient = new MetadataClient(system, 1 seconds) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) awaitProduce(produce(topic1, 1 to 10)) val beginningOffsets = metadataClient - .getBeginningOffsets(consumerSettings, Set(partition0)) + .getBeginningOffsets(Set(partition0)) .futureValue beginningOffsets(partition0) shouldBe 0 - metadataClient.stopConsumerActor(consumerSettings) + metadataClient.stop() } "fail in case of an exception during fetch beginning offsets for non-existing topics" in assertAllStagesStopped { @@ -39,14 +39,14 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val nonExistingPartition = new TopicPartition("non-existing topic", 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val metadataClient = new MetadataClient(system, 1 second) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) val beginningOffsetsFuture = metadataClient - .getBeginningOffsets(consumerSettings, Set(nonExistingPartition)) + .getBeginningOffsets(Set(nonExistingPartition)) beginningOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - metadataClient.stopConsumerActor(consumerSettings) + metadataClient.stop() } "fetch beginning offset for given partition" in assertAllStagesStopped { @@ -55,17 +55,17 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { val partition0 = new TopicPartition(topic1, 0) val consumerSettings = consumerDefaults.withGroupId(group1) - val metadataClient = new MetadataClient(system, 1 second) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) awaitProduce(produce(topic1, 1 to 10)) val beginningOffset = metadataClient - .getBeginningOffsetForPartition(consumerSettings, partition0) + .getBeginningOffsetForPartition(partition0) .futureValue beginningOffset shouldBe 0 - metadataClient.stopConsumerActor(consumerSettings) + metadataClient.stop() } } } From bedd7d744b0c98a871719328ccff487505cf1645 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Tue, 8 Oct 2019 15:22:41 +0200 Subject: [PATCH 20/31] Refactoring --- .../akka/kafka/scaladsl/MetadataClient.scala | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 76044916e..e108179b1 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -37,8 +37,6 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici object MetadataClient { - private val consumerActors = scala.collection.mutable.Map[ConsumerSettings[Any, Any], ActorRef]() - def create(consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext): MetadataClient = new MetadataClient(consumerActor, timeout) @@ -46,18 +44,7 @@ object MetadataClient { consumerSettings: ConsumerSettings[K, V], timeout: Timeout )(implicit system: ActorSystem, ec: ExecutionContext): MetadataClient = { - val consumerActor = getConsumerActor(system, consumerSettings) + val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) new MetadataClient(consumerActor, timeout) } - - private def getConsumerActor[K, V](system: ActorSystem, consumerSettings: ConsumerSettings[K, V]) = { - val consumerActor = consumerActors.get(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]]) - if (consumerActor.isEmpty) { - val newConsumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - consumerActors.put(consumerSettings.asInstanceOf[ConsumerSettings[Any, Any]], newConsumerActor) - newConsumerActor - } else { - consumerActor.get - } - } } From b8a55a6303a81414008f4e1ac98accd77c912708 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 27 Oct 2019 13:57:35 +0100 Subject: [PATCH 21/31] Add MetadataClient.getEndOffsets and MetadataClient.getEndOffsetForPartition --- .../akka/kafka/javadsl/MetadataClient.scala | 16 ++++ .../akka/kafka/scaladsl/MetadataClient.scala | 15 +++- .../java/docs/javadsl/MetadataClientTest.java | 74 +++++++++++++++++-- .../kafka/scaladsl/MetadataClientSpec.scala | 45 +++++++++++ 4 files changed, 143 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 16e6771ca..53ddf1e68 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -41,6 +41,22 @@ class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient .map(Long.box)(ExecutionContexts.sameThreadExecutionContext) .toJava + def getEndOffsets( + partitions: java.util.Set[TopicPartition] + ): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = + metadataClient + .getEndOffsets(partitions.asScala.toSet) + .map { endOffsets => + endOffsets.view.mapValues(Long.box).toMap.asJava + }(ExecutionContexts.sameThreadExecutionContext) + .toJava + + def getEndOffsetForPartition(partition: TopicPartition): CompletionStage[java.lang.Long] = + metadataClient + .getEndOffsetForPartition(partition) + .map(Long.box)(ExecutionContexts.sameThreadExecutionContext) + .toJava + def stop(): Unit = metadataClient.stop() } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index e108179b1..fe6ff1a07 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -8,7 +8,7 @@ package akka.kafka.scaladsl import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts import akka.kafka.{ConsumerSettings, KafkaConsumerActor} -import akka.kafka.Metadata.{BeginningOffsets, GetBeginningOffsets} +import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets} import akka.pattern.ask import akka.util.Timeout import org.apache.kafka.common.TopicPartition @@ -31,6 +31,19 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici getBeginningOffsets(Set(partition)) .map(beginningOffsets => beginningOffsets(partition)) + def getEndOffsets(partitions: Set[TopicPartition]): Future[Map[TopicPartition, Long]] = + (consumerActor ? GetEndOffsets(partitions))(timeout) + .mapTo[EndOffsets] + .map(_.response) + .flatMap { + case Success(res) => Future.successful(res) + case Failure(e) => Future.failed(e) + }(ExecutionContexts.sameThreadExecutionContext) + + def getEndOffsetForPartition(partition: TopicPartition): Future[Long] = + getEndOffsets(Set(partition)) + .map(endOffsets => endOffsets(partition)) + def stop(): Unit = consumerActor ! KafkaConsumerActor.Stop } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 2cbd7c1c4..8dfd764cb 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -53,10 +53,10 @@ public MetadataClientTest() { public void shouldFetchBeginningOffsetsForGivenPartitions() { final String topic1 = createTopic(); final String group1 = createGroupId(); - final TopicPartition partition0 = new TopicPartition(topic1, 0); + final TopicPartition partition = new TopicPartition(topic1, 0); final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); - final Set partitions = Collections.singleton(partition0); + final Set partitions = Collections.singleton(partition); final MetadataClient metadataClient = MetadataClient.create(consumerSettings, timeout, sys, executor); @@ -64,7 +64,7 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { metadataClient.getBeginningOffsets(partitions); final Map beginningOffsets = response.toCompletableFuture().join(); - assertThat(beginningOffsets.get(partition0), is(0L)); + assertThat(beginningOffsets.get(partition), is(0L)); metadataClient.stop(); } @@ -95,14 +95,13 @@ public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetsForNonExisti public void shouldFetchBeginningOffsetForGivenPartition() { final String topic1 = createTopic(); final String group1 = createGroupId(); - final TopicPartition partition0 = new TopicPartition(topic1, 0); + final TopicPartition partition = new TopicPartition(topic1, 0); final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); final MetadataClient metadataClient = MetadataClient.create(consumerSettings, timeout, sys, executor); - final CompletionStage response = - metadataClient.getBeginningOffsetForPartition(partition0); + final CompletionStage response = metadataClient.getBeginningOffsetForPartition(partition); final Long beginningOffset = response.toCompletableFuture().join(); assertThat(beginningOffset, is(0L)); @@ -110,6 +109,69 @@ public void shouldFetchBeginningOffsetForGivenPartition() { metadataClient.stop(); } + @Test + public void shouldFetchEndOffsetsForGivenPartitions() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); + + produceString(topic1, 10, partition.partition()).toCompletableFuture().join(); + + final CompletionStage> response = + metadataClient.getEndOffsets(Collections.singleton(partition)); + final Map endOffsets = response.toCompletableFuture().join(); + + assertThat(endOffsets.get(partition), is(10L)); + + metadataClient.stop(); + } + + @Test + public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetsForNonExistingTopic() { + expectedException.expect(CompletionException.class); + expectedException.expectCause( + IsInstanceOf.instanceOf(org.apache.kafka.common.errors.InvalidTopicException.class)); + + final String group1 = createGroupId(); + final TopicPartition nonExistingPartition = new TopicPartition("non-existing topic", 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); + + final CompletionStage> response = + metadataClient.getEndOffsets(Collections.singleton(nonExistingPartition)); + + metadataClient.stop(); + response.toCompletableFuture().join(); + } + + @Test + public void shouldFetchEndOffsetForGivenPartition() { + final String topic1 = createTopic(); + final String group1 = createGroupId(); + final TopicPartition partition = new TopicPartition(topic1, 0); + final ConsumerSettings consumerSettings = + consumerDefaults().withGroupId(group1); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); + + produceString(topic1, 10, partition.partition()).toCompletableFuture().join(); + + final CompletionStage response = metadataClient.getEndOffsetForPartition(partition); + final Long endOffset = response.toCompletableFuture().join(); + + assertThat(endOffset, is(10L)); + metadataClient.stop(); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index baf4964df..ca4216448 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -67,5 +67,50 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { metadataClient.stop() } + + "fetch end offsets for given partitions" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) + + awaitProduce(produce(topic1, 1 to 10)) + + val endOffsets = metadataClient.getEndOffsets(Set(partition0)).futureValue + + endOffsets(partition0) shouldBe 10 + + metadataClient.stop() + } + + "fail in case of an exception during fetch end offsets for non-existing topics" in assertAllStagesStopped { + val group1 = createGroupId(1) + val nonExistingPartition = new TopicPartition("non-existing topic", 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) + + val endOffsetsFuture = metadataClient.getEndOffsets(Set(nonExistingPartition)) + + endOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] + + metadataClient.stop() + } + + "fetch end offset for given partition" in assertAllStagesStopped { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) + + awaitProduce(produce(topic1, 1 to 10)) + + val endOffset = metadataClient.getEndOffsetForPartition(partition0).futureValue + + endOffset shouldBe 10 + + metadataClient.stop() + } } } From b0ea82aab10177a38376e4c1de11472de1c7ccc7 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 27 Oct 2019 14:30:32 +0100 Subject: [PATCH 22/31] Add MetadataClient.listTopics --- .../akka/kafka/javadsl/MetadataClient.scala | 10 +++++- .../akka/kafka/scaladsl/MetadataClient.scala | 9 ++++-- .../java/docs/javadsl/MetadataClientTest.java | 32 +++++++++++++++++++ .../kafka/scaladsl/MetadataClientSpec.scala | 23 ++++++++++++- 4 files changed, 70 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 53ddf1e68..d554a632f 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -16,7 +16,7 @@ import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts import akka.kafka.ConsumerSettings import akka.util.Timeout -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.compat.java8.FutureConverters._ import scala.collection.compat._ @@ -57,6 +57,14 @@ class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient .map(Long.box)(ExecutionContexts.sameThreadExecutionContext) .toJava + def listTopics(): CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]] = + metadataClient + .listTopics() + .map { topics => + topics.view.mapValues(partitionsInfo => partitionsInfo.asJava).toMap.asJava + }(ExecutionContexts.sameThreadExecutionContext) + .toJava + def stop(): Unit = metadataClient.stop() } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index fe6ff1a07..9111a2b93 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -7,11 +7,11 @@ package akka.kafka.scaladsl import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts +import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets, ListTopics, Topics} import akka.kafka.{ConsumerSettings, KafkaConsumerActor} -import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets} import akka.pattern.ask import akka.util.Timeout -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} @@ -44,6 +44,11 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici getEndOffsets(Set(partition)) .map(endOffsets => endOffsets(partition)) + def listTopics(): Future[Map[String, List[PartitionInfo]]] = + (consumerActor ? ListTopics)(timeout) + .mapTo[Topics] + .map(_.response.get) + def stop(): Unit = consumerActor ! KafkaConsumerActor.Stop } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 8dfd764cb..39592a5a0 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -18,6 +18,7 @@ import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; import akka.util.Timeout; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.hamcrest.core.IsInstanceOf; import org.junit.AfterClass; @@ -26,6 +27,7 @@ import org.junit.rules.ExpectedException; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletionException; @@ -34,7 +36,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import static java.util.stream.Collectors.toSet; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.core.Is.is; public class MetadataClientTest extends TestcontainersKafkaJunit4Test { @@ -172,6 +176,34 @@ public void shouldFetchEndOffsetForGivenPartition() { metadataClient.stop(); } + @Test + public void shouldFetchTopicList() { + final String group = createGroupId(); + final String topic1 = createTopic(1, 2); + final String topic2 = createTopic(2, 1); + final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); + + produceString(topic1, 10, 0).toCompletableFuture().join(); + produceString(topic1, 10, 1).toCompletableFuture().join(); + produceString(topic2, 10, 0).toCompletableFuture().join(); + + final CompletionStage>> response = metadataClient.listTopics(); + + final Map> topics = response.toCompletableFuture().join(); + final Set partitionsForTopic1 = + topics.get(topic1).stream().map(PartitionInfo::partition).collect(toSet()); + final Set partitionsForTopic2 = + topics.get(topic2).stream().map(PartitionInfo::partition).collect(toSet()); + + assertThat(partitionsForTopic1, containsInAnyOrder(0, 1)); + assertThat(partitionsForTopic2, containsInAnyOrder(0)); + + metadataClient.stop(); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index ca4216448..eae254855 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -7,7 +7,7 @@ package akka.kafka.scaladsl import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.language.postfixOps import scala.concurrent.duration._ @@ -112,5 +112,26 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { metadataClient.stop() } + + "fetch list of topics" in assertAllStagesStopped { + val group = createGroupId(1) + val topic1 = createTopic(suffix = 1, partitions = 2) + val topic2 = createTopic(suffix = 2, partitions = 1) + val consumerSettings = consumerDefaults.withGroupId(group) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) + + awaitProduce(produce(topic1, 1 to 10, partition = 0)) + awaitProduce(produce(topic1, 1 to 10, partition = 1)) + awaitProduce(produce(topic2, 1 to 10, partition = 0)) + + val topics = metadataClient.listTopics().futureValue + val expectedPartitionsForTopic1 = (topic1, 0) :: (topic1, 1) :: Nil + val expectedPartitionsForTopic2 = (topic2, 0) :: Nil + + topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 + topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 + } } + + private val mapToTopicPartition = (p: PartitionInfo) => (p.topic(), p.partition()) } From 7a4b78326f25dce22ef80bbdff43369b3d042feb Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Sun, 27 Oct 2019 14:51:39 +0100 Subject: [PATCH 23/31] Add MetadataClient.getPartitionsFor --- .../akka/kafka/javadsl/MetadataClient.scala | 8 ++++++ .../akka/kafka/scaladsl/MetadataClient.scala | 26 +++++++++++++++++-- .../java/docs/javadsl/MetadataClientTest.java | 23 ++++++++++++++++ .../kafka/scaladsl/MetadataClientSpec.scala | 18 +++++++++++++ 4 files changed, 73 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index d554a632f..7ad2c0abd 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -65,6 +65,14 @@ class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient }(ExecutionContexts.sameThreadExecutionContext) .toJava + def getPartitionsFor(topic: java.lang.String): CompletionStage[java.util.List[PartitionInfo]] = + metadataClient + .getPartitionsFor(topic) + .map { partitionsInfo => + partitionsInfo.asJava + }(ExecutionContexts.sameThreadExecutionContext) + .toJava + def stop(): Unit = metadataClient.stop() } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 9111a2b93..5a70f7405 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -7,7 +7,16 @@ package akka.kafka.scaladsl import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts -import akka.kafka.Metadata.{BeginningOffsets, EndOffsets, GetBeginningOffsets, GetEndOffsets, ListTopics, Topics} +import akka.kafka.Metadata.{ + BeginningOffsets, + EndOffsets, + GetBeginningOffsets, + GetEndOffsets, + GetPartitionsFor, + ListTopics, + PartitionsFor, + Topics +} import akka.kafka.{ConsumerSettings, KafkaConsumerActor} import akka.pattern.ask import akka.util.Timeout @@ -47,7 +56,20 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici def listTopics(): Future[Map[String, List[PartitionInfo]]] = (consumerActor ? ListTopics)(timeout) .mapTo[Topics] - .map(_.response.get) + .map(_.response) + .flatMap { + case Success(res) => Future.successful(res) + case Failure(e) => Future.failed(e) + }(ExecutionContexts.sameThreadExecutionContext) + + def getPartitionsFor(topic: String): Future[List[PartitionInfo]] = + (consumerActor ? GetPartitionsFor(topic))(timeout) + .mapTo[PartitionsFor] + .map(_.response) + .flatMap { + case Success(res) => Future.successful(res) + case Failure(e) => Future.failed(e) + }(ExecutionContexts.sameThreadExecutionContext) def stop(): Unit = consumerActor ! KafkaConsumerActor.Stop diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 39592a5a0..571d91d73 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -204,6 +204,29 @@ public void shouldFetchTopicList() { metadataClient.stop(); } + @Test + public void shouldFetchPartitionsInfoForGivenTopic() { + final String group = createGroupId(); + final String topic = createTopic(1, 2); + final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group); + final Timeout timeout = new Timeout(1, TimeUnit.SECONDS); + final MetadataClient metadataClient = + MetadataClient.create(consumerSettings, timeout, sys, executor); + + produceString(topic, 10, 0).toCompletableFuture().join(); + produceString(topic, 10, 1).toCompletableFuture().join(); + + final CompletionStage> response = metadataClient.getPartitionsFor(topic); + + final List partitionInfos = response.toCompletableFuture().join(); + final Set partitions = + partitionInfos.stream().map(PartitionInfo::partition).collect(toSet()); + + assertThat(partitions, containsInAnyOrder(0, 1)); + + metadataClient.stop(); + } + @AfterClass public static void afterClass() { TestKit.shutdownActorSystem(sys); diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index eae254855..8cac70366 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -130,6 +130,24 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 + + metadataClient.stop() + } + + "fetch partitions of given topic" in assertAllStagesStopped { + val group = createGroupId(1) + val topic = createTopic(suffix = 1, partitions = 2) + val consumerSettings = consumerDefaults.withGroupId(group) + val metadataClient = MetadataClient.create(consumerSettings, 1 second) + + awaitProduce(produce(topic, 1 to 10, partition = 0)) + awaitProduce(produce(topic, 1 to 10, partition = 1)) + + val partitionsInfo = metadataClient.getPartitionsFor(topic).futureValue + + partitionsInfo.leftSideValue.map(_.partition()) shouldBe List(0, 1) + + metadataClient.stop() } } From 261b778dd7cde72092967c6868f36afa56759352 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Wed, 6 Nov 2019 19:27:37 +0100 Subject: [PATCH 24/31] Add MetadataClient.getCommittedOffset --- .../scala/akka/kafka/javadsl/MetadataClient.scala | 6 ++++++ .../scala/akka/kafka/scaladsl/MetadataClient.scala | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 7ad2c0abd..7238a9f54 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -16,6 +16,7 @@ import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts import akka.kafka.ConsumerSettings import akka.util.Timeout +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.compat.java8.FutureConverters._ @@ -73,6 +74,11 @@ class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient }(ExecutionContexts.sameThreadExecutionContext) .toJava + def getCommittedOffset(partition: TopicPartition): CompletionStage[OffsetAndMetadata] = + metadataClient + .getCommittedOffset(partition) + .toJava + def stop(): Unit = metadataClient.stop() } diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 5a70f7405..2a0e3a0a3 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -9,8 +9,10 @@ import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.ExecutionContexts import akka.kafka.Metadata.{ BeginningOffsets, + CommittedOffset, EndOffsets, GetBeginningOffsets, + GetCommittedOffset, GetEndOffsets, GetPartitionsFor, ListTopics, @@ -20,6 +22,7 @@ import akka.kafka.Metadata.{ import akka.kafka.{ConsumerSettings, KafkaConsumerActor} import akka.pattern.ask import akka.util.Timeout +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.concurrent.{ExecutionContext, Future} @@ -71,6 +74,15 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici case Failure(e) => Future.failed(e) }(ExecutionContexts.sameThreadExecutionContext) + def getCommittedOffset(partition: TopicPartition): Future[OffsetAndMetadata] = + (consumerActor ? GetCommittedOffset(partition))(timeout) + .mapTo[CommittedOffset] + .map(_.response) + .flatMap { + case Success(res) => Future.successful(res) + case Failure(e) => Future.failed(e) + }(ExecutionContexts.sameThreadExecutionContext) + def stop(): Unit = consumerActor ! KafkaConsumerActor.Stop } From 66ac730c7c28a2b2f37270373e9d5a3739bf5dc2 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Thu, 7 Nov 2019 09:52:01 +0100 Subject: [PATCH 25/31] Stop actors created by MetadataClient --- .../scala/akka/kafka/scaladsl/MetadataClient.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index 2a0e3a0a3..ba439b102 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -28,7 +28,9 @@ import org.apache.kafka.common.{PartitionInfo, TopicPartition} import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} -class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext) { +class MetadataClient private (consumerActor: ActorRef, timeout: Timeout, managedActor: Boolean)( + implicit ec: ExecutionContext +) { def getBeginningOffsets(partitions: Set[TopicPartition]): Future[Map[TopicPartition, Long]] = (consumerActor ? GetBeginningOffsets(partitions))(timeout) @@ -84,19 +86,21 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout)(implici }(ExecutionContexts.sameThreadExecutionContext) def stop(): Unit = - consumerActor ! KafkaConsumerActor.Stop + if (managedActor) { + consumerActor ! KafkaConsumerActor.Stop + } } object MetadataClient { def create(consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext): MetadataClient = - new MetadataClient(consumerActor, timeout) + new MetadataClient(consumerActor, timeout, false) def create[K, V]( consumerSettings: ConsumerSettings[K, V], timeout: Timeout )(implicit system: ActorSystem, ec: ExecutionContext): MetadataClient = { val consumerActor = system.actorOf(KafkaConsumerActor.props(consumerSettings)) - new MetadataClient(consumerActor, timeout) + new MetadataClient(consumerActor, timeout, true) } } From 00d227e244d7b1159ae5c8eb17618e553b6f35a5 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Fri, 22 Nov 2019 17:09:44 +0100 Subject: [PATCH 26/31] Update docs --- docs/src/main/paradox/consumer.md | 1 + docs/src/main/paradox/metadata-client.md | 42 +++++++++++++++++++ .../java/docs/javadsl/MetadataClientTest.java | 6 +++ .../scala/docs/scaladsl/FetchMetadata.scala | 26 ++++++++++++ 4 files changed, 75 insertions(+) create mode 100644 docs/src/main/paradox/metadata-client.md diff --git a/docs/src/main/paradox/consumer.md b/docs/src/main/paradox/consumer.md index 8e0aa8c2e..fe18f54cf 100644 --- a/docs/src/main/paradox/consumer.md +++ b/docs/src/main/paradox/consumer.md @@ -350,5 +350,6 @@ Java * [subscription](subscription.md) * [metadata](consumer-metadata.md) +* [metadata-client](metadata-client.md) @@@ diff --git a/docs/src/main/paradox/metadata-client.md b/docs/src/main/paradox/metadata-client.md new file mode 100644 index 000000000..ae33fe88a --- /dev/null +++ b/docs/src/main/paradox/metadata-client.md @@ -0,0 +1,42 @@ +--- +project.description: Access Kafka consumer metadata by calling MetadataClient. +--- +# Metadata Client + +`MetadataClient` is a thin wrapper for `KafkaConsumerActor` hiding the ask calls and mapping to the correct response types. + +To access the Kafka consumer metadata you need to create the `KafkaConsumerActor` as described in the @ref[Consumer documentation](consumer.md#sharing-the-kafkaconsumer-instance) pass it to `MetadataClient`'s factory method `create`. + +Another approach to create metadata client is passing the `ConsumerSettings` and `ActorSystem` objects to factory method. Then the metadata client manages the internal actor and stops it when the `close` method is called. + +The metadata the `MetadataClient` provides is documented in the @javadoc[Kafka Consumer API](org.apache.kafka.clients.consumer.KafkaConsumer). + +## Supported metadata + +The supported metadata are + +| Metadata | Response type | +|-------| ------- | +| Topics list | `Future[Map[String, List[PartitionInfo]]]` | +| Partitions | `Future[List[PartitionInfo]]` | +| Beginning offsets | `Future[Map[TopicPartition, Long]]` | +| End offsets | `Future[Map[TopicPartition, Long]]` | +| Committed offset | `Future[OffsetAndMetadata]` | + +@@@ warning + +Processing of these requests blocks the actor loop. The `KafkaConsumerActor` is configured to run on its own dispatcher, so just as the other remote calls to Kafka, the blocking happens within a designated thread pool. + +However, calling these during consuming might affect performance and even cause timeouts in extreme cases. + +Please consider to use a dedicated `KafkaConsumerActor` to create metadata client requests against. + +@@@ + +## Example + +Scala +: @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadata-client } + +Java +: @@ snip [snip](/tests/src/test/java/docs/javadsl/FetchMetadataTest.java) { #metadata-client } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 571d91d73..9d2d14a27 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -10,6 +10,7 @@ // package docs.javadsl; +// #metadata-client import akka.actor.ActorSystem; import akka.kafka.ConsumerSettings; import akka.kafka.javadsl.MetadataClient; @@ -18,6 +19,7 @@ import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; import akka.util.Timeout; +// #metadata-client import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.hamcrest.core.IsInstanceOf; @@ -57,6 +59,7 @@ public MetadataClientTest() { public void shouldFetchBeginningOffsetsForGivenPartitions() { final String topic1 = createTopic(); final String group1 = createGroupId(); + // #metadata-client final TopicPartition partition = new TopicPartition(topic1, 0); final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); @@ -67,10 +70,13 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { final CompletionStage> response = metadataClient.getBeginningOffsets(partitions); final Map beginningOffsets = response.toCompletableFuture().join(); + // #metadata-client assertThat(beginningOffsets.get(partition), is(0L)); + // #metadata-client metadataClient.stop(); + // #metadata-client } @Test diff --git a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala index ae71ea3b6..fb9a6100c 100644 --- a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala +++ b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala @@ -5,11 +5,13 @@ package docs.scaladsl +import akka.kafka.scaladsl.MetadataClient import akka.kafka.testkit.scaladsl.TestcontainersKafkaLike import org.scalatest.TryValues import org.scalatest.time.{Seconds, Span} // #metadata +// #metadata-client import akka.actor.ActorRef import akka.kafka.{KafkaConsumerActor, Metadata} import akka.pattern.ask @@ -20,6 +22,7 @@ import scala.concurrent.Future import scala.concurrent.duration._ // #metadata +// #metadata-client class FetchMetadata extends DocsSpecBase with TestcontainersKafkaLike with TryValues { @@ -52,6 +55,29 @@ class FetchMetadata extends DocsSpecBase with TestcontainersKafkaLike with TryVa topicsFuture.futureValue.response.get(topic) should not be Symbol("empty") } + "Get topic list" should "return result" in { + val topic1 = createTopic(1) + val group1 = createGroupId(1) + val partition0 = new TopicPartition(topic1, 0) + val consumerSettings = consumerDefaults.withGroupId(group1) + + awaitProduce(produce(topic1, 1 to 10)) + + // #metadata-client + val metadataClient = MetadataClient.create(consumerSettings, 1.second) + + val beginningOffsets = metadataClient + .getBeginningOffsets(Set(partition0)) + .futureValue + // #metadata-client + + beginningOffsets(partition0) shouldBe 0 + + // #metadata-client + metadataClient.stop() + // #metadata-client + } + "Get offsets" should "timeout fast" in { val consumerSettings = consumerDefaults .withGroupId(createGroupId()) From c2e7a86d176101be2eb7859b4c4b77a94a52e9af Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Fri, 22 Nov 2019 22:32:52 +0100 Subject: [PATCH 27/31] Fix docs --- docs/src/main/paradox/metadata-client.md | 4 ++-- .../test/java/docs/javadsl/MetadataClientTest.java | 12 ++++++------ .../src/test/scala/docs/scaladsl/FetchMetadata.scala | 12 ++++++------ 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/src/main/paradox/metadata-client.md b/docs/src/main/paradox/metadata-client.md index ae33fe88a..1ec07429d 100644 --- a/docs/src/main/paradox/metadata-client.md +++ b/docs/src/main/paradox/metadata-client.md @@ -36,7 +36,7 @@ Please consider to use a dedicated `KafkaConsumerActor` to create metadata clien ## Example Scala -: @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadata-client } +: @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadataClient } Java -: @@ snip [snip](/tests/src/test/java/docs/javadsl/FetchMetadataTest.java) { #metadata-client } +: @@ snip [snip](/tests/src/test/java/docs/javadsl/MetadataClientTest.java) { #metadataClient } diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 9d2d14a27..5717cfbd6 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -10,7 +10,7 @@ // package docs.javadsl; -// #metadata-client +// #metadataClient import akka.actor.ActorSystem; import akka.kafka.ConsumerSettings; import akka.kafka.javadsl.MetadataClient; @@ -19,7 +19,7 @@ import akka.stream.Materializer; import akka.testkit.javadsl.TestKit; import akka.util.Timeout; -// #metadata-client +// #metadataClient import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.hamcrest.core.IsInstanceOf; @@ -59,7 +59,7 @@ public MetadataClientTest() { public void shouldFetchBeginningOffsetsForGivenPartitions() { final String topic1 = createTopic(); final String group1 = createGroupId(); - // #metadata-client + // #metadataClient final TopicPartition partition = new TopicPartition(topic1, 0); final ConsumerSettings consumerSettings = consumerDefaults().withGroupId(group1); @@ -70,13 +70,13 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { final CompletionStage> response = metadataClient.getBeginningOffsets(partitions); final Map beginningOffsets = response.toCompletableFuture().join(); - // #metadata-client + // #metadataClient assertThat(beginningOffsets.get(partition), is(0L)); - // #metadata-client + // #metadataClient metadataClient.stop(); - // #metadata-client + // #metadataClient } @Test diff --git a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala index fb9a6100c..6c7ecf22b 100644 --- a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala +++ b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala @@ -11,7 +11,7 @@ import org.scalatest.TryValues import org.scalatest.time.{Seconds, Span} // #metadata -// #metadata-client +// #metadataClient import akka.actor.ActorRef import akka.kafka.{KafkaConsumerActor, Metadata} import akka.pattern.ask @@ -22,7 +22,7 @@ import scala.concurrent.Future import scala.concurrent.duration._ // #metadata -// #metadata-client +// #metadataClient class FetchMetadata extends DocsSpecBase with TestcontainersKafkaLike with TryValues { @@ -63,19 +63,19 @@ class FetchMetadata extends DocsSpecBase with TestcontainersKafkaLike with TryVa awaitProduce(produce(topic1, 1 to 10)) - // #metadata-client + // #metadataClient val metadataClient = MetadataClient.create(consumerSettings, 1.second) val beginningOffsets = metadataClient .getBeginningOffsets(Set(partition0)) .futureValue - // #metadata-client + // #metadataClient beginningOffsets(partition0) shouldBe 0 - // #metadata-client + // #metadataClient metadataClient.stop() - // #metadata-client + // #metadataClient } "Get offsets" should "timeout fast" in { From dc9cc474347b2e4ad20dff3fa2501d8807efc138 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 25 Nov 2019 21:50:00 +0100 Subject: [PATCH 28/31] Remove dumplicated comments --- core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala | 5 ----- tests/src/test/java/docs/javadsl/MetadataClientTest.java | 5 ----- 2 files changed, 10 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 7238a9f54..21bc984ed 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -3,11 +3,6 @@ * Copyright (C) 2016 - 2019 Lightbend Inc. */ -/* - * Copyright (C) 2014 - 2016 Softwaremill - * Copyright (C) 2016 - 2019 Lightbend Inc. - */ - package akka.kafka.javadsl import java.util.concurrent.{CompletionStage, Executor} diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 5717cfbd6..0814a081c 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -3,11 +3,6 @@ * Copyright (C) 2016 - 2019 Lightbend Inc. */ -/// * -// * Copyright (C) 2014 - 2016 Softwaremill -// * Copyright (C) 2016 - 2019 Lightbend Inc. -// */ -// package docs.javadsl; // #metadataClient From dde1d389dd72ac636483075baa790c667ff5279c Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 25 Nov 2019 21:56:38 +0100 Subject: [PATCH 29/31] Rename method MetadataClient#stop to close --- .../akka/kafka/javadsl/MetadataClient.scala | 4 ++-- .../akka/kafka/scaladsl/MetadataClient.scala | 2 +- docs/src/main/paradox/metadata-client.md | 2 +- .../java/docs/javadsl/MetadataClientTest.java | 16 ++++++++-------- .../akka/kafka/scaladsl/MetadataClientSpec.scala | 16 ++++++++-------- .../test/scala/docs/scaladsl/FetchMetadata.scala | 2 +- 6 files changed, 21 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala index 21bc984ed..c8117ccc4 100644 --- a/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala @@ -74,8 +74,8 @@ class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient .getCommittedOffset(partition) .toJava - def stop(): Unit = - metadataClient.stop() + def close(): Unit = + metadataClient.close() } object MetadataClient { diff --git a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala index ba439b102..5d88885ee 100644 --- a/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala +++ b/core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala @@ -85,7 +85,7 @@ class MetadataClient private (consumerActor: ActorRef, timeout: Timeout, managed case Failure(e) => Future.failed(e) }(ExecutionContexts.sameThreadExecutionContext) - def stop(): Unit = + def close(): Unit = if (managedActor) { consumerActor ! KafkaConsumerActor.Stop } diff --git a/docs/src/main/paradox/metadata-client.md b/docs/src/main/paradox/metadata-client.md index 1ec07429d..631a29c4f 100644 --- a/docs/src/main/paradox/metadata-client.md +++ b/docs/src/main/paradox/metadata-client.md @@ -7,7 +7,7 @@ project.description: Access Kafka consumer metadata by calling MetadataClient. To access the Kafka consumer metadata you need to create the `KafkaConsumerActor` as described in the @ref[Consumer documentation](consumer.md#sharing-the-kafkaconsumer-instance) pass it to `MetadataClient`'s factory method `create`. -Another approach to create metadata client is passing the `ConsumerSettings` and `ActorSystem` objects to factory method. Then the metadata client manages the internal actor and stops it when the `close` method is called. +Another approach to create metadata client is passing the `ConsumerSettings` and `ActorSystem` objects to the factory method. Then the metadata client manages the internal actor and stops it when the `close` method is called. The metadata the `MetadataClient` provides is documented in the @javadoc[Kafka Consumer API](org.apache.kafka.clients.consumer.KafkaConsumer). diff --git a/tests/src/test/java/docs/javadsl/MetadataClientTest.java b/tests/src/test/java/docs/javadsl/MetadataClientTest.java index 0814a081c..463f66ca5 100644 --- a/tests/src/test/java/docs/javadsl/MetadataClientTest.java +++ b/tests/src/test/java/docs/javadsl/MetadataClientTest.java @@ -70,7 +70,7 @@ public void shouldFetchBeginningOffsetsForGivenPartitions() { assertThat(beginningOffsets.get(partition), is(0L)); // #metadataClient - metadataClient.stop(); + metadataClient.close(); // #metadataClient } @@ -91,7 +91,7 @@ public void shouldFailInCaseOfAnExceptionDuringFetchBeginningOffsetsForNonExisti final CompletionStage> response = metadataClient.getBeginningOffsets(partitions); - metadataClient.stop(); + metadataClient.close(); response.toCompletableFuture().join(); } @@ -111,7 +111,7 @@ public void shouldFetchBeginningOffsetForGivenPartition() { assertThat(beginningOffset, is(0L)); - metadataClient.stop(); + metadataClient.close(); } @Test @@ -133,7 +133,7 @@ public void shouldFetchEndOffsetsForGivenPartitions() { assertThat(endOffsets.get(partition), is(10L)); - metadataClient.stop(); + metadataClient.close(); } @Test @@ -153,7 +153,7 @@ public void shouldFailInCaseOfAnExceptionDuringFetchEndOffsetsForNonExistingTopi final CompletionStage> response = metadataClient.getEndOffsets(Collections.singleton(nonExistingPartition)); - metadataClient.stop(); + metadataClient.close(); response.toCompletableFuture().join(); } @@ -174,7 +174,7 @@ public void shouldFetchEndOffsetForGivenPartition() { final Long endOffset = response.toCompletableFuture().join(); assertThat(endOffset, is(10L)); - metadataClient.stop(); + metadataClient.close(); } @Test @@ -202,7 +202,7 @@ public void shouldFetchTopicList() { assertThat(partitionsForTopic1, containsInAnyOrder(0, 1)); assertThat(partitionsForTopic2, containsInAnyOrder(0)); - metadataClient.stop(); + metadataClient.close(); } @Test @@ -225,7 +225,7 @@ public void shouldFetchPartitionsInfoForGivenTopic() { assertThat(partitions, containsInAnyOrder(0, 1)); - metadataClient.stop(); + metadataClient.close(); } @AfterClass diff --git a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala index 8cac70366..c67d96dcd 100644 --- a/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala +++ b/tests/src/test/scala/akka/kafka/scaladsl/MetadataClientSpec.scala @@ -31,7 +31,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { beginningOffsets(partition0) shouldBe 0 - metadataClient.stop() + metadataClient.close() } "fail in case of an exception during fetch beginning offsets for non-existing topics" in assertAllStagesStopped { @@ -46,7 +46,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { beginningOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - metadataClient.stop() + metadataClient.close() } "fetch beginning offset for given partition" in assertAllStagesStopped { @@ -65,7 +65,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { beginningOffset shouldBe 0 - metadataClient.stop() + metadataClient.close() } "fetch end offsets for given partitions" in assertAllStagesStopped { @@ -81,7 +81,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { endOffsets(partition0) shouldBe 10 - metadataClient.stop() + metadataClient.close() } "fail in case of an exception during fetch end offsets for non-existing topics" in assertAllStagesStopped { @@ -94,7 +94,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { endOffsetsFuture.failed.futureValue shouldBe a[org.apache.kafka.common.errors.InvalidTopicException] - metadataClient.stop() + metadataClient.close() } "fetch end offset for given partition" in assertAllStagesStopped { @@ -110,7 +110,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { endOffset shouldBe 10 - metadataClient.stop() + metadataClient.close() } "fetch list of topics" in assertAllStagesStopped { @@ -131,7 +131,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { topics(topic1).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic1 topics(topic2).leftSideValue.map(mapToTopicPartition) shouldBe expectedPartitionsForTopic2 - metadataClient.stop() + metadataClient.close() } "fetch partitions of given topic" in assertAllStagesStopped { @@ -147,7 +147,7 @@ class MetadataClientSpec extends SpecBase with TestcontainersKafkaLike { partitionsInfo.leftSideValue.map(_.partition()) shouldBe List(0, 1) - metadataClient.stop() + metadataClient.close() } } diff --git a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala index 6c7ecf22b..a922cf60d 100644 --- a/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala +++ b/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala @@ -74,7 +74,7 @@ class FetchMetadata extends DocsSpecBase with TestcontainersKafkaLike with TryVa beginningOffsets(partition0) shouldBe 0 // #metadataClient - metadataClient.stop() + metadataClient.close() // #metadataClient } From 845e1fff9a6f4f39ba66e075965031bea1db8015 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Mon, 25 Nov 2019 22:14:48 +0100 Subject: [PATCH 30/31] Response types in the reader's preferred language --- docs/src/main/paradox/metadata-client.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/src/main/paradox/metadata-client.md b/docs/src/main/paradox/metadata-client.md index 631a29c4f..baa280f8b 100644 --- a/docs/src/main/paradox/metadata-client.md +++ b/docs/src/main/paradox/metadata-client.md @@ -17,11 +17,11 @@ The supported metadata are | Metadata | Response type | |-------| ------- | -| Topics list | `Future[Map[String, List[PartitionInfo]]]` | -| Partitions | `Future[List[PartitionInfo]]` | -| Beginning offsets | `Future[Map[TopicPartition, Long]]` | -| End offsets | `Future[Map[TopicPartition, Long]]` | -| Committed offset | `Future[OffsetAndMetadata]` | +| Topics list | @scala[Future[Map[String, List[PartitionInfo]]]]@java[CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]]] | +| Partitions | @scala[Future[List[PartitionInfo]]]@java[CompletionStage[java.util.List[PartitionInfo]]] | +| Beginning offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | +| End offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | +| Committed offset | @scala[Future[OffsetAndMetadata]]@java[CompletionStage[OffsetAndMetadata]] | @@@ warning From 18ca14abbb796f96ce48acf4202694530a335776 Mon Sep 17 00:00:00 2001 From: Jacek Ewertowski Date: Tue, 26 Nov 2019 21:37:20 +0100 Subject: [PATCH 31/31] Docs for MetadataClient in consumer-metadata.md --- docs/src/main/paradox/consumer-metadata.md | 57 ++++++++++++++++++---- docs/src/main/paradox/consumer.md | 1 - docs/src/main/paradox/metadata-client.md | 42 ---------------- 3 files changed, 48 insertions(+), 52 deletions(-) delete mode 100644 docs/src/main/paradox/metadata-client.md diff --git a/docs/src/main/paradox/consumer-metadata.md b/docs/src/main/paradox/consumer-metadata.md index e7e16043d..a322402ae 100644 --- a/docs/src/main/paradox/consumer-metadata.md +++ b/docs/src/main/paradox/consumer-metadata.md @@ -3,11 +3,54 @@ project.description: Access Kafka consumer metadata by sending messages to the a --- # Consumer Metadata +## Metadata Client + +`MetadataClient` is a thin wrapper for `KafkaConsumerActor` hiding the ask calls and mapping to the correct response types. + +To access the Kafka consumer metadata you need to create the `KafkaConsumerActor` as described in the @ref[Consumer documentation](consumer.md#sharing-the-kafkaconsumer-instance) pass it to `MetadataClient`'s factory method `create`. + +Another approach to create metadata client is passing the `ConsumerSettings` and `ActorSystem` objects to the factory method. Then the metadata client manages the internal actor and stops it when the `close` method is called. + +The metadata the `MetadataClient` provides is documented in the @javadoc[Kafka Consumer API](org.apache.kafka.clients.consumer.KafkaConsumer). + +## Supported metadata by MetadataClient + +The supported metadata are + +| Metadata | Response type | +|-------| ------- | +| Topics list | @scala[Future[Map[String, List[PartitionInfo]]]]@java[CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]]] | +| Partitions | @scala[Future[List[PartitionInfo]]]@java[CompletionStage[java.util.List[PartitionInfo]]] | +| Beginning offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | +| End offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | +| Committed offset | @scala[Future[OffsetAndMetadata]]@java[CompletionStage[OffsetAndMetadata]] | + +@@@ warning + +Processing of these requests blocks the actor loop. The `KafkaConsumerActor` is configured to run on its own dispatcher, so just as the other remote calls to Kafka, the blocking happens within a designated thread pool. + +However, calling these during consuming might affect performance and even cause timeouts in extreme cases. + +Please consider to use a dedicated `KafkaConsumerActor` to create metadata client requests against. + +@@@ + +Example: + +Scala +: @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadataClient } + +Java +: @@ snip [snip](/tests/src/test/java/docs/javadsl/MetadataClientTest.java) { #metadataClient } + + +## Accessing metadata using KafkaConsumerActor + To access the Kafka consumer metadata you need to create the `KafkaConsumerActor` as described in the @ref[Consumer documentation](consumer.md#sharing-the-kafkaconsumer-instance) and send messages from `Metadata` (@scaladoc[API](akka.kafka.Metadata$)) to it. The metadata the Kafka Consumer provides is documented in the @javadoc[Kafka Consumer API](org.apache.kafka.clients.consumer.KafkaConsumer). -## Supported metadata +## Supported metadata by KafkaConsumerActor The supported metadata are @@ -21,18 +64,14 @@ The supported metadata are | GetCommittedOffset | CommittedOffset | These requests are blocking within the Kafka client library up to a timeout configured by `metadata-request-timeout` or `ConsumerSettings.withMetadataRequestTimeout` respectively. - -@@@ warning -Processing of these requests blocks the actor loop. The `KafkaConsumerActor` is configured to run on its own dispatcher, so just as the other remote calls to Kafka, the blocking happens within a designated thread pool. - -However, calling these during consuming might affect performance and even cause timeouts in extreme cases. +@@@ warning -Please consider to use a dedicated `KafkaConsumerActor` to run metadata requests against. +Accessing the Kafka consumer metadata using the `KafkaConsumerActor` is not a recommended approach. It is reasonable only when you need to perform a request `GetOffsetsForTimes` which is not supported by the `MetadataClient` yet. -@@@ +@@@ -## Example +Example: Scala : @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadata } diff --git a/docs/src/main/paradox/consumer.md b/docs/src/main/paradox/consumer.md index fe18f54cf..8e0aa8c2e 100644 --- a/docs/src/main/paradox/consumer.md +++ b/docs/src/main/paradox/consumer.md @@ -350,6 +350,5 @@ Java * [subscription](subscription.md) * [metadata](consumer-metadata.md) -* [metadata-client](metadata-client.md) @@@ diff --git a/docs/src/main/paradox/metadata-client.md b/docs/src/main/paradox/metadata-client.md deleted file mode 100644 index baa280f8b..000000000 --- a/docs/src/main/paradox/metadata-client.md +++ /dev/null @@ -1,42 +0,0 @@ ---- -project.description: Access Kafka consumer metadata by calling MetadataClient. ---- -# Metadata Client - -`MetadataClient` is a thin wrapper for `KafkaConsumerActor` hiding the ask calls and mapping to the correct response types. - -To access the Kafka consumer metadata you need to create the `KafkaConsumerActor` as described in the @ref[Consumer documentation](consumer.md#sharing-the-kafkaconsumer-instance) pass it to `MetadataClient`'s factory method `create`. - -Another approach to create metadata client is passing the `ConsumerSettings` and `ActorSystem` objects to the factory method. Then the metadata client manages the internal actor and stops it when the `close` method is called. - -The metadata the `MetadataClient` provides is documented in the @javadoc[Kafka Consumer API](org.apache.kafka.clients.consumer.KafkaConsumer). - -## Supported metadata - -The supported metadata are - -| Metadata | Response type | -|-------| ------- | -| Topics list | @scala[Future[Map[String, List[PartitionInfo]]]]@java[CompletionStage[java.util.Map[java.lang.String, java.util.List[PartitionInfo]]]] | -| Partitions | @scala[Future[List[PartitionInfo]]]@java[CompletionStage[java.util.List[PartitionInfo]]] | -| Beginning offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | -| End offsets | @scala[Future[Map[TopicPartition, Long]]]@java[CompletionStage[java.util.Map[TopicPartition, java.lang.Long]]] | -| Committed offset | @scala[Future[OffsetAndMetadata]]@java[CompletionStage[OffsetAndMetadata]] | - -@@@ warning - -Processing of these requests blocks the actor loop. The `KafkaConsumerActor` is configured to run on its own dispatcher, so just as the other remote calls to Kafka, the blocking happens within a designated thread pool. - -However, calling these during consuming might affect performance and even cause timeouts in extreme cases. - -Please consider to use a dedicated `KafkaConsumerActor` to create metadata client requests against. - -@@@ - -## Example - -Scala -: @@ snip [snip](/tests/src/test/scala/docs/scaladsl/FetchMetadata.scala) { #metadataClient } - -Java -: @@ snip [snip](/tests/src/test/java/docs/javadsl/MetadataClientTest.java) { #metadataClient }