Home | About | Sematext search-lucene.com search-hadoop.com
NEW: Monitor These Apps!
elasticsearch, apache solr, apache hbase, hadoop, redis, casssandra, amazon cloudwatch, mysql, memcached, apache kafka, apache zookeeper, apache storm, ubuntu, centOS, red hat, debian, puppet labs, java, senseiDB
 Search Hadoop and all its subprojects:

Switch to Plain View
Kafka >> mail # dev >> Random Partitioning Issue


+
Joe Stein 2013-09-14, 05:11
+
Joel Koshy 2013-09-14, 12:17
+
Joe Stein 2013-09-14, 18:19
+
Jun Rao 2013-09-15, 03:15
+
Jay Kreps 2013-09-15, 15:37
+
Jay Kreps 2013-09-15, 15:45
+
Joel Koshy 2013-09-17, 17:19
+
Jay Kreps 2013-09-17, 17:41
+
Joe Stein 2013-09-18, 23:23
+
Jun Rao 2013-09-23, 00:57
+
Joe Stein 2013-09-27, 16:24
+
Jun Rao 2013-09-27, 16:46
+
Joe Stein 2013-09-27, 16:54
+
Jun Rao 2013-09-27, 17:10
+
Joe Stein 2013-09-27, 17:31
+
Jun Rao 2013-09-28, 04:12
+
Guozhang Wang 2013-09-29, 04:52
+
Jun Rao 2013-09-29, 16:15
+
Joe Stein 2013-10-01, 05:22
+
Jun Rao 2013-10-01, 15:27
Copy link to this message
-
Re: Random Partitioning Issue
agreed, lets hold off until after 0.8

I will update the JIRA ticket I created with your feedback and options we
can discuss it there and then deal with changes in 0.8.1 or 0.9 or such.

I will update the FAQ (should have time tomorrow unless someone else gets
to it first) I think we should have it in there at the least, yes?

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/
On Tue, Oct 1, 2013 at 11:26 AM, Jun Rao <[EMAIL PROTECTED]> wrote:

> This proposal still doesn't address the following fundamental issue: The
> random partitioner cannot select a random and AVAILABLE partition.
>
> So, we have the following two choices.
>
> 1. Stick with the current partitioner api.
> Then, we have to pick one way to do random partitioning (when key is null).
> The current behavior may not be very intuitive, but is one of the possible
> behaviors in 0.7.
>
> 2. Change the partitioner api so that we can (1) be aware of available
> partitions and (2) have pluggable partitioners for doing random
> distribution.
>
> Option (2) is probably the right approach. However, it's a non-trivial
> change. So, I am not sure if it should be done in 0.8 or not.
>
> Thanks,
>
> Jun
>
>
>
> On Mon, Sep 30, 2013 at 10:21 PM, Joe Stein <[EMAIL PROTECTED]> wrote:
>
> > How about making UUID.randomUUID.toString() the default in KeyedMessage
> > instead of null if not supplied
> >
> > def this(topic: String, message: V) = this(topic,
> > UUID.randomUUID.toString(),
> > message)
> >
> > and if you want the random refresh behavior then pass in "*" on the
> > KeyedMessage construction which we can then later check for in
> > defaulteventhandler
> >
> >  val partition =
> >       if(key =="*") {
> >
> > we then throw NPE if key == null in KeyedMessage like we do topic
> >
> > I believe any null flow control logic is something to shy away from
> >
> > if this is wrong or too much or still not the best solution we could also
> > hold over and just put this in the FAQ with the JIRA and let people know
> > when they run into this and want to randomize in development / testing
> and
> > in many production situations where the producer count is not large
> enough
> > then they have to pass in their own continuous random key... if we can
> get
> > a consensus for what we want to-do with minimal changes then I think it
> is
> > important for 0.8 otherwise wait.
> >
> > On Sun, Sep 29, 2013 at 12:14 PM, Jun Rao <[EMAIL PROTECTED]> wrote:
> >
> > > The main issue is that if we do that, when key is null, we can only
> > select
> > > a random partition, but not a random and available partition, without
> > > changing the partitioner api. Being able to do the latter is important
> in
> > > my opinion. For example, a user may choose the replication factor of a
> > > topic to be 1. If a broker is down, it's much better to select
> partitions
> > > on other brokers for producing than losing messages.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > >
> > > On Sat, Sep 28, 2013 at 9:51 PM, Guozhang Wang <[EMAIL PROTECTED]>
> > wrote:
> > >
> > > > I think Joe's suggesting that we can remove the checking logic for
> > > > key==null in DefaultEventHandler, and do that in partitioner.
> > > >
> > > > One thing about this idea is any customized partitioner also has to
> > > > consider key == null case then.
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Fri, Sep 27, 2013 at 9:12 PM, Jun Rao <[EMAIL PROTECTED]> wrote:
> > > >
> > > > > We have the following code in DefaultEventHandler:
> > > > >
> > > > >     val partition =
> > > > >       if(key == null) {
> > > > >         // If the key is null, we don't really need a partitioner
> > > > >         // So we look up in the send partition cache for the topic
> to
> > > > > decide the target partition
> > > > >         val id = sendPartitionPerTopicCache.get(topic)

 
+
Jun Rao 2013-10-01, 16:32
NEW: Monitor These Apps!
elasticsearch, apache solr, apache hbase, hadoop, redis, casssandra, amazon cloudwatch, mysql, memcached, apache kafka, apache zookeeper, apache storm, ubuntu, centOS, red hat, debian, puppet labs, java, senseiDB