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

    https://github.com/apache/spark/pull/3798#discussion_r24030347
  
    --- 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
    +   * (as opposed to output actions) exactly once, even in most failure 
situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets 
yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after 
failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For 
interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output 
operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, 
receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output 
operation is
    +   * idempotent, or transactionally store offsets with the output. See the 
programming guide for
    +   * more details.
    +   *
    +   * @param ssc StreamingContext 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 messageHandler function for translating each message into the 
desired type
    +   * @param fromOffsets per-topic/partition Kafka offsets defining the 
(inclusive)
    +   *  starting point of the stream
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    I think createOffsetRangeStream conveys that it returns a stream based on
    offset ranges, and from which you can obtain offset ranges.
    
    If the thought is that it eventually becomes the recommended method, just
    naming these createStream may be the best option.  The parameter list
    doesn't conflict with any of the existing overloads.
    
    On Tue, Feb 3, 2015 at 12:57 PM, Tathagata Das <[email protected]>
    wrote:
    
    > In
    > 
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r24028643>:
    >
    > > +   * exactly once. To ensure end-to-end exactly-once semantics (that 
is, receiving exactly once and
    > > +   * outputting exactly once), you have to either ensure that the 
output operation is
    > > +   * idempotent, or transactionally store offsets with the output. See 
the programming guide for
    > > +   * more details.
    > > +   *
    > > +   * @param ssc StreamingContext 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 messageHandler function for translating each message into 
the desired type
    > > +   * @param fromOffsets per-topic/partition Kafka offsets defining the 
(inclusive)
    > > +   *  starting point of the stream
    > > +   */
    > > +  @Experimental
    > > +  def createNewStream[
    >
    > CreateOffsetRangeStream seems like you have give a OffsetRange to create
    > one, whereas you can create a stream without explicitly specifying the
    > offsetrange.
    > I have found all the names to either becoming too long, or not be
    > conveying that it is a receiver-less stream. That includes
    > createDirectStream. But createDirectStream does not convey much, but is
    > definitely not overpromising.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r24028643>.
    >


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