You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Gokul <sl...@gmail.com> on 2016/07/12 05:47:05 UTC

NotAssignedReplicaException

We had an issue last week when kafka cluster reported under replicated
partitions for quite a while but there were no brokers down. All the
brokers were reporting unknownException on Broker 1. When checked broker 1
logs, it just reported below errors(NotAssignedReplicaException)
continuously. Issue got resolved after bouncing broker 1. Think this
exception comes when Controller issues StopReplicaRequest to broker 1 and
it is in the process of leader election. But what is spooky is that this
exception was reported more than 20 minutes by the broker and it was
chocking entire ingestion(totally 10 brokers). Unfortunately I don't have
the controller logs to debug. Any pointers here? We are using 0.8.2.1

ERROR [2016-07-07 11:45:09,248] [kafka-request-handler-1][]
kafka.server.KafkaApis - [KafkaApi-1] error when handling request Name:
FetchRequest; Version: 0; CorrelationId: 1890972; ClientId:
ReplicaFetcherThread-2-1; ReplicaId: 2; MaxWait: 500 ms; MinBytes: 1 bytes;
RequestInfo: <value>
kafka.common.NotAssignedReplicaException: Leader 1 failed to record
follower 2's position 20240372 since the replica is not recognized to be
one of the assigned replicas 1,5,6 for partition [<topic>,1]
        at
kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
        at
kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
        at
kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
        at
scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
        at
scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
        at
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:778)
        at
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
        at
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
        at
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
        at
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:777)
        at scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245)
        at
kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
        at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
        at
kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
        at java.lang.Thread.run(Thread.java:745)

-- 
Thanks and Regards,
Gokul

Re: NotAssignedReplicaException

Posted by Gokul <go...@gmail.com>.
No, not using any script of that sort. Sure.

Thanks.
On 12 Jul 2016 22:08, "Tom Crayford" <tc...@heroku.com> wrote:

> Ah, that's around leader rebalancing. Do you have any scripts that run
> kafka-assign-partitions or similar?
>
> I will recheck but this doesn't sound like a thing that auto rebalance
> would impact
>
> On Tuesday, 12 July 2016, Gokul <go...@gmail.com> wrote:
>
>> Thanks. Auto rebalance is set to true, so rebalancing may be happening at
>> that time. Is there any issue tracker that I can refer to?
>> On 12 Jul 2016 21:48, "Tom Crayford" <tc...@heroku.com> wrote:
>>
>>> Hi,
>>>
>>> Were you rebalancing that topic or partition at that time? If there are
>>> rebalancing bugs this might point at that.
>>>
>>> Thanks
>>>
>>> Tom
>>>
>>> On Tue, Jul 12, 2016 at 6:47 AM, Gokul <sl...@gmail.com> wrote:
>>>
>>> > We had an issue last week when kafka cluster reported under replicated
>>> > partitions for quite a while but there were no brokers down. All the
>>> > brokers were reporting unknownException on Broker 1. When checked
>>> broker 1
>>> > logs, it just reported below errors(NotAssignedReplicaException)
>>> > continuously. Issue got resolved after bouncing broker 1. Think this
>>> > exception comes when Controller issues StopReplicaRequest to broker 1
>>> and
>>> > it is in the process of leader election. But what is spooky is that
>>> this
>>> > exception was reported more than 20 minutes by the broker and it was
>>> > chocking entire ingestion(totally 10 brokers). Unfortunately I don't
>>> have
>>> > the controller logs to debug. Any pointers here? We are using 0.8.2.1
>>> >
>>> > ERROR [2016-07-07 11:45:09,248] [kafka-request-handler-1][]
>>> > kafka.server.KafkaApis - [KafkaApi-1] error when handling request Name:
>>> > FetchRequest; Version: 0; CorrelationId: 1890972; ClientId:
>>> > ReplicaFetcherThread-2-1; ReplicaId: 2; MaxWait: 500 ms; MinBytes: 1
>>> bytes;
>>> > RequestInfo: <value>
>>> > kafka.common.NotAssignedReplicaException: Leader 1 failed to record
>>> > follower 2's position 20240372 since the replica is not recognized to
>>> be
>>> > one of the assigned replicas 1,5,6 for partition [<topic>,1]
>>> >         at
>>> >
>>> >
>>> kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
>>> >         at
>>> >
>>> >
>>> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
>>> >         at
>>> >
>>> >
>>> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
>>> >         at
>>> >
>>> >
>>> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>>> >         at
>>> >
>>> >
>>> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>>> >         at
>>> >
>>> >
>>> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:778)
>>> >         at
>>> > scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
>>> >         at
>>> >
>>> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>>> >         at
>>> >
>>> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>>> >         at
>>> >
>>> >
>>> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:777)
>>> >         at
>>> scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245)
>>> >         at
>>> > kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
>>> >         at
>>> kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
>>> >         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
>>> >         at
>>> > kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
>>> >         at java.lang.Thread.run(Thread.java:745)
>>> >
>>> > --
>>> > Thanks and Regards,
>>> > Gokul
>>> >
>>>
>>

Re: NotAssignedReplicaException

Posted by Tom Crayford <tc...@heroku.com>.
Ah, that's around leader rebalancing. Do you have any scripts that run
kafka-assign-partitions or similar?

I will recheck but this doesn't sound like a thing that auto rebalance
would impact

On Tuesday, 12 July 2016, Gokul <go...@gmail.com> wrote:

> Thanks. Auto rebalance is set to true, so rebalancing may be happening at
> that time. Is there any issue tracker that I can refer to?
> On 12 Jul 2016 21:48, "Tom Crayford" <tcrayford@heroku.com
> <javascript:_e(%7B%7D,'cvml','tcrayford@heroku.com');>> wrote:
>
>> Hi,
>>
>> Were you rebalancing that topic or partition at that time? If there are
>> rebalancing bugs this might point at that.
>>
>> Thanks
>>
>> Tom
>>
>> On Tue, Jul 12, 2016 at 6:47 AM, Gokul <slukog@gmail.com
>> <javascript:_e(%7B%7D,'cvml','slukog@gmail.com');>> wrote:
>>
>> > We had an issue last week when kafka cluster reported under replicated
>> > partitions for quite a while but there were no brokers down. All the
>> > brokers were reporting unknownException on Broker 1. When checked
>> broker 1
>> > logs, it just reported below errors(NotAssignedReplicaException)
>> > continuously. Issue got resolved after bouncing broker 1. Think this
>> > exception comes when Controller issues StopReplicaRequest to broker 1
>> and
>> > it is in the process of leader election. But what is spooky is that this
>> > exception was reported more than 20 minutes by the broker and it was
>> > chocking entire ingestion(totally 10 brokers). Unfortunately I don't
>> have
>> > the controller logs to debug. Any pointers here? We are using 0.8.2.1
>> >
>> > ERROR [2016-07-07 11:45:09,248] [kafka-request-handler-1][]
>> > kafka.server.KafkaApis - [KafkaApi-1] error when handling request Name:
>> > FetchRequest; Version: 0; CorrelationId: 1890972; ClientId:
>> > ReplicaFetcherThread-2-1; ReplicaId: 2; MaxWait: 500 ms; MinBytes: 1
>> bytes;
>> > RequestInfo: <value>
>> > kafka.common.NotAssignedReplicaException: Leader 1 failed to record
>> > follower 2's position 20240372 since the replica is not recognized to be
>> > one of the assigned replicas 1,5,6 for partition [<topic>,1]
>> >         at
>> >
>> >
>> kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
>> >         at
>> >
>> >
>> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
>> >         at
>> >
>> >
>> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
>> >         at
>> >
>> >
>> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>> >         at
>> >
>> >
>> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>> >         at
>> >
>> >
>> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:778)
>> >         at
>> > scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
>> >         at
>> >
>> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>> >         at
>> >
>> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>> >         at
>> >
>> >
>> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:777)
>> >         at
>> scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245)
>> >         at
>> > kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
>> >         at
>> kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
>> >         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
>> >         at
>> > kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
>> >         at java.lang.Thread.run(Thread.java:745)
>> >
>> > --
>> > Thanks and Regards,
>> > Gokul
>> >
>>
>

Re: NotAssignedReplicaException

Posted by Gokul <go...@gmail.com>.
Thanks. Auto rebalance is set to true, so rebalancing may be happening at
that time. Is there any issue tracker that I can refer to?
On 12 Jul 2016 21:48, "Tom Crayford" <tc...@heroku.com> wrote:

> Hi,
>
> Were you rebalancing that topic or partition at that time? If there are
> rebalancing bugs this might point at that.
>
> Thanks
>
> Tom
>
> On Tue, Jul 12, 2016 at 6:47 AM, Gokul <sl...@gmail.com> wrote:
>
> > We had an issue last week when kafka cluster reported under replicated
> > partitions for quite a while but there were no brokers down. All the
> > brokers were reporting unknownException on Broker 1. When checked broker
> 1
> > logs, it just reported below errors(NotAssignedReplicaException)
> > continuously. Issue got resolved after bouncing broker 1. Think this
> > exception comes when Controller issues StopReplicaRequest to broker 1 and
> > it is in the process of leader election. But what is spooky is that this
> > exception was reported more than 20 minutes by the broker and it was
> > chocking entire ingestion(totally 10 brokers). Unfortunately I don't have
> > the controller logs to debug. Any pointers here? We are using 0.8.2.1
> >
> > ERROR [2016-07-07 11:45:09,248] [kafka-request-handler-1][]
> > kafka.server.KafkaApis - [KafkaApi-1] error when handling request Name:
> > FetchRequest; Version: 0; CorrelationId: 1890972; ClientId:
> > ReplicaFetcherThread-2-1; ReplicaId: 2; MaxWait: 500 ms; MinBytes: 1
> bytes;
> > RequestInfo: <value>
> > kafka.common.NotAssignedReplicaException: Leader 1 failed to record
> > follower 2's position 20240372 since the replica is not recognized to be
> > one of the assigned replicas 1,5,6 for partition [<topic>,1]
> >         at
> >
> >
> kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
> >         at
> >
> >
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
> >         at
> >
> >
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
> >         at
> >
> >
> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
> >         at
> >
> >
> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
> >         at
> >
> >
> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:778)
> >         at
> > scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
> >         at
> > scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
> >         at
> > scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
> >         at
> >
> >
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:777)
> >         at
> scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245)
> >         at
> > kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
> >         at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
> >         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
> >         at
> > kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
> >         at java.lang.Thread.run(Thread.java:745)
> >
> > --
> > Thanks and Regards,
> > Gokul
> >
>

Re: NotAssignedReplicaException

Posted by Tom Crayford <tc...@heroku.com>.
Hi,

Were you rebalancing that topic or partition at that time? If there are
rebalancing bugs this might point at that.

Thanks

Tom

On Tue, Jul 12, 2016 at 6:47 AM, Gokul <sl...@gmail.com> wrote:

> We had an issue last week when kafka cluster reported under replicated
> partitions for quite a while but there were no brokers down. All the
> brokers were reporting unknownException on Broker 1. When checked broker 1
> logs, it just reported below errors(NotAssignedReplicaException)
> continuously. Issue got resolved after bouncing broker 1. Think this
> exception comes when Controller issues StopReplicaRequest to broker 1 and
> it is in the process of leader election. But what is spooky is that this
> exception was reported more than 20 minutes by the broker and it was
> chocking entire ingestion(totally 10 brokers). Unfortunately I don't have
> the controller logs to debug. Any pointers here? We are using 0.8.2.1
>
> ERROR [2016-07-07 11:45:09,248] [kafka-request-handler-1][]
> kafka.server.KafkaApis - [KafkaApi-1] error when handling request Name:
> FetchRequest; Version: 0; CorrelationId: 1890972; ClientId:
> ReplicaFetcherThread-2-1; ReplicaId: 2; MaxWait: 500 ms; MinBytes: 1 bytes;
> RequestInfo: <value>
> kafka.common.NotAssignedReplicaException: Leader 1 failed to record
> follower 2's position 20240372 since the replica is not recognized to be
> one of the assigned replicas 1,5,6 for partition [<topic>,1]
>         at
>
> kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
>         at
>
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
>         at
>
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
>         at
>
> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>         at
>
> scala.collection.MapLike$MappedValues$$anonfun$foreach$3.apply(MapLike.scala:245)
>         at
>
> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:778)
>         at
> scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
>         at
> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>         at
> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>         at
>
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:777)
>         at scala.collection.MapLike$MappedValues.foreach(MapLike.scala:245)
>         at
> kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
>         at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
>         at
> kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
>         at java.lang.Thread.run(Thread.java:745)
>
> --
> Thanks and Regards,
> Gokul
>