Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -462,32 +462,24 @@ class KafkaTestUtils(
server.logManager.cleanupLogs()
}

private def getOffsets(topics: Set[String], offsetSpec: OffsetSpec): Map[TopicPartition, Long] = {
val listOffsetsParams = adminClient.describeTopics(topics.asJava).all().get().asScala
.flatMap { topicDescription =>
topicDescription._2.partitions().asScala.map { topicPartitionInfo =>
new TopicPartition(topicDescription._1, topicPartitionInfo.partition())
}
}.map(_ -> offsetSpec).toMap.asJava
val partitionOffsets = adminClient.listOffsets(listOffsetsParams).all().get().asScala
.map(result => result._1 -> result._2.offset()).toMap
partitionOffsets
}

def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = {
val kc = new KafkaConsumer[String, String](consumerConfiguration)
logInfo("Created consumer to get earliest offsets")
kc.subscribe(topics.asJavaCollection)
kc.poll(0)
val partitions = kc.assignment()
kc.pause(partitions)
kc.seekToBeginning(partitions)
val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap
kc.close()
logInfo("Closed consumer to get earliest offsets")
offsets
getOffsets(topics, OffsetSpec.earliest())
}

def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = {
val kc = new KafkaConsumer[String, String](consumerConfiguration)
logInfo("Created consumer to get latest offsets")
kc.subscribe(topics.asJavaCollection)
kc.poll(0)
val partitions = kc.assignment()
kc.pause(partitions)
kc.seekToEnd(partitions)
val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap
kc.close()
logInfo("Closed consumer to get latest offsets")
offsets
getOffsets(topics, OffsetSpec.latest())
}

def listConsumerGroups(): ListConsumerGroupsResult = {
Expand Down Expand Up @@ -559,17 +551,6 @@ class KafkaTestUtils(
}
}

private def consumerConfiguration: Properties = {
val props = new Properties()
props.put("bootstrap.servers", brokerAddress)
props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt)
props.put("value.deserializer", classOf[StringDeserializer].getName)
props.put("key.deserializer", classOf[StringDeserializer].getName)
props.put("enable.auto.commit", "false")
setAuthenticationConfigIfNeeded(props)
props
}

private def setAuthenticationConfigIfNeeded(props: Properties): Unit = {
if (secure) {
val jaasParams = KafkaTokenUtil.getKeytabJaasParams(
Expand Down