On Tue, Mar 4, 2014 at 10:33 AM, Guozhang Wang <wangg...@gmail.com> wrote:
> Hi David, > > When you (re)-start the producer/consumer, the broker list is used for > bootstrap, so it should guarantee that some of the brokers listed is alive > when the client starts. When you migrate from 1,2,3 to 4,5,6 (in fact, in > this case it is better to just keep the broker id but just replace the > host/port), and then bounce the clients, then it is better to also change > the bootstrap broker list since clients are memoryless. Does this make > sense? > > Sure, when we were saying broker {1,2,3} and {4,5,6} I made the mistake of assuming we were referring to names/ips and ports symbolically. That list will have to be updated and correct between process restarts however it's implemented. > I agree that using ZK the clients can get ride of the bootstrap broker > list, but that would add the dependency of ZooKeeper on the clients code > base. So we decided to remove the ZK dependency from the client instead, > and if people do want to use ZK for bootstraping, they can always add a > simple script which reads the broker list from ZK and give that to the > broker config. > This is effectively what I'm talking about. A 'wrapper' that discovers the broker id list from zookeeper and then retrieves the ip:ports of those brokers from zookeeper where a wrapper is just more code and not really a script. I should mention that I'm not using the scala/jvm client libs. I'm on my own in golang land and I've had to implement a high-level consumer on my own which has been an adventure. > Guozhang > > > On Mon, Mar 3, 2014 at 4:16 PM, David Birdsong <david.birds...@gmail.com > >wrote: > > > On Mon, Mar 3, 2014 at 4:00 PM, Guozhang Wang <wangg...@gmail.com> > wrote: > > > > > Hi Chris, > > > > > > In 0.9 we will have just one "broker list", i.e. the list of brokers > read > > > from the config file will be updated during bootstraping and all the > > future > > > metadata refresh operations. This feature should lift this limit you > are > > > describing, for example, if your broker list in config is {1,2,3}, and > > > later on you extend the cluster to {1,2,3,4,5,6}, then now you can shut > > > down 1,2,3 all at once. > > > > > > > But if you producer or consumer ever restarts and only knows about > {1,2,3}, > > the problem still exists no? > > > > This is why I bootstrap off of zk and expect to have to maintain an > > accurate list of zk nodes to all processes. > > > > > > > > > > Guozhang > > > > > > > > > On Mon, Mar 3, 2014 at 1:35 PM, Christofer Hedbrandh < > > > christo...@knewton.com > > > > wrote: > > > > > > > Thanks again Guozhang. > > > > > > > > There are still some details here that are unclear to me, but if > what I > > > am > > > > describing is not a bug, do you think it is reasonable to file this > as > > a > > > > feature request? We agree that it is not ideal to have to keep "at > > least > > > > one broker in the list is alive", when replacing a cluster i.e. > > migrating > > > > from one set of brokers to another? > > > > > > > > Christofer > > > > > > > > > > > > > > > > On Wed, Feb 26, 2014 at 9:16 PM, Guozhang Wang <wangg...@gmail.com> > > > wrote: > > > > > > > > > kafka-preferred-replica-election.sh is only used to move leaders > > > between > > > > > brokers, as long as the broker in the broker.metadata.list, i.e. > the > > > > second > > > > > broker list I mentioned in previous email is still alive then the > > > > producer > > > > > can learn the leader change from it. > > > > > > > > > > In terms of broker discovery, I think it depends on how you > "define" > > > the > > > > > future. For example, originally there are 3 brokers 1,2,3, and you > > > start > > > > > the producer with metadata list = {1,2,3}, and later on another > three > > > > > brokers 4,5,6 are added, the producer can still find these newly > > added > > > > > brokers. It is just that if all the brokers in the metadata list, > > i.e. > > > > > 1,2,3 are gone, then the producer will not be able to refresh its > > > > metadata. > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > On Wed, Feb 26, 2014 at 11:04 AM, Christofer Hedbrandh < > > > > > christo...@knewton.com> wrote: > > > > > > > > > > > Thanks for your response Guozhang. > > > > > > > > > > > > I did make sure that new meta data is fetched before taking out > the > > > old > > > > > > broker. I set the topic.metadata.refresh.interval.ms to > something > > > very > > > > > > low, > > > > > > and I confirm in the producer log that new meta data is actually > > > > fetched, > > > > > > after the new broker is brought up, and before the old broker is > > > taken > > > > > > down. Does this not mean that the dynamic current brokers list > > would > > > > hold > > > > > > the new broker at this point? > > > > > > > > > > > > If you are saying that the dynamic current brokers list is never > > used > > > > for > > > > > > fetching meta data, this does not explain how the producer does > NOT > > > > fail > > > > > > when kafka-preferred-replica-election.sh makes the new broker > > become > > > > the > > > > > > leader. > > > > > > > > > > > > Lastly, if broker discovery is not a producer feature in 0.8.0 > > > Release, > > > > > and > > > > > > I have to "make sure at least one broker in the list is alive > > during > > > > the > > > > > > rolling bounce", is this a feature you are considering for future > > > > > versions? > > > > > > > > > > > > > > > > > > > > > > > > On Wed, Feb 26, 2014 at 12:04 PM, Guozhang Wang < > > wangg...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > Hello Chris, > > > > > > > > > > > > > > The broker.metadata.list, once read in at start up time, will > not > > > be > > > > > > > changed. In other words, during the life time of a producer it > > has > > > > two > > > > > > > lists of brokers: > > > > > > > > > > > > > > 1. The current brokers in the cluster that is returned in the > > > > metadata > > > > > > > request response, which is dynamic > > > > > > > > > > > > > > 2. The broker list that is used for bootstraping, this is read > > from > > > > > > > broker.metadata.list and is fixed. This list could for example > > be a > > > > VIP > > > > > > and > > > > > > > a hardware load balancer behind it will distribute the metadata > > > > > requests > > > > > > to > > > > > > > the brokers. > > > > > > > > > > > > > > So in your case, the metadata list only has broker B, and once > it > > > is > > > > > > taken > > > > > > > out and the producer failed to send message to it and hence > tries > > > to > > > > > > > refresh its metadata, it has no broker to go. > > > > > > > > > > > > > > Therefore, when you are trying to do a rolling bounce of the > > > cluster > > > > > to, > > > > > > > for example, do a in-place upgrade, you need to make sure at > > least > > > > one > > > > > > > broker in the list is alive during the rolling bounce. > > > > > > > > > > > > > > Hope this helps. > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Wed, Feb 26, 2014 at 8:19 AM, Christofer Hedbrandh < > > > > > > > christo...@knewton.com> wrote: > > > > > > > > > > > > > > > Hi all, > > > > > > > > > > > > > > > > I ran into a problem with the Kafka producer when attempting > to > > > > > replace > > > > > > > all > > > > > > > > the nodes in a 0.8.0 Beta1 Release Kafka cluster, with 0.8.0 > > > > Release > > > > > > > nodes. > > > > > > > > I started a producer/consumer test program to measure the > > > clusters > > > > > > > > performance during the process, I added new brokers, I ran > > > > > > > > kafka-reassign-partitions.sh, and I removed the old brokers. > > > When I > > > > > > > removed > > > > > > > > the old brokers my producer failed. > > > > > > > > > > > > > > > > The simplest scenario that I could come up with where I still > > see > > > > > this > > > > > > > > behavior is this. Using version 0.8.0 Release, we have a 1 > > > > partition > > > > > > > topic > > > > > > > > with 2 replicas on 2 brokers, broker A and broker B. Broker A > > is > > > > > taken > > > > > > > > down. A producer is started with only broker B in the > > > > > > > metadata.broker.list. > > > > > > > > Broker A is brought back up. We let > > > > > > > > topic.metadata.refresh.interval.msamount of time pass. > Broker B > > > is > > > > > > > > taken down, and we get > > > > > > > > kafka.common.FailedToSendMessageException after all the > (many) > > > > > retries > > > > > > > have > > > > > > > > failed. > > > > > > > > > > > > > > > > During my experimentation I have made sure that the producer > > > > fetches > > > > > > meta > > > > > > > > data before the old broker is taken down. And I have made > sure > > > that > > > > > > > enough > > > > > > > > retries with enough backoff time were used for the producer > to > > > not > > > > > give > > > > > > > up > > > > > > > > prematurely. > > > > > > > > > > > > > > > > The documentation for the producer config > metadata.broker.list > > > > > suggests > > > > > > > to > > > > > > > > me that this list of brokers is only used at startup. "This > is > > > for > > > > > > > > bootstrapping and the producer will only use it for getting > > > > metadata > > > > > > > > (topics, partitions and replicas)". And when I read about > > > > > > > > topic.metadata.refresh.interval.ms and retry.backoff.ms I > > learn > > > > that > > > > > > > meta > > > > > > > > data is indeed fetched at later times. Based on this > > > > documentation, I > > > > > > > make > > > > > > > > the assumption that the producer would learn about any new > > > brokers > > > > > when > > > > > > > new > > > > > > > > meta data is fetched. > > > > > > > > > > > > > > > > I also want to point out that the cluster seems to work just > > fine > > > > > > during > > > > > > > > this process, it only seems to be a problem with the > producer. > > > > > Between > > > > > > > all > > > > > > > > these steps I run kafka-list-topic.sh, I try the console > > producer > > > > and > > > > > > > > consumer, and everything is as expected. > > > > > > > > > > > > > > > > Also I found another interesting thing when experimenting > with > > > > > running > > > > > > > > kafka-preferred-replica-election.sh before taking down the > old > > > > > broker. > > > > > > > This > > > > > > > > script only causes any changes when the leader and the > > preferred > > > > > > replica > > > > > > > > are different. In the scenario when they are in fact > different, > > > and > > > > > the > > > > > > > new > > > > > > > > broker takes the role of leader from the old broker, the > > producer > > > > > does > > > > > > > NOT > > > > > > > > fail. This makes me think that perhaps the producer only > keeps > > > meta > > > > > > data > > > > > > > > about topic leaders and not all replicas, as the > documentation > > > > > suggests > > > > > > > to > > > > > > > > me. > > > > > > > > > > > > > > > > It is clear that I am making a lot of assumptions here, and I > > am > > > > > > > relatively > > > > > > > > new to Kafka so I could very well me missing something > > important. > > > > The > > > > > > > way I > > > > > > > > see it, there are a few possibilities. > > > > > > > > > > > > > > > > 1. Broker discovery is a supposed producer feature, and it > has > > a > > > > bug. > > > > > > > > 2. Broker discovery is not a producer feature, in which case > I > > > > think > > > > > > many > > > > > > > > people might benefit from a clearer documentation. > > > > > > > > 3. I am doing something dumb e.g. forgetting about some > > important > > > > > > > > configuration. > > > > > > > > > > > > > > > > Please let me know what you make of this. > > > > > > > > > > > > > > > > Thanks, > > > > > > > > Christofer Hedbrandh > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > -- > > > -- Guozhang > > > > > > > > > -- > -- Guozhang >