[ 
https://issues.apache.org/jira/browse/KAFKA-13330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17436730#comment-17436730
 ] 

Fabio Hecht commented on KAFKA-13330:
-------------------------------------

Reproducible by setting topic config segment.index.bytes=0

> Incorrect topic configurations can damage cluster health
> --------------------------------------------------------
>
>                 Key: KAFKA-13330
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13330
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.6.1
>            Reporter: Florian Rossier
>            Priority: Critical
>
> One of our user created multiple topics with obviously wrong configurations 
> in our prod cluster, damaging the health of it for quite some times.
> After investigations, we managed to reproduce the problem in our test 
> environment (3 brokers, 3 zookeepers). 
> The topic configuration was looking like this (obviously wrong, but still 
> allowed by Kafka), the topic is empty : 
> {code:java}
> bash-4.2$ /opt/kafka/bin/kafka-topics.sh --zookeeper zookeeper:2181/kafka 
> --describe --topic topic.prod.test-failurebash-4.2$ 
> /opt/kafka/bin/kafka-topics.sh --zookeeper zookeeper:2181/kafka --describe 
> --topic topic.prod.test-failure
> Topic:topic.prod.test-failure PartitionCount:3 ReplicationFactor:2 
> Configs:message.downconversion.enable=true,file.delete.delay.ms=0,segment.ms=604800000,min.compaction.lag.ms=0,retention.bytes=0,segment.index.bytes=0,cleanup.policy=compact,follower.replication.throttled.replicas=,message.timestamp.difference.max.ms=0,segment.jitter.ms=0,preallocate=true,message.timestamp.type=CreateTime,message.format.version=2.6-IV0,segment.bytes=1073741824,unclean.leader.election.enable=true,max.message.bytes=0,retention.ms=86400000,flush.ms=0,delete.retention.ms=0,leader.replication.throttled.replicas=,min.insync.replicas=1,flush.messages=0,compression.type=uncompressed,min.cleanable.dirty.ratio=0.0,index.interval.bytes=0
>  
> Topic: topic.prod.test-failure Partition: 0 Leader: 0 Replicas: 0,1 Isr: 0,1 
> Topic: topic.prod.test-failure Partition: 1 Leader: 1 Replicas: 1,2 Isr: 1,2 
> Topic: topic.prod.test-failure Partition: 2 Leader: 2 Replicas: 2,0 Isr: 2,0
> {code}
>  
> When creating the topic, all brokers receiving the leadership of a partition 
> will directly report this error, while continuing to work (The replication of 
> this topic is certainly broken at this point).
> {code:java}
> [2021-09-28 07:25:14,871] ERROR [KafkaApi-0] Error when handling request: 
> clientId=2, correlationId=1125, api=LEADER_AND_ISR, version=4, 
> body={controller_id=2,controller_epoch=123,broker_epoch=115964299479,topic_states=[{topic_name=topic.prod.test-failure,partition_states=[{partition_index=0,controller_epoch=123,leader=0,leader_epoch=0,isr=[0,1],zk_version=0,replicas=[0,1],adding_replicas=[],removing_replicas=[],is_new=true,_tagged_fields={}},{partition_index=2,controller_epoch=123,leader=2,leader_epoch=0,isr=[2,0],zk_version=0,replicas=[2,0],adding_replicas=[],removing_replicas=[],is_new=true,_tagged_fields={}}],_tagged_fields={}}],live_leaders=[{broker_id=2,host_name=broker-2.kafka,port=9093,_tagged_fields={}},{broker_id=0,host_name=broker-0.kafka,port=9093,_tagged_fields={}}],_tagged_fields={}}
>  (kafka.server.KafkaApis)[2021-09-28 07:25:14,871] ERROR [KafkaApi-0] Error 
> when handling request: clientId=2, correlationId=1125, api=LEADER_AND_ISR, 
> version=4, 
> body={controller_id=2,controller_epoch=123,broker_epoch=115964299479,topic_states=[{topic_name=topic.prod.test-failure,partition_states=[{partition_index=0,controller_epoch=123,leader=0,leader_epoch=0,isr=[0,1],zk_version=0,replicas=[0,1],adding_replicas=[],removing_replicas=[],is_new=true,_tagged_fields={}},{partition_index=2,controller_epoch=123,leader=2,leader_epoch=0,isr=[2,0],zk_version=0,replicas=[2,0],adding_replicas=[],removing_replicas=[],is_new=true,_tagged_fields={}}],_tagged_fields={}}],live_leaders=[{broker_id=2,host_name=broker-2.kafka,port=9093,_tagged_fields={}},{broker_id=0,host_name=broker-0.kafka,port=9093,_tagged_fields={}}],_tagged_fields={}}
>  (kafka.server.KafkaApis)
> java.lang.IllegalArgumentException: Invalid max index size: 0 at 
>   kafka.log.AbstractIndex.<init>(AbstractIndex.scala:117) at 
>   kafka.log.OffsetIndex.<init>(OffsetIndex.scala:54) at 
>   kafka.log.LazyIndex$.$anonfun$forOffset$1(LazyIndex.scala:106)
> {code}
> This is staying like this without causing any others errors. But then one 
> broker goes down (rolling restart, host crash, simple broker restart). If the 
> broker is hosting one of those partitions, it will fail to start with this 
> error. 
> {code:java}
> [2021-09-28 07:33:52,087] ERROR There was an error in one of the threads 
> during logs loading: org.apache.kafka.common.errors.CorruptRecordException: 
> Found record size 0 smaller than minimum record overhead (14) in file 
> /data/2/topic.prod.test-failure-0/00000000000000000000.log. 
> (kafka.log.LogManager)
> [2021-09-28 07:33:52,151] ERROR [KafkaServer id=0] Fatal error during 
> KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
> org.apache.kafka.common.errors.CorruptRecordException: Found record size 0 
> smaller than minimum record overhead (14) in file 
> /data/2/topic.prod.test-failure-0/00000000000000000000.log.
> {code}
> At this point, the broker will crash in loop while others will report those 
> kind of errors on unrelated topics : 
> {code:java}
> [2021-09-28 07:33:04,776] INFO [Partition audit.stage.keycloak-1 broker=2] 
> Shrinking ISR from 2,1,0 to 2. Leader: (highWatermark: 78397, endOffset: 
> 78398). Out of sync replicas: (brokerId: 1, endOffset: 78397) (brokerId: 0, 
> endOffset: 78397). (kafka.cluster.Partition)
> [2021-09-28 07:33:04,779] INFO [Partition audit.stage.keycloak-1 broker=2] 
> Cached zkVersion 1411 not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> [2021-09-28 07:33:04,780] INFO [Partition datacastsimulator.dev.sip-1 
> broker=2] Shrinking ISR from 2,0 to 2. Leader: (highWatermark: 1302629961, 
> endOffset: 1302629971). Out of sync replicas: (brokerId: 0, endOffset: 
> 1302629961). (kafka.cluster.Partition)
> [2021-09-28 07:33:04,782] INFO [Partition datacastsimulator.dev.sip-1 
> broker=2] Cached zkVersion 1231 not equal to that in zookeeper, skip updating 
> ISR (kafka.cluster.Partition)
> [2021-09-28 07:33:04,782] INFO [Partition audit.stage.netflows-3 broker=2] 
> Shrinking ISR from 2,1,0 to 2. Leader: (highWatermark: 312387953, endOffset: 
> 312388360). Out of sync replicas: (brokerId: 1, endOffset: 312387953) 
> (brokerId: 0, endOffset: 312387953). (kafka.cluster.Partition)
> [2021-09-28 07:33:04,818] INFO [Partition audit.stage.netflows-3 broker=2] 
> Cached zkVersion 1103 not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> [2021-09-28 07:33:04,819] INFO [Partition audit.stage.kafka-1 broker=2] 
> Shrinking ISR from 2,1,0 to 2. Leader: (highWatermark: 80988951, endOffset: 
> 80988993). Out of sync replicas: (brokerId: 1, endOffset: 80988951) 
> (brokerId: 0, endOffset: 80988951). (kafka.cluster.Partition)
> [2021-09-28 07:33:04,821] INFO [Partition audit.stage.kafka-1 broker=2] 
> Cached zkVersion 1403 not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> {code}
> At this point, the replication can not be done for unrelated topics, creating 
> underreplicated partitions in the cluster with the problem that can be caused 
> by them on replication factor < 3. At this moment, it start to look bad and 
> alerts can start to be triggered at this point, and difficult to 
> investigation because the under replicated partition will only be reported on 
> random topics. 
> After discovering that this topic is not correct and is actually the problem, 
> deleting it via the command line won't actually delete the folder/files and 
> they will stay on disk, still causing issue to start the broker. 
> {code:java}
> /data/1/topic.prod.test-failure-1:
> total 16
> drwxr-sr-x.  2  100 bin       4096 Sep 28 09:25 .
> drwxrwsr-x. 98 root bin      12288 Sep 28 09:36 ..
> -rw-r--r--.  1  100 bin          0 Sep 28 09:25 00000000000000000000.index
> -rw-r--r--.  1  100 bin 1073741824 Sep 28 09:25 00000000000000000000.log
> -rw-r--r--.  1  100 bin          0 Sep 28 09:25 leader-epoch-checkpoint
> {code}
> Only way to recover the broker is to manually delete the files on disk. This 
> allow the broker to start again. 
> Then other partitions are still around in the disks of other brokers, causing 
> this kind of issues on other brokers (that will actually fail with the 
> previous error if restarted).
> {code:java}
> [2021-09-27 15:17:16,507] WARN  (kafka.utils.CoreUtils$)
> [2021-09-27 15:17:16,507] WARN  
> (kafka.utils.CoreUtils$)java.nio.BufferOverflowException at   
>    java.base/java.nio.Buffer.nextPutIndex(Buffer.java:673) at 
>    java.base/java.nio.DirectByteBuffer.putLong(DirectByteBuffer.java:882) at 
>    kafka.log.TimeIndex.$anonfun$maybeAppend$1(TimeIndex.scala:134) at 
>    kafka.log.TimeIndex.maybeAppend(TimeIndex.scala:114) at 
>    kafka.log.LogSegment.$anonfun$close$1(LogSegment.scala:590) at 
>    kafka.utils.CoreUtils$.swallow(CoreUtils.scala:69)
> {code}
> The only way to completely recover from this is to : 
>  # detect the topic with incorrect configuration
>  # remove it using the kafka-topic command
>  # Remove the topic-partition on disk in all the brokers
>  # At least restart all the brokers where a partition of the topic was 
> present. 
>  # Ideally perform a rolling restart
> While to topic configuration is obviously wrong, we think that Kafka should 
> prevent the creation of such topics or handling it properly since it can 
> greatly impact the health of the cluster, data loss can occurs for partitions 
> with replication factor less than 3, etc. 
> This problem wasn't tested with other versions than 2.6.1, but we could 
> imagine that others versions are impacted as well.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to