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

    https://github.com/apache/spark/pull/3798#discussion_r23990592
  
    --- Diff: 
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala 
---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, 
mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, 
o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, 
leaders, 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 offsetRanges 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, 
l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, 
messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in 
transformations
    --- End diff --
    
    For the top level doc here isn't it something like:
    
    ```
    Returns a Kafka stream that computes a specific offset range for each
    partition, then reads those offsets directly from Kafka without the use of 
receivers.
    
    Because this stream deterministically assigns offset ranges to specific 
batches, it can
    support "exactly once" semantics (as defined in the programming guide). 
Specifically,
    a streaming program experiences task failures or restarts from a job 
failure, output
    batches appear as if each record was ingested and processed exactly once.
    ```


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