Have you looked at #3 in http://kafka.apache.org/faq.html?

Thanks,

Jun


On Thu, Jun 13, 2013 at 6:41 AM, Alexandre Rodrigues <
alexan...@blismedia.com> wrote:

> I think I know what's happening:
>
> I tried to run both brokers and ZK on the same machine and it worked. I
> also attempted to do the same but with a ZK node on other machine and it
> also worked.
>
> My guess is something related with ports. All the machines are on EC2 and
> there might be something related with the security group. I am going to run
> the first setup with all open doors and see how it goes. If the ports are
> really the problem, shouldn't this kind of problem be logged somewhere?
>
>
>
>
> On 13 June 2013 12:13, Alexandre Rodrigues <alexan...@blismedia.com>
> wrote:
>
> > I've tried the console producer, so I will assume that's not related with
> > the producer. I keep seeing the same entries in the producer from time to
> > time:
> >
> > [2013-06-13 11:04:00,670] WARN Error while fetching metadata
> > [{TopicMetadata for topic C ->
> > No partition metadata for topic C due to
> > kafka.common.LeaderNotAvailableException}] for topic [C]: class
> > kafka.common.LeaderNotAvailableException
> >  (kafka.producer.BrokerPartitionInfo)
> >
> > Which I assume is when the consumer asks a broker who is responsible for
> a
> > partition. I might be wrong but I think one of the brokers doesn't know,
> so
> > I thought it might be related with the ZK where partition leader
> elections
> > happen (I think).
> >
> > I was using a 3 node ZK 3.3.5. First I've deleted the snapshot of all the
> > ZK nodes and started one without ensemble. Cleaned the brokers dataDir
> and
> > restarted them against that solo ZK node. The problem still the same. I
> > though it could be because of the ZK version, so I've decided to start a
> ZK
> > instance using the jar that ships with Kafka and the problem remains.
> >
> > I am not sure if this is a real bug or just anything that might be
> missing
> > to me. I don't know if it helps, but all the trials were run without any
> > kind of consumer (which should be OK, no?)
> >
> > Thanks,
> > Alex
> >
> >
> >
> >
> > On 13 June 2013 10:15, Alexandre Rodrigues <alexan...@blismedia.com
> >wrote:
> >
> >> Hi Jun,
> >>
> >> I was using the 0.8 branch with 2 commits behind but now I am using the
> >> latest with the same issue. 3 topics A,B,C, created automatically with
> >> replication factor of 2 and partitions 2. 2 brokers (0 and 1).
> >>
> >> List of topics in zookeeper is the following:
> >>
> >> topic: A  partition: 0    leader: 1       replicas: 1,0   isr: 1
> >> topic: A  partition: 1    leader: 0       replicas: 0,1   isr: 0,1
> >> topic: B partition: 0    leader: 0       replicas: 0,1   isr: 0,1
> >> topic: B partition: 1    leader: 1       replicas: 1,0   isr: 1
> >> topic: C      partition: 0    leader: 1       replicas: 1,0   isr: 1
> >> topic: C      partition: 1    leader: 0       replicas: 0,1   isr: 0,1
> >>
> >>
> >> *Broker 1*
> >>
> >> This was the one I've started first. This works well and writes messages
> >> to the disk.
> >> In the state-change.log I have got no errors, just trace rows:
> >>
> >> [2013-06-13 08:51:33,505] TRACE Broker 1 cached leader info
> >>
> (LeaderAndIsrInfo:(Leader:0,ISR:0,1,LeaderEpoch:0,ControllerEpoch:1),ReplicationFactor:2),AllReplicas:0,1)
> >> for partition [C,1] in response to UpdateMetadata request sent by
> >> controller 1 epoch 1 with correlation id 10 (state.change.logger)
> >> [2013-06-13 08:51:33,506] TRACE Controller 1 epoch 1 received response
> >> correlationId 10 for a request sent to broker 1 (state.change.logger)
> >> [2013-06-13 08:51:33,509] TRACE Controller 1 epoch 1 changed state of
> >> replica 0 for partition [C,1] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,510] TRACE Controller 1 epoch 1 changed state of
> >> replica 1 for partition [C,0] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,511] TRACE Controller 1 epoch 1 changed state of
> >> replica 0 for partition [B,1] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,511] TRACE Controller 1 epoch 1 changed state of
> >> replica 0 for partition [C,0] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,512] TRACE Controller 1 epoch 1 changed state of
> >> replica 0 for partition [B,0] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,512] TRACE Controller 1 epoch 1 changed state of
> >> replica 1 for partition [B,0] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,513] TRACE Controller 1 epoch 1 changed state of
> >> replica 1 for partition [B,1] to OnlineReplica (state.change.logger)
> >> [2013-06-13 08:51:33,513] TRACE Controller 1 epoch 1 changed state of
> >> replica 1 for partition [C,1] to OnlineReplica (state.change.logger)
> >>
> >> $ du -sh /mnt/kafka-logs/*
> >>
> >> 4.0K    /mnt/kafka-logs/replication-offset-checkpoint
> >> 163M    /mnt/kafka-logs/A-0
> >> 4.0K    /mnt/kafka-logs/A-1
> >> 4.0K    /mnt/kafka-logs/B-0
> >> 90M     /mnt/kafka-logs/B-1
> >> 16K     /mnt/kafka-logs/C-0
> >> 4.0K    /mnt/kafka-logs/C-1
> >>
> >>
> >>
> >> *Broker 0*
> >> *
> >> *
> >> Configuration is the same as Broker #1, with different broker.id. This
> >> doesn't write to the disk. The /mnt/kafka-logs is empty without any
> file.
> >>
> >> Logging a non-stopping stream of:
> >>
> >> [2013-06-13 09:08:53,814] WARN [KafkaApi-0] Produce request with
> >> correlation id 735114 from client  on partition [A,1] failed due to
> >> Partition [request,1] doesn't exist on 0 (kafka.server.KafkaApis)
> >> [2013-06-13 09:08:53,815] WARN [KafkaApi-0] Produce request with
> >> correlation id 519064 from client  on partition [B,0] failed due to
> >> Partition [response,0] doesn't exist on 0 (kafka.server.KafkaApis)
> >>  [2013-06-13 09:08:53,815] WARN [KafkaApi-0] Produce request with
> >> correlation id 735118 from client  on partition [A,1] failed due to
> >> Partition [request,1] doesn't exist on 0 (kafka.server.KafkaApis)
> >> [2013-06-13 09:08:53,815] WARN [KafkaApi-0] Produce request with
> >> correlation id 519068 from client  on partition [B,0] failed due to
> >> Partition [response,0] doesn't exist on 0 (kafka.server.KafkaApis)
> >> ...
> >>
> >> *Server Configuration *
> >> *
> >> *
> >> port=9092
> >> num.network.threads=2
> >> num.io.threads=2
> >> socket.send.buffer.bytes=1048576
> >> socket.receive.buffer.bytes=1048576
> >> socket.request.max.bytes=104857600
> >> log.dir=/mnt/kafka-logs
> >> auto.create.topics.enable=true
> >> default.replication.factor=2
> >> num.partitions=2
> >> log.flush.interval.messages=10000
> >> log.flush.interval.ms=1000
> >> log.retention.hours=168
> >> log.segment.bytes=536870912
> >> log.cleanup.interval.mins=1
> >> zookeeper.connect=xxx1:2181,xxx2:2181,xxx3:2181
> >> zookeeper.connection.timeout.ms=1000000
> >> kafka.metrics.polling.interval.secs=5
> >> kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter
> >> kafka.csv.metrics.dir=/mnt/kafka_metrics
> >> kafka.csv.metrics.reporter.enabled=false
> >>
> >>
> >> I can't understand why doesn't broker0 doesn't act like a leader in
> their
> >> partitions nor receive replicated data from the broker1. To eliminate
> the
> >> possibility of the problem being from the producer, I will run similar
> >> tests with the console producer.
> >>
> >> Alex
> >>
> >>
> >> On 13 June 2013 04:57, Jun Rao <jun...@gmail.com> wrote:
> >>
> >>> Any error in state-change.log? Also, are you using the latest code in
> the
> >>> 0.8 branch?
> >>>
> >>> Thanks,
> >>>
> >>> Jun
> >>>
> >>>
> >>> On Wed, Jun 12, 2013 at 9:27 AM, Alexandre Rodrigues <
> >>> alexan...@blismedia.com> wrote:
> >>>
> >>> > Hi Jun,
> >>> >
> >>> > Thanks for your prompt answer. The producer yields those errors in
> the
> >>> > beginning, so I think the topic metadata refresh has nothing to do
> >>> with it.
> >>> >
> >>> > The problem is one of the brokers isn't leader on any partition
> >>> assigned to
> >>> > it and because topics were created with a replication factor of 1,
> the
> >>> > producer will never connect to that broker at all. What I don't
> >>> understand
> >>> > is why doesn't the broker assume the lead of those partitions.
> >>> >
> >>> > I deleted all the topics and tried now with a replication factor of
> two
> >>> >
> >>> > topic: A  partition: 0    leader: 1       replicas: 1,0   isr: 1
> >>> > topic: A  partition: 1    leader: 0       replicas: 0,1   isr: 0,1
> >>> > topic: B partition: 0    leader: 0       replicas: 0,1   isr: 0,1
> >>> > topic: B partition: 1    leader: 1       replicas: 1,0   isr: 1
> >>> > topic: C      partition: 0    leader: 1       replicas: 1,0   isr: 1
> >>> > topic: C      partition: 1    leader: 0       replicas: 0,1   isr:
> 0,1
> >>> >
> >>> >
> >>> > Now producer doesn't yield errors. However, one of the brokers (
> >>> broker 0 )
> >>> > generates lots of lines like this:
> >>> >
> >>> > [2013-06-12 16:19:41,805] WARN [KafkaApi-0] Produce request with
> >>> > correlation id 404999 from client  on partition [B,0] failed due to
> >>> > Partition [B,0] doesn't exist on 0 (kafka.server.KafkaApis)
> >>> >
> >>> > There should be a replica there, so I don't know why it complains
> about
> >>> > that message.
> >>> >
> >>> > Have you ever found anything like this?
> >>> >
> >>> >
> >>> >
> >>> > On 12 June 2013 16:27, Jun Rao <jun...@gmail.com> wrote:
> >>> >
> >>> > > If the leaders exist in both brokers, the producer should be able
> to
> >>> > > connect to both of them, assuming you don't provide any key when
> >>> sending
> >>> > > the data. Could you try restarting the producer? If there has been
> >>> broker
> >>> > > failures, it may take topic.metadata.refresh.interval.ms for the
> >>> > producer
> >>> > > to pick up the newly available partitions (see
> >>> > > http://kafka.apache.org/08/configuration.html for details).
> >>> > >
> >>> > > Thanks,
> >>> > >
> >>> > > Jun
> >>> > >
> >>> > >
> >>> > > On Wed, Jun 12, 2013 at 8:01 AM, Alexandre Rodrigues <
> >>> > > alexan...@blismedia.com> wrote:
> >>> > >
> >>> > > > Hi,
> >>> > > >
> >>> > > > I have a Kafka 0.8 cluster with two nodes connected to three ZKs,
> >>> with
> >>> > > the
> >>> > > > same configuration but the brokerId (one is 0 and the other 1). I
> >>> > created
> >>> > > > three topics A, B and C with 4 partitions and a replication
> factor
> >>> of
> >>> > 1.
> >>> > > My
> >>> > > > idea was to have 2 partitions per topic in each broker. However,
> >>> when I
> >>> > > > connect a producer, I can't have both brokers to write at the
> same
> >>> time
> >>> > > and
> >>> > > > I don't know what's going on.
> >>> > > >
> >>> > > > My server.config has the following entries:
> >>> > > >
> >>> > > > auto.create.topics.enable=true
> >>> > > > num.partitions=2
> >>> > > >
> >>> > > >
> >>> > > > When I run bin/kafka-list-topic.sh --zookeeper localhost:2181   I
> >>> get
> >>> > the
> >>> > > > following partition leader assignments:
> >>> > > >
> >>> > > > topic: A  partition: 0    leader: 1       replicas: 1     isr: 1
> >>> > > > topic: A  partition: 1    leader: 0       replicas: 0     isr: 0
> >>> > > > topic: A  partition: 2    leader: 1       replicas: 1     isr: 1
> >>> > > > topic: A  partition: 3    leader: 0       replicas: 0     isr: 0
> >>> > > > topic: B partition: 0    leader: 0       replicas: 0     isr: 0
> >>> > > > topic: B partition: 1    leader: 1       replicas: 1     isr: 1
> >>> > > > topic: B partition: 2    leader: 0       replicas: 0     isr: 0
> >>> > > > topic: B partition: 3    leader: 1       replicas: 1     isr: 1
> >>> > > > topic: C      partition: 0    leader: 0       replicas: 0
> isr:
> >>> 0
> >>> > > > topic: C      partition: 1    leader: 1       replicas: 1
> isr:
> >>> 1
> >>> > > > topic: C      partition: 2    leader: 0       replicas: 0
> isr:
> >>> 0
> >>> > > > topic: C      partition: 3    leader: 1       replicas: 1
> isr:
> >>> 1
> >>> > > >
> >>> > > >
> >>> > > > I've forced reassignment using the kafka-reassign-partitions tool
> >>> with
> >>> > > the
> >>> > > > following JSON:
> >>> > > >
> >>> > > > {"partitions":  [
> >>> > > >    {"topic": "A", "partition": 1, "replicas": [0] },
> >>> > > >    {"topic": "A", "partition": 3, "replicas": [0] },
> >>> > > >    {"topic": "A", "partition": 0, "replicas": [1] },
> >>> > > >    {"topic": "A", "partition": 2, "replicas": [1] },
> >>> > > >    {"topic": "B", "partition": 1, "replicas": [0] },
> >>> > > >    {"topic": "B", "partition": 3, "replicas": [0] },
> >>> > > >    {"topic": "B", "partition": 0, "replicas": [1] },
> >>> > > >    {"topic": "B", "partition": 2, "replicas": [1] },
> >>> > > >    {"topic": "C", "partition": 0, "replicas": [0] },
> >>> > > >    {"topic": "C", "partition": 1, "replicas": [1] },
> >>> > > >    {"topic": "C", "partition": 2, "replicas": [0] },
> >>> > > >    {"topic": "C", "partition": 3, "replicas": [1] }
> >>> > > > ]}
> >>> > > >
> >>> > > > After reassignment, I've restarted producer and nothing worked.
> >>> I've
> >>> > > tried
> >>> > > > also to restart both brokers and producer and nothing.
> >>> > > >
> >>> > > > The producer contains this logs:
> >>> > > >
> >>> > > > 2013-06-12 14:48:46,467] WARN Error while fetching metadata
> >>> >  partition
> >>> > > 0
> >>> > > >     leader: none    replicas:       isr:    isUnderReplicated:
> >>> false
> >>> > for
> >>> > > > topic partition [C,0]: [class
> >>> kafka.common.LeaderNotAvailableException]
> >>> > > > (kafka.producer.BrokerPartitionInfo)
> >>> > > > [2013-06-12 14:48:46,467] WARN Error while fetching metadata
> >>> > >  partition 0
> >>> > > >     leader: none    replicas:       isr:    isUnderReplicated:
> >>> false
> >>> > for
> >>> > > > topic partition [C,0]: [class
> >>> kafka.common.LeaderNotAvailableException]
> >>> > > > (kafka.producer.BrokerPartitionInfo)
> >>> > > > [2013-06-12 14:48:46,468] WARN Error while fetching metadata
> >>> > >  partition 2
> >>> > > >     leader: none    replicas:       isr:    isUnderReplicated:
> >>> false
> >>> > for
> >>> > > > topic partition [C,2]: [class
> >>> kafka.common.LeaderNotAvailableException]
> >>> > > > (kafka.producer.BrokerPartitionInfo)
> >>> > > > [2013-06-12 14:48:46,468] WARN Error while fetching metadata
> >>> > >  partition 2
> >>> > > >     leader: none    replicas:       isr:    isUnderReplicated:
> >>> false
> >>> > for
> >>> > > > topic partition [C,2]: [class
> >>> kafka.common.LeaderNotAvailableException]
> >>> > > > (kafka.producer.BrokerPartitionInfo)
> >>> > > >
> >>> > > >
> >>> > > > And sometimes lines like this:
> >>> > > >
> >>> > > > [2013-06-12 14:55:37,339] WARN Error while fetching metadata
> >>> > > > [{TopicMetadata for topic B ->
> >>> > > > No partition metadata for topic B due to
> >>> > > > kafka.common.LeaderNotAvailableException}] for topic [B]: class
> >>> > > > kafka.common.LeaderNotAvailableException
> >>> > > >  (kafka.producer.BrokerPartitionInfo)
> >>> > > >
> >>> > > >
> >>> > > > Do you guys have any idea what's going on?
> >>> > > >
> >>> > > > Thanks in advance,
> >>> > > > Alex
> >>> > > >
> >>> > > > --
> >>> > > >
> >>> > > > @BlisMedia <http://twitter.com/BlisMedia>
> >>> > > >
> >>> > > > www.blismedia.com <http://blismedia.com>
> >>> > > >
> >>> > > > This email and any attachments to it may be confidential and are
> >>> > intended
> >>> > > > solely
> >>> > > > for the use of the individual to whom it is addressed. Any views
> or
> >>> > > > opinions
> >>> > > > expressed are solely those of the author and do not necessarily
> >>> > represent
> >>> > > > those of BlisMedia Ltd, a company registered in England and Wales
> >>> with
> >>> > > > registered number 06455773. Its registered office is 3rd Floor,
> >>> 101 New
> >>> > > > Cavendish St, London, W1W 6XH, United Kingdom.
> >>> > > >
> >>> > > > If you are not the intended recipient of this email, you must
> >>> neither
> >>> > > take
> >>> > > > any action based upon its contents, nor copy or show it to
> anyone.
> >>> > Please
> >>> > > > contact the sender if you believe you have received this email in
> >>> > error.
> >>> > > >
> >>> > >
> >>> >
> >>> > --
> >>> >
> >>> > @BlisMedia <http://twitter.com/BlisMedia>
> >>> >
> >>> > www.blismedia.com <http://blismedia.com>
> >>> >
> >>> > This email and any attachments to it may be confidential and are
> >>> intended
> >>> > solely
> >>> > for the use of the individual to whom it is addressed. Any views or
> >>> > opinions
> >>> > expressed are solely those of the author and do not necessarily
> >>> represent
> >>> > those of BlisMedia Ltd, a company registered in England and Wales
> with
> >>> > registered number 06455773. Its registered office is 3rd Floor, 101
> New
> >>> > Cavendish St, London, W1W 6XH, United Kingdom.
> >>> >
> >>> > If you are not the intended recipient of this email, you must neither
> >>> take
> >>> > any action based upon its contents, nor copy or show it to anyone.
> >>> Please
> >>> > contact the sender if you believe you have received this email in
> >>> error.
> >>> >
> >>>
> >>
> >>
> >
>
> --
>
> @BlisMedia <http://twitter.com/BlisMedia>
>
> www.blismedia.com <http://blismedia.com>
>
> This email and any attachments to it may be confidential and are intended
> solely
> for the use of the individual to whom it is addressed. Any views or
> opinions
> expressed are solely those of the author and do not necessarily represent
> those of BlisMedia Ltd, a company registered in England and Wales with
> registered number 06455773. Its registered office is 3rd Floor, 101 New
> Cavendish St, London, W1W 6XH, United Kingdom.
>
> If you are not the intended recipient of this email, you must neither take
> any action based upon its contents, nor copy or show it to anyone. Please
> contact the sender if you believe you have received this email in error.
>

Reply via email to