[ 
https://issues.apache.org/jira/browse/KAFKA-7635?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steven Aerts updated KAFKA-7635:
--------------------------------
    Description: 
After disabling unclean leader leader again after recovery of a situation where 
we enabled unclean leader due to a split brain in zookeeper, we saw that some 
of our brokers stopped replicating their partitions.

Digging into the logs, we saw that the replica thread was stopped because one 
partition had a failure which threw a [{{Error processing data for partition}} 
exception|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L207].
  But the broker kept running and serving the partitions from which it was 
leader.

We saw three different types of exceptions triggering this (example stacktraces 
attached):
* {{kafka.common.UnexpectedAppendOffsetException}}
* {{Trying to roll a new log segment for topic partition partition-b-97 with 
start offset 1388 while it already exists.}}
* {{Kafka scheduler is not running.}}

We think there are two acceptable ways for the kafka broker to handle this:
* Mark those partitions as a partition with error and handle them accordingly.  
As is done [when a {{CorruptRecordException}} or 
{{KafkaStorageException}}|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L196]
 is thrown.
* Exit the broker as is done [when log truncation is not 
allowed|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala#L189].
 
Maybe even a combination of both.  Our probably naive idea is that for the 
first two types the first strategy would be the best, but for the last type, it 
is probably better to re-throw a {{FatalExitError}} and exit the broker.



  was:
After disabling unclean leader leader again after recovery of a situation where 
we enabled unclean leader due to a split brain in zookeeper, we saw that some 
of our stopped replicating their partitions.

Digging into the logs, we saw that the replica thread was stopped because one 
partition had a failure which threw a [{{Error processing data for partition}} 
exception|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L207].
  But the broker kept running and serving the partitions from which it was 
leader.

We saw three different types of exceptions triggering this (example stacktraces 
attached):
* {{kafka.common.UnexpectedAppendOffsetException}}
* {{Trying to roll a new log segment for topic partition partition-b-97 with 
start offset 1388 while it already exists.}}
* {{Kafka scheduler is not running.}}

We think there are two acceptable ways for the kafka broker to handle this:
* Mark those partitions as a partition with error and handle them accordingly.  
As is done [when a {{CorruptRecordException}} or 
{{KafkaStorageException}}|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L196]
 is thrown.
* Exit the broker as is done [when log truncation is not 
allowed|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala#L189].
 
Maybe even a combination of both.  Our probably naive idea is that for the 
first two types the first strategy would be the best, but for the last type, it 
is probably better to re-throw a {{FatalExitError}} and exit the broker.




> FetcherThread stops processing after "Error processing data for partition"
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-7635
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7635
>             Project: Kafka
>          Issue Type: Bug
>          Components: replication
>    Affects Versions: 2.0.0
>            Reporter: Steven Aerts
>            Priority: Major
>         Attachments: stacktraces.txt
>
>
> After disabling unclean leader leader again after recovery of a situation 
> where we enabled unclean leader due to a split brain in zookeeper, we saw 
> that some of our brokers stopped replicating their partitions.
> Digging into the logs, we saw that the replica thread was stopped because one 
> partition had a failure which threw a [{{Error processing data for 
> partition}} 
> exception|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L207].
>   But the broker kept running and serving the partitions from which it was 
> leader.
> We saw three different types of exceptions triggering this (example 
> stacktraces attached):
> * {{kafka.common.UnexpectedAppendOffsetException}}
> * {{Trying to roll a new log segment for topic partition partition-b-97 with 
> start offset 1388 while it already exists.}}
> * {{Kafka scheduler is not running.}}
> We think there are two acceptable ways for the kafka broker to handle this:
> * Mark those partitions as a partition with error and handle them 
> accordingly.  As is done [when a {{CorruptRecordException}} or 
> {{KafkaStorageException}}|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L196]
>  is thrown.
> * Exit the broker as is done [when log truncation is not 
> allowed|https://github.com/apache/kafka/blob/2.0.0/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala#L189].
>  
> Maybe even a combination of both.  Our probably naive idea is that for the 
> first two types the first strategy would be the best, but for the last type, 
> it is probably better to re-throw a {{FatalExitError}} and exit the broker.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to