[ https://issues.apache.org/jira/browse/KAFKA-8709?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Christian Becker updated KAFKA-8709: ------------------------------------ Description: We attempted to do an update from 2.2 to 2.3 and then a rollback was done after {{inter.broker.protocol}} was changed. (We know this shouldn't be done, but it happened). After downgrading to 2.2 again, some {{__consumer-offsets}} partitions fail to load with the message {{Unknown group metadata version 3}}. Subsequently the broker continues it's startup and the consumer groups won't exist. So the consumers are starting at their configured OLDEST or NEWEST position and start committing their offsets. However on subsequent restarts of the brokers, the {{Unknown group metadata version}} exception remains and so the restarts are happening over and over again. In order to prevent this, I'd suggest a updated flow when loading the offsets: - the loading should continue reading the __consumer-offsets partition to see if a subsequent offset is commited that is readable - if no "valid" offset could be found, throw the existing exception to let the operator know about the situation - if a valid offset can be found, continue as normal This would cause the following sequence of events: 1. corrupted offsets are written 2. broker restart 2a. broker loads offset partition 2b. {{KafkaException}} when loading the offset partition 2c. no "valid" offset is found after the "corrupt" record 2d. offsets reset 3. consumergroups are recreated and "valid" offsets are appended 4. broker restart 4a. broker loads offset partition 4b. {{KafkaException}} when loading the offset partition 4c. "valid" offset is found after the "corrupted" ones 5. consumergroups still have their latest offset It's a special case now, that this happened after some human error, but this also poses a danger for situations where the offsets might be corrupted for some unrelated reason. losing the offsets is a very serious situation and there should be safeguards against it, especially when there might be offsets recoverable. With this improvement, the offsets would be still lost once, but the broker is able to recover automatically over time and after compaction the corrupted records will be gone. (In our case this caused serious confusion as we've lost the offsets multiple times, as the error message {{Error loading offsets from}} implies, that the corrupted data is deleted and therefore the situation is recovered, whereas in reality this continues to be a issue until the corrupt data is gone once and for all which might take a long time. In our case we seem to have evicted the broken records with temporarily setting the segment time to a very low value and deactivation of compaction {code:java} /opt/kafka/bin/kafka-topics.sh --alter --config segment.ms=900000 --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=delete --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --config retention.ms=1800000 --topic __consumer_offsets --zookeeper localhost:2181 < wait for the cleaner to clean up > /opt/kafka/bin/kafka-topics.sh --alter --delete-config segment.ms --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=compact --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --delete-config retention.ms --topic __consumer_offsets --zookeeper localhost:2181 < hope all consumer groups commited their offset before a failover needs to happen >{code} was: We attempted to do an update from 2.2 to 2.3 and then a rollback was done after {{inter.broker.protocol}} was changed. (We know this shouldn't be done, but it happened). After downgrading to 2.2 again, some {{__consumer-offsets}} partitions fail to load with the message {{Unknown group metadata version 3}}. Subsequently the broker continues it's startup and the consumer groups won't exist. So the consumers are starting at their configured OLDEST or NEWEST position and start committing their offsets. However on subsequent restarts of the brokers, the {{Unknown group metadata version}} exception remains and so the restarts are happening over and over again. In order to prevent this, I'd suggest a updated flow when loading the offsets: - the loading should continue reading the __consumer-offsets partition to see if a subsequent offset is commited that is readable - if no "valid" offset could be found, throw the existing exception to let the operator know about the situation - if a valid offset can be found, continue as normal This would cause the following sequence of events: 1. corrupted offsets are written 2. broker restart 2a. broker loads offset partition 2b. {{KafkaException}} when loading the offset partition 2c. no "valid" offset is found after the "corrupt" record 2d. offsets reset 3. consumergroups are recreated and "valid" offsets are appended 4. broker restart 4a. broker loads offset partition 4b. {{KafkaException}} when loading the offset partition 4c. "valid" offset is found after the "corrupted" ones 5. consumergroups still have their latest offset It's a special case now, that this happened after some human error, but this also poses a danger for situations where the offsets might be corrupted for some unrelated reason. losing the offsets is a very serious situation and there should be safeguards against it, especially when there might be offsets recoverable. With this improvement, the offsets would be still lost once, but the broker is able to recover automatically over time and after compaction the corrupted records will be gone. (In our case this caused serious confusion as we've lost the offsets multiple times, as the error message {{Error loading offsets from}} implies, that the corrupted data is deleted and therefore the situation is recovered, whereas in reality this continues to be a issue until the corrupt data is gone once and for all which might take a long time. In our case we seem to have evicted the broken records with temporarily setting the segment time to a very low value and deactivation of compaction {code:java} /opt/kafka/bin/kafka-topics.sh --alter --config segment.ms=900000 --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=delete --topic __consumer_offsets --zookeeper localhost:2181 < wait for the cleaner to clean up > /opt/kafka/bin/kafka-topics.sh --alter --config segment.ms=604800000 --topic __consumer_offsets --zookeeper localhost:2181 /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=compact --topic __consumer_offsets --zookeeper localhost:2181{code} > improve consumer offsets resiliency > ----------------------------------- > > Key: KAFKA-8709 > URL: https://issues.apache.org/jira/browse/KAFKA-8709 > Project: Kafka > Issue Type: Improvement > Reporter: Christian Becker > Priority: Major > > We attempted to do an update from 2.2 to 2.3 and then a rollback was done > after {{inter.broker.protocol}} was changed. (We know this shouldn't be done, > but it happened). > After downgrading to 2.2 again, some {{__consumer-offsets}} partitions fail > to load with the message {{Unknown group metadata version 3}}. Subsequently > the broker continues it's startup and the consumer groups won't exist. So the > consumers are starting at their configured OLDEST or NEWEST position and > start committing their offsets. > However on subsequent restarts of the brokers, the {{Unknown group metadata > version}} exception remains and so the restarts are happening over and over > again. > > In order to prevent this, I'd suggest a updated flow when loading the offsets: > - the loading should continue reading the __consumer-offsets partition to > see if a subsequent offset is commited that is readable > - if no "valid" offset could be found, throw the existing exception to let > the operator know about the situation > - if a valid offset can be found, continue as normal > > This would cause the following sequence of events: > 1. corrupted offsets are written > 2. broker restart > 2a. broker loads offset partition > 2b. {{KafkaException}} when loading the offset partition > 2c. no "valid" offset is found after the "corrupt" record > 2d. offsets reset > 3. consumergroups are recreated and "valid" offsets are appended > 4. broker restart > 4a. broker loads offset partition > 4b. {{KafkaException}} when loading the offset partition > 4c. "valid" offset is found after the "corrupted" ones > 5. consumergroups still have their latest offset > It's a special case now, that this happened after some human error, but this > also poses a danger for situations where the offsets might be corrupted for > some unrelated reason. losing the offsets is a very serious situation and > there should be safeguards against it, especially when there might be offsets > recoverable. With this improvement, the offsets would be still lost once, but > the broker is able to recover automatically over time and after compaction > the corrupted records will be gone. (In our case this caused serious > confusion as we've lost the offsets multiple times, as the error message > {{Error loading offsets from}} implies, that the corrupted data is deleted > and therefore the situation is recovered, whereas in reality this continues > to be a issue until the corrupt data is gone once and for all which might > take a long time. > In our case we seem to have evicted the broken records with temporarily > setting the segment time to a very low value and deactivation of compaction > {code:java} > /opt/kafka/bin/kafka-topics.sh --alter --config segment.ms=900000 --topic > __consumer_offsets --zookeeper localhost:2181 > /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=delete --topic > __consumer_offsets --zookeeper localhost:2181 > /opt/kafka/bin/kafka-topics.sh --alter --config retention.ms=1800000 --topic > __consumer_offsets --zookeeper localhost:2181 > < wait for the cleaner to clean up > > /opt/kafka/bin/kafka-topics.sh --alter --delete-config segment.ms --topic > __consumer_offsets --zookeeper localhost:2181 > /opt/kafka/bin/kafka-topics.sh --alter --config cleanup.policy=compact > --topic __consumer_offsets --zookeeper localhost:2181 > /opt/kafka/bin/kafka-topics.sh --alter --delete-config retention.ms --topic > __consumer_offsets --zookeeper localhost:2181 > < hope all consumer groups commited their offset before a failover needs to > happen >{code} > -- This message was sent by Atlassian JIRA (v7.6.14#76016)