kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Thunder Stumpges <tstump...@ntent.com>
Subject RE: Problem with node after restart no partitions?
Date Wed, 01 Apr 2015 18:54:33 GMT
Well it appears we lost all the data on the one node again. It appears to be all or part of
KAFKA-1647<https://issues.apache.org/jira/browse/KAFKA-1647> as we saw this in our logs
(for all topics):

[2015-04-01 10:46:58,901] WARN Partition [logactivity-redirect,3] on broker 6: No checkpointed
highwatermark is found for partition [logactivity-redirect,3] (kafka.cluster.Partition)
[2015-04-01 10:46:58,902] WARN Partition [pageimpression,1] on broker 6: No checkpointed highwatermark
is found for partition [pageimpression,1] (kafka.cluster.Partition)
[2015-04-01 10:46:58,904] WARN Partition [campaignplatformtarget,6] on broker 6: No checkpointed
highwatermark is found for partition [campaignplatformtarget,6] (kafka.cluster.Partition)
[2015-04-01 10:46:58,905] WARN Partition [trackingtags-c2,1] on broker 6: No checkpointed
highwatermark is found for partition [trackingtags-c2,1] (kafka.cluster.Partition)

Followed by:

[2015-04-01 10:46:58,911] INFO Truncating log trafficshaperlog-3 to offset 0. (kafka.log.Log)
[2015-04-01 10:46:58,928] INFO Truncating log videorecrequest-0 to offset 0. (kafka.log.Log)
[2015-04-01 10:46:58,928] INFO Truncating log filteredredirect-2 to offset 0. (kafka.log.Log)
[2015-04-01 10:46:58,985] INFO Truncating log precheckrequest-3 to offset 0. (kafka.log.Log)
[2015-04-01 10:46:58,990] INFO Truncating log filteredclicklog-8 to offset 0. (kafka.log.Log)

Followed by:
[2015-04-01 10:46:59,107] INFO Scheduling log segment 90276704 for log pageview-0 for deletion.
(kafka.log.Log)
[2015-04-01 10:46:59,107] INFO Scheduling log segment 90593329 for log pageview-0 for deletion.
(kafka.log.Log)
[2015-04-01 10:46:59,107] INFO Scheduling log segment 90906048 for log pageview-0 for deletion.
(kafka.log.Log)
[2015-04-01 10:46:59,107] INFO Scheduling log segment 91226773 for log pageview-0 for deletion.
(kafka.log.Log)


The strange thing however is that I don't believe we ever had the scenario mentioned in the
bug (all brokers for a topic down) As we have 0 unclean leader elections, our applications
never complained the cluster was down, and we never actually stopped more than one node (the
node in question).

Anyway, really unsure if this could have somehow been related to the attempted upgrade to
0.8.2.1 (the occurrence this morning was NOT, it was a simple stop, restart broker)

We are really nervous to do any kafka service restarts now that this has happened twice on
this machine.
Any suggestions? Should we go back and make another attempt to upgrade to 0.8.2.1 ?

Thanks,
Thunder


From: Thunder Stumpges
Sent: Wednesday, April 01, 2015 10:30 AM
To: kafka-users@incubator.apache.org
Subject: Problem with node after restart no partitions?

Hi group,

We've been trying to track down a problem this morning for a little while, and thought I'd
ask here while we keep looking.

We have 3 nodes (rep-3) running 8.1.1. We attempted a rolling upgrade yesterday to 8.2.1,
and on the first node, after restarting, a single topic (a samza intermediate topic) started
throwing replica fetcher errors over and over ("NotLeaderForPartition"). There may or may
not have been other things attempted at this time (not by me so I cannot say for sure). Anyway
we ended up rolling back to 8.1.1 and ALL data had been DELETED from that node. It spent most
of yesterday re-syncing, and came into sync last night, and a rebalance made everything run
smoothly (*except for these damn replica fetcher errors for that one partition).

Today my colleague attempted the "unsupported" topic delete command for the "bad" partition,
and bounced that one troublesome node.

Upon coming up, I can see in server.log that it is reading in all of the segments in, and
then starts spitting out a samza topic fetch error, and through JMX the "ReplicaManager".LeaderCount
is 0. It is not attempting to fetch or load any topics.

The other two brokers are showing under-replicated (obviously). What is going wrong? How can
we get that samza topic really and truly gone? (if that is the cause of the broker not coming
up)

Thanks,
Thunder


Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message