Hi Peter Bukowinski,

Thanks for your response. That's a great deal of observation. You have
rightly nailed down the problematic instance.
Please find broker #21 and zookeeper logs. I see a lot happening, but
something unwell! Any advice on the problem. Thanks in advance!

Broker #21 logs:
----------------

ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-3 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-1 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-4 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=31, fetcherId=1] Error for
partition xxx-2 to broker
31:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-4 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=31, fetcherId=1] Error for
partition xxxx-19 to broker
31:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-3 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-0 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-0 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
ERROR [ReplicaFetcher replicaId=21, leaderId=11, fetcherId=1] Error for
partition xxx-10 to broker
11:org.apache.kafka.common.errors.NotLeaderForPartitionException: This
server is not the leader for that topic-partition.
(kafka.server.ReplicaFetcherThread)
kafka.common.KafkaException: Error processing data for partition xxx-1
offset <some random offset>


Zookeeper #21 logs:
-------------------

[myid:2] - INFO  [NIOServerCxn.Factory:zookeeper/ip:2181:ZooKeeperServer@928]
- Client attempting to establish new session at /ip:37186
[myid:2] - INFO  [CommitProcessor:2:ZooKeeperServer@673] - Established
session 0x27adea7b3240000 with negotiated timeout 40000 for client /ip:37186
[myid:2] - INFO  [SessionTracker:ZooKeeperServer@358] - Expiring session
0x37ad88230400000, timeout of 40000ms exceeded
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@487]
- Processed session termination for sessionid: 0x37ad88230400000
[myid:2] - INFO  [SessionTracker:ZooKeeperServer@358] - Expiring session
0x17ad98233330000, timeout of 40000ms exceeded
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@487]
- Processed session termination for sessionid: 0x17ad98233330000
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@487]
- Processed session termination for sessionid: 0x17adea7b3120004
[myid:2] - INFO  [PurgeTask:DatadirCleanupManager$PurgeTask@138] - Purge
task started.
[myid:2] - INFO  [PurgeTask:DatadirCleanupManager$PurgeTask@144] - Purge
task completed.
[myid:2] - ERROR [LearnerHandler-/ip:32828:LearnerHandler@631] - Unexpected
exception causing shutdown while sock still open
java.net.SocketTimeoutException: Read timed out
        at java.net.SocketInputStream.socketRead0(Native Method)
        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
        at java.net.SocketInputStream.read(SocketInputStream.java:171)
        at java.net.SocketInputStream.read(SocketInputStream.java:141)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
        at java.io.DataInputStream.readInt(DataInputStream.java:387)
        at
org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
        at
org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:83)
        at
org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:99)
        at
org.apache.zookeeper.server.quorum.LearnerHandler.run(LearnerHandler.java:542)
[myid:2] - WARN  [LearnerHandler-/ip:32828:LearnerHandler@644] - *******
GOODBYE /ip:32828 ********
[myid:2] - INFO  [WorkerReceiver[myid=2]:FastLeaderElection@600] -
Notification: 1 (message format version), 1 (n.leader), 0x4ddc0000022b
(n.zxid), 0x4e34 (n.round), LOOKING (n.state), 1 (n.sid), 0x4ddc
(n.peerEpoch) LEADING (my state)
[myid:2] - INFO  [LearnerHandler-/xxx:51114:LearnerHandler@329] - Follower
sid: 1 : info :
org.apache.zookeeper.server.quorum.QuorumPeer$QuorumServer@65deb578
[myid:2] - INFO  [LearnerHandler-/xxx:51114:LearnerHandler@384] -
Synchronizing with Follower sid: 1 maxCommittedLog=0x4ddc00000308
minCommittedLog=0x4ddc00000114 peerLastZxid=0x4ddc0000022b
[myid:2] - INFO  [LearnerHandler-/xxx:51114:LearnerHandler@458] - Sending
DIFF
[myid:2] - INFO  [LearnerHandler-/xxx:51114:LearnerHandler@518] - Received
NEWLEADER-ACK message from 1
[myid:2] - ERROR [LearnerHandler-/xxx:51114:LearnerHandler@631] -
Unexpected exception causing shutdown while sock still open
java.net.SocketTimeoutException: Read timed out
        at java.net.SocketInputStream.socketRead0(Native Method)
        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
        at java.net.SocketInputStream.read(SocketInputStream.java:171)
        at java.net.SocketInputStream.read(SocketInputStream.java:141)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
        at java.io.DataInputStream.readInt(DataInputStream.java:387)
        at
org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
        at
org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:83)
        at
org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:99)
        at
org.apache.zookeeper.server.quorum.LearnerHandler.run(LearnerHandler.java:542)
[myid:2] - WARN  [LearnerHandler-/xxx:51114:LearnerHandler@644] - *******
GOODBYE /xxx:51114 ********
[myid:2] - INFO  [WorkerReceiver[myid=2]:FastLeaderElection@600] -
Notification: 1 (message format version), 1 (n.leader), 0x4ddc00000440
(n.zxid), 0x4e34 (n.round), LOOKING (n.state), 1 (n.sid), 0x4ddc
(n.peerEpoch) LEADING (my state)

[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x1 zxid:0x4ddc00000699
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/1/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/1/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x8 zxid:0x4ddc0000069b
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/4/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/4/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:create cxid:0x9 zxid:0x4ddc0000069c
txntype:-1 reqpath:n/a Error Path:/brokers Error:KeeperErrorCode =
NodeExists for /brokers
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:create cxid:0xa zxid:0x4ddc0000069d
txntype:-1 reqpath:n/a Error Path:/brokers/ids Error:KeeperErrorCode =
NodeExists for /brokers/ids
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0xd zxid:0x4ddc0000069f
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/1/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/1/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x12 zxid:0x4ddc000006a0
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/2/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/2/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x17 zxid:0x4ddc000006a1
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/5/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/5/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x19 zxid:0x4ddc000006a2
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/1/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/1/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x1b zxid:0x4ddc000006a3
txntype:-1 reqpath:n/a Error
Path:/brokers/topics/__consumer_offsets/partitions/29/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/__consumer_offsets/partitions/29/state
[myid:2] - INFO  [ProcessThread(sid:2 cport:-1)::PrepRequestProcessor@649]
- Got user-level KeeperException when processing
sessionid:0x37adea7f42f0000 type:setData cxid:0x1d zxid:0x4ddc000006a4
txntype:-1 reqpath:n/a Error Path:/brokers/topics/xxx/partitions/0/state
Error:KeeperErrorCode = BadVersion for
/brokers/topics/xxx/partitions/0/state

On Wed, Jul 28, 2021 at 1:46 AM Peter Bukowinski <pmb...@gmail.com> wrote:

> Hi Sridhar,
>
> If your min.insync.replicas value is set to 3, then kafka won’t be able to
> move replicas until there are three replicas listed in the ISR. I would
> look into the health of broker 21 — it’s either down or unhealthy. It’s the
> only one not showing in the ISR list.
>
> —
> Peter Bukowinski
>
> > On Jul 27, 2021, at 1:12 AM, Sridhar Rao <sridharnuvv...@gmail.com>
> wrote:
> >
> > Hi Fabio Pardi,
> >
> > Thanks for your prompt response.
> > Split brain was our suspicion and we are investigating other
> possibilities.
> > Perhaps our understanding of the problem might be incorrect at the
> moment.
> > The issue started when one of the broker instances went down abruptly (3
> > brokers, 3 zookeepers) and the cluster was unstable.
> >
> > Later, we were able to restart the affected broker instance followed by
> > rolling restart of other 2 brokers. The cluster was stabilized at this
> > point.
> > However, we noticed un-repl partitions and Preferred Replica imbalance
> > irregularities.
> >
> > [xxx(user):/xxx/install/1.0.0/bin] ./kafka-topics.sh --describe
> --zookeeper
> > zookeeper1:2181 --under-replicated-partitions
> >        Topic: ABC      Partition: 3    Leader: 31      Replicas: 31,21,11
> >     Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 1    Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 3    Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 7    Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 9    Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 13   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 15   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 19   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 21   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 25   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 27   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 31   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 33   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 37   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 43   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 45   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: __consumer_offsets       Partition: 49   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: _kafka_lenses_alerts     Partition: 0    Leader: 31
> > Replicas: 31,21,11      Isr: 31,11
> >        Topic: _kafka_lenses_alerts_settings    Partition: 0    Leader: 31
> >     Replicas: 31,21,11      Isr: 31,11
> >        Topic: _kafka_lenses_processors Partition: 0    Leader: 31
> > Replicas: 31,21,11      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 0    Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 4    Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 6    Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 10   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 12   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 16   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 18   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 22   Leader: 31
> > Replicas: 21,11,31      Isr: 31,11
> >        Topic: connect-kfkxxxprd-offset Partition: 24   Leader: 31
> > Replicas: 31,11,21      Isr: 31,11
> >        Topic: connect-kfkxxxprd-status Partition: 3    Leader: 31
> > Replicas: 21,31,11      Isr: 31,11
> >
> > On Tue, Jul 27, 2021 at 9:46 AM Fabio Pardi <f.pa...@portavita.eu>
> wrote:
> >
> >>
> >>
> >> On 27/07/2021 09:19, Sridhar Rao wrote:
> >>> Hi Everyone,
> >>>
> >>> Recently we noticed a high number of under-replicated-partitions after
> >>> zookeeper split brain issue.
> >>> We tried fixing the issue by executing ./kafka-reassign-partitions.sh
> >>> procedure. However Kafka refuses to re-assign the partitions in ISR and
> >>> un-repl partitions remain the same.
> >>>
> >>> Kafka version: Apache Kafka 1.0.0
> >>>
> >>> Any advice in this regard would be highly appreciated. Please feel free
> >> to
> >>> request any missing information. Thank you!
> >>>
> >>
> >> Zookeeper needs a quorum to elect a leader therefore a split brain is
> >> unlikely. Can you describe how split brain occurred? (number of
> instances,
> >> what happened and how it was resolved)
> >>
> >> Given that, if your topic/partitions topology did not change, (that I
> >> know) even a split brain should not affect your data, therefore your
> ISR.
> >>
> >> Running the below command should tell you which partitions are under
> >> replicated
> >>
> >> kafka-topics.sh --describe
> >>
> >> I think it might be a good starting point to understand what is going
> on.
> >>
> >>
> >> To blindly reassign partitions is in my experience not an ideal
> solution,
> >> because you will have data shuffling around unnecessarily.
> >>
> >>
> >> regards,
> >>
> >> fabio pardi
> >>
> >>
>
>

Reply via email to