Github user dibbhatt commented on a diff in the pull request:
https://github.com/apache/spark/pull/3798#discussion_r24015154
--- Diff:
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
---
@@ -0,0 +1,344 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.util.control.NonFatal
+import scala.util.Random
+import scala.collection.mutable.ArrayBuffer
+import java.util.Properties
+import kafka.api._
+import kafka.common.{ErrorMapping, OffsetMetadataAndError,
TopicAndPartition}
+import kafka.consumer.{ConsumerConfig, SimpleConsumer}
+import org.apache.spark.SparkException
+
+/**
+ * Convenience methods for interacting with a Kafka cluster.
+ * @param kafkaParams Kafka <a
href="http://kafka.apache.org/documentation.html#configuration">
+ * configuration parameters</a>.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set with
Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form
+ */
+private[spark]
+class KafkaCluster(val kafkaParams: Map[String, String]) extends
Serializable {
+ import KafkaCluster.{Err, LeaderOffset}
+
+ val seedBrokers: Array[(String, Int)] =
+ kafkaParams.get("metadata.broker.list")
+ .orElse(kafkaParams.get("bootstrap.servers"))
+ .getOrElse(throw new SparkException("Must specify
metadata.broker.list or bootstrap.servers"))
+ .split(",").map { hp =>
+ val hpa = hp.split(":")
+ (hpa(0), hpa(1).toInt)
+ }
+
+ // ConsumerConfig isn't serializable
+ @transient private var _config: ConsumerConfig = null
+
+ def config: ConsumerConfig = this.synchronized {
+ if (_config == null) {
+ _config = KafkaCluster.consumerConfig(kafkaParams)
+ }
+ _config
+ }
+
+ def connect(host: String, port: Int): SimpleConsumer =
+ new SimpleConsumer(host, port, config.socketTimeoutMs,
+ config.socketReceiveBufferBytes, config.clientId)
+
+ def connectLeader(topic: String, partition: Int): Either[Err,
SimpleConsumer] =
+ findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
+
+ // Metadata api
+ // scalastyle:off
+ //
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
+ // scalastyle:on
+
+ def findLeader(topic: String, partition: Int): Either[Err, (String,
Int)] = {
+ val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
+ 0, config.clientId, Seq(topic))
+ val errs = new Err
+ withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
+ val resp: TopicMetadataResponse = consumer.send(req)
+ resp.topicsMetadata.find(_.topic == topic).flatMap { tm:
TopicMetadata =>
+ tm.partitionsMetadata.find(_.partitionId == partition)
+ }.foreach { pm: PartitionMetadata =>
+ pm.leader.foreach { leader =>
+ return Right((leader.host, leader.port))
+ }
+ }
+ }
+ Left(errs)
+ }
+
+ def findLeaders(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
+ val topics = topicAndPartitions.map(_.topic)
+ val response = getPartitionMetadata(topics).right
+ val answer = response.flatMap { tms: Set[TopicMetadata] =>
+ val leaderMap = tms.flatMap { tm: TopicMetadata =>
+ tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
+ val tp = TopicAndPartition(tm.topic, pm.partitionId)
+ if (topicAndPartitions(tp)) {
+ pm.leader.map { l =>
+ tp -> (l.host -> l.port)
+ }
+ } else {
+ None
+ }
+ }
+ }.toMap
+
+ if (leaderMap.keys.size == topicAndPartitions.size) {
+ Right(leaderMap)
+ } else {
+ val missing = topicAndPartitions.diff(leaderMap.keySet)
+ val err = new Err
+ err.append(new SparkException(s"Couldn't find leaders for
${missing}"))
+ Left(err)
+ }
+ }
+ answer
+ }
+
+ def getPartitions(topics: Set[String]): Either[Err,
Set[TopicAndPartition]] = {
+ getPartitionMetadata(topics).right.map { r =>
+ r.flatMap { tm: TopicMetadata =>
+ tm.partitionsMetadata.map { pm: PartitionMetadata =>
+ TopicAndPartition(tm.topic, pm.partitionId)
+ }
+ }
+ }
+ }
+
+ def getPartitionMetadata(topics: Set[String]): Either[Err,
Set[TopicMetadata]] = {
+ val req = TopicMetadataRequest(
+ TopicMetadataRequest.CurrentVersion, 0, config.clientId,
topics.toSeq)
+ val errs = new Err
+ withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
+ val resp: TopicMetadataResponse = consumer.send(req)
+ // error codes here indicate missing / just created topic,
+ // repeating on a different broker wont be useful
+ return Right(resp.topicsMetadata.toSet)
+ }
+ Left(errs)
+ }
+
+ // Leader offset api
+ // scalastyle:off
+ //
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
+ // scalastyle:on
+
+ def getLatestLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
+ getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
+
+ def getEarliestLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
+ getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
+
+ def getLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition],
+ before: Long
+ ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
+ getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
+ r.map { kv =>
+ // mapValues isnt serializable, see SI-7005
+ kv._1 -> kv._2.head
+ }
+ }
+ }
+
+ private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
+ m.groupBy(_._2).map { kv =>
+ kv._1 -> kv._2.keys.toSeq
+ }
+
+ def getLeaderOffsets(
+ topicAndPartitions: Set[TopicAndPartition],
+ before: Long,
+ maxNumOffsets: Int
+ ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
+ findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
+ val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] =
flip(tpToLeader)
+ val leaders = leaderToTp.keys
+ var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
+ val errs = new Err
+ withBrokers(leaders, errs) { consumer =>
+ val partitionsToGetOffsets: Seq[TopicAndPartition] =
+ leaderToTp((consumer.host, consumer.port))
+ val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
+ tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
+ }.toMap
+ val req = OffsetRequest(reqMap)
+ val resp = consumer.getOffsetsBefore(req)
+ val respMap = resp.partitionErrorAndOffsets
+ partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
+ respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
+ if (por.error == ErrorMapping.NoError) {
+ if (por.offsets.nonEmpty) {
+ result += tp -> por.offsets.map { off =>
+ LeaderOffset(consumer.host, consumer.port, off)
+ }
+ } else {
+ errs.append(new SparkException(
+ s"Empty offsets for ${tp}, is ${before} before log
beginning?"))
+ }
+ } else {
+ errs.append(ErrorMapping.exceptionFor(por.error))
+ }
+ }
+ }
+ if (result.keys.size == topicAndPartitions.size) {
+ return Right(result)
+ }
+ }
+ val missing = topicAndPartitions.diff(result.keySet)
+ errs.append(new SparkException(s"Couldn't find leader offsets for
${missing}"))
+ Left(errs)
+ }
+ }
+
+ // Consumer offset api
+ // scalastyle:off
+ //
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
+ // scalastyle:on
+
+ def getConsumerOffsets(
+ groupId: String,
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, Long]] = {
+ getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
+ r.map { kv =>
+ kv._1 -> kv._2.offset
+ }
+ }
+ }
+
+ def getConsumerOffsetMetadata(
+ groupId: String,
+ topicAndPartitions: Set[TopicAndPartition]
+ ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
+ var result = Map[TopicAndPartition, OffsetMetadataAndError]()
+ val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
+ val errs = new Err
+ withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
+ val resp = consumer.fetchOffsets(req)
+ val respMap = resp.requestInfo
+ val needed = topicAndPartitions.diff(result.keySet)
+ needed.foreach { tp: TopicAndPartition =>
+ respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
+ if (ome.error == ErrorMapping.NoError) {
+ result += tp -> ome
+ } else {
+ errs.append(ErrorMapping.exceptionFor(ome.error))
+ }
+ }
+ }
+ if (result.keys.size == topicAndPartitions.size) {
+ return Right(result)
+ }
+ }
+ val missing = topicAndPartitions.diff(result.keySet)
+ errs.append(new SparkException(s"Couldn't find consumer offsets for
${missing}"))
+ Left(errs)
+ }
+
+ def setConsumerOffsets(
+ groupId: String,
+ offsets: Map[TopicAndPartition, Long]
+ ): Either[Err, Map[TopicAndPartition, Short]] = {
+ setConsumerOffsetMetadata(groupId, offsets.map { kv =>
+ kv._1 -> OffsetMetadataAndError(kv._2)
+ })
+ }
+
+ def setConsumerOffsetMetadata(
+ groupId: String,
+ metadata: Map[TopicAndPartition, OffsetMetadataAndError]
+ ): Either[Err, Map[TopicAndPartition, Short]] = {
+ var result = Map[TopicAndPartition, Short]()
+ val req = OffsetCommitRequest(groupId, metadata)
--- End diff --
This is what mentioned in the Kafka-1841
"To make this really backward compatible, we have to make sure that version
0 of OffsetCommitRequest only writes to ZK. However, this doesn't quite work
together with OffsetFetchRequest since in 0.8.2, it only has one version and it
always reads offsets from Kafka. To address this issue, I bumped up the version
of OffsetFetchRequest in 0.8.2 (with same wire protocol). Then, version 0 of
OffsetFetchRequest will read from ZK and version 1 of OffsetFetchRequest will
read from Kafka. This works as long as people are only using released final
version. However, since this introduces an incompatible change of
OffsetFetchRequest in 0.8.2-beta and trunk, this will create problems for
people (assuming that they are using this api) who have a deployment of
0.8.2-beta and want to upgrade to 0.8.2 final, or a deployment from trunk and
want to upgrade to a later version of trunk in the future. In either case, the
upgrade of the broker will cause the old client to behave differently and
incorre
ctly."
If you want to avoid such confusion, you can using ZK Client API directly
to commit offsets to ZK instead using OffsetCommitRequest..that would have
ported to all kafka versions seamlessly.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]