-
Notifications
You must be signed in to change notification settings - Fork 387
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
- Loading branch information
Showing
6 changed files
with
668 additions
and
12 deletions.
There are no files selected for viewing
97 changes: 97 additions & 0 deletions
97
core/src/main/scala/akka/kafka/javadsl/MetadataClient.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
/* | ||
* Copyright (C) 2014 - 2016 Softwaremill <http://softwaremill.com> | ||
* Copyright (C) 2016 - 2019 Lightbend Inc. <http://www.lightbend.com> | ||
*/ | ||
|
||
package akka.kafka.javadsl | ||
|
||
import java.util.concurrent.{CompletionStage, Executor} | ||
|
||
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._ | ||
import scala.collection.compat._ | ||
import scala.collection.JavaConverters._ | ||
import scala.concurrent.ExecutionContextExecutor | ||
|
||
class MetadataClient private (metadataClient: akka.kafka.scaladsl.MetadataClient) { | ||
|
||
def getBeginningOffsets[K, V]( | ||
partitions: java.util.Set[TopicPartition] | ||
): CompletionStage[java.util.Map[TopicPartition, java.lang.Long]] = | ||
metadataClient | ||
.getBeginningOffsets(partitions.asScala.toSet) | ||
.map { beginningOffsets => | ||
beginningOffsets.view.mapValues(Long.box).toMap.asJava | ||
}(ExecutionContexts.sameThreadExecutionContext) | ||
.toJava | ||
|
||
def getBeginningOffsetForPartition[K, V](partition: TopicPartition): CompletionStage[java.lang.Long] = | ||
metadataClient | ||
.getBeginningOffsetForPartition(partition) | ||
.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 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 getPartitionsFor(topic: java.lang.String): CompletionStage[java.util.List[PartitionInfo]] = | ||
metadataClient | ||
.getPartitionsFor(topic) | ||
.map { partitionsInfo => | ||
partitionsInfo.asJava | ||
}(ExecutionContexts.sameThreadExecutionContext) | ||
.toJava | ||
|
||
def getCommittedOffset(partition: TopicPartition): CompletionStage[OffsetAndMetadata] = | ||
metadataClient | ||
.getCommittedOffset(partition) | ||
.toJava | ||
|
||
def close(): Unit = | ||
metadataClient.close() | ||
} | ||
|
||
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) | ||
} | ||
} |
106 changes: 106 additions & 0 deletions
106
core/src/main/scala/akka/kafka/scaladsl/MetadataClient.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,106 @@ | ||
/* | ||
* Copyright (C) 2014 - 2016 Softwaremill <http://softwaremill.com> | ||
* Copyright (C) 2016 - 2019 Lightbend Inc. <http://www.lightbend.com> | ||
*/ | ||
|
||
package akka.kafka.scaladsl | ||
|
||
import akka.actor.{ActorRef, ActorSystem} | ||
import akka.dispatch.ExecutionContexts | ||
import akka.kafka.Metadata.{ | ||
BeginningOffsets, | ||
CommittedOffset, | ||
EndOffsets, | ||
GetBeginningOffsets, | ||
GetCommittedOffset, | ||
GetEndOffsets, | ||
GetPartitionsFor, | ||
ListTopics, | ||
PartitionsFor, | ||
Topics | ||
} | ||
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} | ||
import scala.util.{Failure, Success} | ||
|
||
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) | ||
.mapTo[BeginningOffsets] | ||
.map(_.response) | ||
.flatMap { | ||
case Success(res) => Future.successful(res) | ||
case Failure(e) => Future.failed(e) | ||
}(ExecutionContexts.sameThreadExecutionContext) | ||
|
||
def getBeginningOffsetForPartition(partition: TopicPartition): Future[Long] = | ||
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 listTopics(): Future[Map[String, List[PartitionInfo]]] = | ||
(consumerActor ? ListTopics)(timeout) | ||
.mapTo[Topics] | ||
.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 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 close(): Unit = | ||
if (managedActor) { | ||
consumerActor ! KafkaConsumerActor.Stop | ||
} | ||
} | ||
|
||
object MetadataClient { | ||
|
||
def create(consumerActor: ActorRef, timeout: Timeout)(implicit ec: ExecutionContext): MetadataClient = | ||
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, true) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.