Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23822726
  
    --- Diff: 
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala 
---
    @@ -144,4 +149,182 @@ object KafkaUtils {
         createStream[K, V, U, T](
           jssc.ssc, kafkaParams.toMap, 
Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
       }
    +
    +  /** A batch-oriented interface for consuming from Kafka.
    +   * Starting and ending offsets are specified in advance,
    +   * so that you can control exactly-once semantics.
    +   * @param sc SparkContext object
    +   * @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.
    +   * @param batch Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, 
mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, 
o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, 
o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, 
messageHandler)
    +  }
    +
    +  /** A batch-oriented interface for consuming from Kafka.
    +   * Starting and ending offsets are specified in advance,
    +   * so that you can control exactly-once semantics.
    +   * @param sc SparkContext object
    +   * @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.
    +   * @param batch Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the 
desired type
    +   */
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, 
l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, 
o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * Compared to `createStream`, the stream created by this can guarantee 
that each message
    --- End diff --
    
    A scala doc has to start with this method does, not with a comparison with 
another method. Then for understanding this method, they will have to read 
another method first. That's not good. In future if we deprecate the other 
method, then even worse. Its best to say what this method does independent of 
anything else, and as a second para (for those who want to understand more), 
compare with the other method. 


---
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]

Reply via email to