Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Xiao Bo
I also tried changing the broker id to a new id: 5 and still got many many similar warnings and no data generated in log.dir. [2014-01-21 18:00:00,923] WARN [KafkaApi-5] Fetch request with correlation id 1528423082 from client ReplicaFetcherThread-0-1 on partition [co,9] failed due to Partition [c

Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Xiao Bo
Before the broker 1 died, the replicas and leaders were even distribued on the 5 node cluster. And the previous replication factor were 3. 2014/1/22 Guozhang Wang > Do you still have the results of list-topics before broker 1 is dead? From > the current result it seems the replication factor i

Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Guozhang Wang
Do you still have the results of list-topics before broker 1 is dead? From the current result it seems the replication factor is 2 not 3. Guozhang On Tue, Jan 21, 2014 at 9:26 PM, Xiao Bo wrote: > Yes, the topic name is countinfo, Maybe in the log it generates a short > name automatically. >

Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Xiao Bo
Yes, the topic name is countinfo, Maybe in the log it generates a short name automatically. 2014/1/22 Guozhang Wang > Hello, > > What is your topic name? From the log it seems to be "co", but from > list-topic it is "countinfo". > > Guozhang > > > On Tue, Jan 21, 2014 at 8:24 PM, Xiao Bo wrot

Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Xiao Bo
Thanks for reply. Yes, the broker id is the same. We have tried just restarting the broker, but after 1 day there were still no partition files generated in the log.dir and by list-topic I could not see broker-1 back. Any other way can I add it back ? As I said in last mail, the topic seems having

Re: Problem on re-adding a breakdown broker back

2014-01-21 Thread Guozhang Wang
Hello, What is your topic name? From the log it seems to be "co", but from list-topic it is "countinfo". Guozhang On Tue, Jan 21, 2014 at 8:24 PM, Xiao Bo wrote: > Hi guys and jun, > > We have a problem when adding a breakdown broker back to the cluster. Hope > you guys have some solution fo

Re: replica.lag.time.max.ms or doco bugged?

2014-01-21 Thread Guozhang Wang
Hello Sven, After you killed the broker, do you keep sending data to Kafka? If not then the ISR may not change in 0.8 Guozhang On Tue, Jan 21, 2014 at 4:44 PM, Sven Stark wrote: > Howdy! > > I've got a three node 0.8 cluster with replication factor three and > replica.lag.time.max.ms set to 20

Re: [Consumer code not working][Kafka Newbie]

2014-01-21 Thread Jun Rao
Will manual offset commit work? Thanks, Jun On Tue, Jan 21, 2014 at 8:11 AM, Abhishek Bhattacharjee < abhishek.bhattacharje...@gmail.com> wrote: > Thanks for the reply. > Actually in my use-case I need to control the offsets my self so should I > use SimpleConsumer instead of Group Consumers ?

Re: Possibly leaking socket on ReplicaFetcherThread

2014-01-21 Thread Jun Rao
Hmm, without knowing the client ip, it's hard to tell whether those are from replication fetcher threads or not. Are most of those connections in established mode? Thanks, Jun On Tue, Jan 21, 2014 at 8:06 AM, Ahmy Yulrizka wrote: > this is the the line i copied on lsof > > ... > java 118

Problem on re-adding a breakdown broker back

2014-01-21 Thread Xiao Bo
Hi guys and jun, We have a problem when adding a breakdown broker back to the cluster. Hope you guys have some solution for it. A cluster of 5 brokers(id=0~4) of kafka 0.8.0 was running for log aggregation . Because of some issues of the disk, a broker(id=1) was down. We spent one week to repl

replica.lag.time.max.ms or doco bugged?

2014-01-21 Thread Sven Stark
Howdy! I've got a three node 0.8 cluster with replication factor three and replica.lag.time.max.ms set to 2000. Typical list-topic output looks like topic: user.publish partition: 0 leader: 1 replicas: 2,1,3 isr: 1,2,3 I noticed though that when I kill (kill -9 PID) one of the brokers the list

Re: Patterns for message failure handling with Kafka

2014-01-21 Thread Jason Rosenberg
So, I think there are 2 different types of errors you mention. The first is data-dependent (e.g. it's corrupt or some such). So, there's no reason to block consumption of other messages that are likely to be successful, while the data-dependent one won't fix itself no matter times you retry. So,

Re: 0.7 -> 0.8 Protocol Upgrade in production environments

2014-01-21 Thread Jason Rosenberg
In my case, we just rolled out a separate 0.8 cluster, and migrated producers to it over time (took several weeks to get everything updated to the new cluster). In the transition, we had consumers running for both clusters. Once no traffic was flowing on the old cluster, we then shut down the 0.7

Patterns for message failure handling with Kafka

2014-01-21 Thread Jim
I'm looking at message delivery patterns for Kafka consumers and wanted to get people's thoughts on the following problem: The objective is to ensure processing of individual messages with as much certainty as possible for "at least once guarantees". I'm looking to have a kafka consumer pull n me

Re: 0.7 -> 0.8 Protocol Upgrade in production environments

2014-01-21 Thread François Langelier
Hi, I'm also in the process to upgrade from 0.7 to 0.8. WIth the informations I found, your best friend is here : https://cwiki.apache.org/confluence/display/KAFKA/Migrating+from+0.7+to+0.8 It's a migration tool from 0.7 to 0.8. You will have to install 0.8 while you still have you 0.7 running. W

0.7 -> 0.8 Protocol Upgrade in production environments

2014-01-21 Thread Clark Breyman
I'm curious what the recommended best practice is for migrating a production environment with replication from 0.7 to 0.8 given the protocol upgrade. Some specific questions I have are: a) Is it possible to mix 0.7 and 0.8 servers for a given partition during the migration? b) If we can't mix ser

Re: [Consumer code not working][Kafka Newbie]

2014-01-21 Thread Abhishek Bhattacharjee
Thanks for the reply. Actually in my use-case I need to control the offsets my self so should I use SimpleConsumer instead of Group Consumers ? On Tue, Jan 21, 2014 at 9:38 PM, Jun Rao wrote: > "auto.offset.reset" is only used when offsets don't exist in ZK. In your > case, the consumer likely

Re: [Consumer code not working][Kafka Newbie]

2014-01-21 Thread Jun Rao
"auto.offset.reset" is only used when offsets don't exist in ZK. In your case, the consumer likely already committed the offsets to ZK. So, after restarting, the consumer will resume from where it left off, instead of re-getting everything again. This is the expected behavior during normal operatio

Re: Possibly leaking socket on ReplicaFetcherThread

2014-01-21 Thread Ahmy Yulrizka
this is the the line i copied on lsof ... java 11818 kafka 98u sock0,7 0t0 615628183 can't identify protocol java 11818 kafka 99u IPv4 615077352 0t0 TCP somedomain.com:9092->121-123-123-123.someprovider.net:37547(CLOSE_WAIT)

Re: [Consumer code not working][Kafka Newbie]

2014-01-21 Thread Abhishek Bhattacharjee
I read the faqs and I added "auto.offset.reset" property in the configuration setting of storm. Then I ran my producer code and then I ran my consumer code when I ran the consumer code it printed all the messages that were created by producer but after stopping the consumer when I ran it again it d

Re: Question on MessageSizeTooLargeException

2014-01-21 Thread Jun Rao
Great. Please open a jira and attach your patch there. Thanks, Jun On Mon, Jan 20, 2014 at 10:37 PM, Bae, Jae Hyeon wrote: > Nope, just packaging for Netflix cloud environment. > > The first one is, producer discovery(metadata.broker.list) is integrated > with Netflix Eureka. > The second one

Re: Kafka server occure java.nio.BufferUnderflowException

2014-01-21 Thread Jun Rao
This means that the broker somehow gets a corrupted TopicMetadata request. Are you using the java/scala client? Thanks, Jun On Mon, Jan 20, 2014 at 10:30 PM, 陈小军 wrote: > Hi all, >in my kafka cluster, it alway ocuure following erro logs > > [2014-01-21 15:25:24,462] ERROR Closing socket f

Re: Possibly leaking socket on ReplicaFetcherThread

2014-01-21 Thread Jun Rao
What mode are those sockets in (established, closed, etc)? Also, from the ip, could you tell whether those sockets are from the client or from the replica fetcher in the brokers. Thanks, Jun On Tue, Jan 21, 2014 at 3:29 AM, Ahmy Yulrizka wrote: > We are running 3 kafka nodes, which servers 4

Possibly leaking socket on ReplicaFetcherThread

2014-01-21 Thread Ahmy Yulrizka
We are running 3 kafka nodes, which servers 4 partition. We have been experiencing weird behavior during network outage. we had been experiencing twice in the last couple of days. the previous one took down all of the cluster. while this one only 2 out of 3 survive. and 1 node became the leader of