Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
Kafka, mail # user - New Producer Public API


Copy link to this message
-
Re: New Producer Public API
Jay Kreps 2014-01-31, 01:28
Joel--

Yeah we could theoretically retain a neutered Partitioner interface that
only had access to the byte[] key not the original object (which we no
longer have). Ideologically most partitioning should really happen based on
the byte[] not the original object to retain multi-language compatibility,
but sometimes the object is useful.

I kind of think this is one of those things where doing either A or B is
better than doing A and B both just for clarity.

-Jay
On Thu, Jan 30, 2014 at 5:15 PM, Joel Koshy <[EMAIL PROTECTED]> wrote:

> Does it preclude those various implementations? i.e., it could become
> a producer config:
> default.partitioner.strategy="minimize-connections"/"roundrobin" - and
> so on; and implement those partitioners internally in the producer.
> Not as clear as a .class config, but it accomplishes the same effect
> no?
>
> On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <[EMAIL PROTECTED]> wrote:
> > One downside to the 1A proposal is that without a Partitioner interface
> we
> > can't really package up and provide common partitioner implementations.
> > Example of these would be
> > 1. HashPartitioner - The default hash partitioning
> > 2. RoundRobinPartitioner - Just round-robins over partitions
> > 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
> > number of nodes you need to connect maintain TCP connections to.
> > 4. RangePartitioner - User provides break points that align partitions to
> > key ranges
> > 5. LocalityPartitioner - Prefer nodes on the same rack. This would be
> nice
> > for stream-processing use cases that read from one topic and write to
> > another. We would have to include rack information in our metadata.
> >
> > Having this kind of functionality included is actually kind of nice.
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <[EMAIL PROTECTED]> wrote:
> >
> >> Clark and all,
> >>
> >> I thought a little bit about the serialization question. Here are the
> >> options I see and the pros and cons I can think of. I'd love to hear
> >> people's preferences if you have a strong one.
> >>
> >> One important consideration is that however the producer works will also
> >> need to be how the new consumer works (which we hope to write next).
> That
> >> is if you put objects in, you should get objects out. So we need to
> think
> >> through both sides.
> >>
> >> Options:
> >>
> >> Option 0: What is in the existing scala code and the java code I
> >> posted--Serializer and Partitioner plugin provided by the user via
> config.
> >> Partitioner has a sane default, but Serializer needs to be specified in
> >> config.
> >>
> >> Pros: How it works today in the scala code.
> >> Cons: You have to adapt your serialization library of choice to our
> >> interfaces. The reflective class loading means typo in the serializer
> name
> >> give odd errors. Likewise there is little type safety--the
> ProducerRecord
> >> takes Object and any type errors between the object provided and the
> >> serializer give occurs at runtime.
> >>
> >> Option 1: No plugins
> >>
> >> This would mean byte[] key, byte[] value, and partitioning done by
> client
> >> by passing in a partition *number* directly.
> >>
> >> The problem with this is that it is tricky to compute the partition
> >> correctly and probably most people won't. We could add a getCluster()
> >> method to return the Cluster instance you should use for partitioning.
> But
> >> I suspect people would be lazy and not use that and instead hard-code
> >> partitions which would break if partitions were added or they hard
> coded it
> >> wrong. In my experience 3 partitioning strategies cover like 99% of
> cases
> >> so not having a default implementation for this makes the common case
> >> harder. Left to their own devices people will use bad hash functions and
> >> get weird results.
> >>
> >> Option 1A: Alternatively we could partition by the key using the
> existing
> >> default partitioning strategy which only uses the byte[] anyway but