i will try to reproduce this problem later this week.

Bouncing the broker fixed the issue but the issue surfaced again after a
period of time. A little more context about this is that the cluster was
deployed to VMs and I discovered that the issue appeared whenever CPU wait
time was extremely high like 90% CPU time spent on I/O wait. I am more
interesting in understanding under what circumstance this issue would
happen so that I can take appropriate actions

On Fri, May 15, 2015 at 8:04 AM, Jiangjie Qin <j...@linkedin.com.invalid>
wrote:

>
> If you can reproduce this problem steadily, once you see this issue, can
> you grep the controller log for topic partition in question and see if
> there is anything interesting?
>
> Thanks.
>
> Jiangjie (Becket) Qin
>
> On 5/14/15, 3:43 AM, "tao xiao" <xiaotao...@gmail.com> wrote:
>
> >Yes, it does exist in ZK and the node that had the
> >NotLeaderForPartitionException
> >is the leader of the topic
> >
> >On Thu, May 14, 2015 at 6:12 AM, Jiangjie Qin <j...@linkedin.com.invalid>
> >wrote:
> >
> >> Does this topic exist in Zookeeper?
> >>
> >> On 5/12/15, 11:35 PM, "tao xiao" <xiaotao...@gmail.com> wrote:
> >>
> >> >Hi,
> >> >
> >> >Any updates on this issue? I keep seeing this issue happening over and
> >> >over
> >> >again
> >> >
> >> >On Thu, May 7, 2015 at 7:28 PM, tao xiao <xiaotao...@gmail.com> wrote:
> >> >
> >> >> Hi team,
> >> >>
> >> >> I have a 12 nodes cluster that has 800 topics and each of which has
> >> >>only 1
> >> >> partition. I observed that one of the node keeps generating
> >> >> NotLeaderForPartitionException that causes the node to be
> >>unresponsive
> >> >>to
> >> >> all requests. Below is the exception
> >> >>
> >> >> [2015-05-07 04:16:01,014] ERROR [ReplicaFetcherThread-1-12], Error
> >>for
> >> >> partition [topic1,0] to broker 12:class
> >> >> kafka.common.NotLeaderForPartitionException
> >> >> (kafka.server.ReplicaFetcherThread)
> >> >>
> >> >> All other nodes in the cluster generate lots of replication error
> >>too as
> >> >> shown below due to unresponsiveness of above node.
> >> >>
> >> >> [2015-05-07 04:17:34,917] WARN [Replica Manager on Broker 1]: Fetch
> >> >> request with correlation id 3630911 from client
> >> >>ReplicaFetcherThread-0-1 on
> >> >> partition [topic1,0] failed due to Leader not local for partition
> >> >> [cg22_user.item_attr_info.lcr,0] on broker 1
> >> >>(kafka.server.ReplicaManager)
> >> >>
> >> >> Any suggestion why the node runs into the unstable stage and any
> >> >> configuration I can set to prevent this?
> >> >>
> >> >> I use kafka 0.8.2.1
> >> >>
> >> >> And here is the server.properties
> >> >>
> >> >>
> >> >> broker.id=5
> >> >> port=9092
> >> >> num.network.threads=3
> >> >> num.io.threads=8
> >> >> socket.send.buffer.bytes=1048576
> >> >> socket.receive.buffer.bytes=1048576
> >> >> socket.request.max.bytes=104857600
> >> >> log.dirs=/mnt/kafka
> >> >> num.partitions=1
> >> >> num.recovery.threads.per.data.dir=1
> >> >> log.retention.hours=1
> >> >> log.segment.bytes=1073741824
> >> >> log.retention.check.interval.ms=300000
> >> >> log.cleaner.enable=false
> >> >> zookeeper.connect=ip:2181
> >> >> zookeeper.connection.timeout.ms=6000
> >> >> unclean.leader.election.enable=false
> >> >> delete.topic.enable=true
> >> >> default.replication.factor=3
> >> >> num.replica.fetchers=3
> >> >> delete.topic.enable=true
> >> >> kafka.metrics.reporters=report.KafkaMetricsCollector
> >> >> straas.hubble.conf.file=/etc/kafka/report.conf
> >> >>
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Regards,
> >> >> Tao
> >> >>
> >> >
> >> >
> >> >
> >> >--
> >> >Regards,
> >> >Tao
> >>
> >>
> >
> >
> >--
> >Regards,
> >Tao
>
>


-- 
Regards,
Tao

Reply via email to