Github user koeninger commented on the pull request:
https://github.com/apache/spark/pull/3798#issuecomment-72754442
Like patrick said, I really don't see any reason not to just expose
KafkaRDD. You can still hide its constructor without making a superflous
abstract class, and you can still make another subclass of KafkaRDD later
if you need to.
Even if you don't want the static createRDD method to return a KafkaRDD, we
can just take the "with HasOffsetRanges" off and people who care about
getting to the offsets can cast it (they'll have to cast it for the stream
case anyway)
On Tue, Feb 3, 2015 at 4:30 PM, Tathagata Das <[email protected]>
wrote:
> @koeninger <https://github.com/koeninger> Thank you very much for all the
> changes. They are looking good. Unfortunately the issue with createRDD
> returning RDD[] with OffsetRange (i.e., the issue that @pwendell
> <https://github.com/pwendell> raised) could be a problem in the future in
> terms of binary compatibility. Basically, we have not used such things in
> the rest of Spark to keep things as Java-friendly and binary compatible as
> possible. Also in the generated Java doc this looks scary. So here is an
> alternate suggestion that I am trying to implement on top of your PR
> branch. How about this. We effectively combine KafkaRDD and
HasOffsetRanges
> into a abstract class.
>
> abstract class KafkaRDD[T] (val offsetRanges: Array[OffsetRanges], sc:
SparkContext)
> extends RDD[T](sc, Nil)
>
> private[kafka]
> class KafkaRDDImpl[K, V, KD, VD, R] extends KafkaRDD[R] {
> ...
> }
>
> KafkaUtils.createRDD(...simple one without messageHandler...):
KafkaRDD[(K, V)] = {
> // return KafkaRDDImpl[K, V, KD, VD, (K, V)]
> }
>
> KafkaUtils.createRDD(...simple one WITH messageHandler...): KafkaRDD[R] =
{
> // return KafkaRDDImpl[K, V, KD, VD, R]
> }
>
>
> Advantages
>
> - No binary compatibility issues
> - Easy to read from Java
> - KafkaRDD implementation and constructor all hidden as before
> - Can still extend KafkaRDD to expose more methods in future.
>
> What do you think?
>
> â
> Reply to this email directly or view it on GitHub
> <https://github.com/apache/spark/pull/3798#issuecomment-72749763>.
>
---
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]