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

    https://github.com/apache/spark/pull/3798#discussion_r24018460
  
    --- 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 = {
    --- End diff --
    
    There is definitely a bytecode difference, try running diff on the class 
files.  It statically guarantees you can call .hasOffsetRanges on the thing 
returned from createRDD.  Without it, you'd have to cast at runtime.  If you 
add e.g. a .chzburger method to KafkaRDD, you wont be able to call it without 
asInstanceOf.  If you then made a Chzburger interface, implement it on 
KafkaRDD, change the return type to "RDD with HasOffsetranges with Chzburger", 
you would.  I hear your concern about binary compatibility.
    
    As far as exposing KafkaRDD instead... that's the way I originally designed 
things.
    
    The current design is the result of a compromise between TD's desire as a 
maintainer to hide as much as possible, and my desire as a user to expose 
what's necessary to get my job done.  You can usually tell it's a good 
compromise if no one is happy :)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to