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

    https://github.com/apache/spark/pull/3798#discussion_r23726607
  
    --- Diff: 
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala 
---
    @@ -144,4 +150,116 @@ 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
    +   * @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],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] = {
    +    val parts = batch.zipWithIndex.map { case (o, i) =>
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, 
o.untilOffset, o.host, o.port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, parts, messageHandler)
    +  }
    +
    +  /**
    +   * This DOES NOT guarantee that side-effects of an action will see each 
message exactly once.
    --- End diff --
    
    Docs must first start with what the operation does, not what it does not do 
:) You have to first specify what the operation does (see createStream in this 
class). I suggest somethig like this. 
    ```
    Create an input stream that pulls messages exactly-once from Kafka. 
Compared to `createStream`, stream created by this will guarantee that each 
record from Kafka is received by the application only once (despite failures, 
etc.).
    
    Note: This does not guarantee that any output operation will push each 
record only once. To ensure end-to-end exactly-once semantics (that is, 
receiving exactly once and outputting exactly once), you have to ensure that 
the output operation is either idempotent or transactional. See the programming 
guide for more details.
    
    ```



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