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

Colin McCabe updated KAFKA-14074:
---------------------------------
    Summary: Restarting a broker during re-assignment can leave log directory 
entries in ZK mode  (was: Restarting a broker during re-assignment can leave 
log directory entries)

> Restarting a broker during re-assignment can leave log directory entries in 
> ZK mode
> -----------------------------------------------------------------------------------
>
>                 Key: KAFKA-14074
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14074
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.8.0, 3.1.0
>            Reporter: Adrian Preston
>            Priority: Major
>
> Re-starting a broker while replicas are being assigned away from the broker 
> can result in topic partition directories being left in the broker’s log 
> directory. This can trigger further problems if such a topic is deleted and 
> re-created. These problems occur when replicas for the new topic are placed 
> on a broker that hosts a “stale” topic partition directory of the same name, 
> causing the on-disk topic partition state held by different brokers in the 
> cluster to diverge.
> We have also been able to re-produce variants this problem using Kafka 2.8 
> and 3.1, as well as Kafka built from the head of the apache/kafka repository 
> (at the time of writing this is commit: 
> 94d4fdeb28b3cd4d474d943448a7ef653eaa145d). We have *not* being able to 
> re-produce this problem with Kafka running in KRaft mode.
> A minimal re-create for topic directories being left on disk is as follows:
>  # Start ZooKeeper and a broker (both using the sample config)
>  # Create 100 topics: each with 1 partition, and with replication factor 1
>  # Add a second broker to the Kafka cluster (with minor edits to the sample 
> config for: {{{}broker.id{}}}, {{{}listeners{}}}, and {{{}log.dirs{}}})
>  # Issue a re-assignment that moves all of the topic partition replicas  from 
> the first broker to the second broker
>  # While this re-assignment is taking place shutdown the first broker (you 
> need to be quick with only two brokers and 100 topics…)
>  # Wait a few seconds for the re-assignment to stall
>  # Restart the first broker and wait for the re-assignment to complete and it 
> to remove any partially deleted topics (e.g. those with a “-delete” suffix).
> Inspecting the logs directory for the first broker should show directories 
> corresponding to topic partitions that are owned by the second broker. These 
> are not cleaned up when the re-assignment completes, and also remain in the 
> logs directory even if the first broker is restarted.  Deleting the topic 
> also does not clean up the topic partitions left behind on the first broker - 
> which leads to a second potential problem.
> For topics that have more than one replica: a new topic that has the same 
> name as a previously deleted topic might have replicas created on a broker 
> with “stale” topic partition directories. If this happens these topics will 
> remain in an under-replicated state.
> A minimal re-create for this is as follows:
>  # Create a three node Kafka cluster (backed by ZK) based off the sample 
> config (to avoid confusion let’s call these kafka-0, kafka-1, and kafka-2)
>  # Create 100 topics: each with 1 partition, and with replication factor 2
>  # Submit a re-assignment to move all of the topic partition replicas to 
> kafka-0 and kafka-1,  and wait for it to complete
>  # Submit a re-assignment to move all of the topic partition replicas on 
> kafka-0 to kafka-2.
>  # While this re-assignment is taking place shutdown and re-start kafka-0.
>  # Wait for the re-assignment to complete, and check that there’s unexpected 
> topic partition directories in kafka-0’s logs directory
>  # Delete all 100 topics, and re-create 100 new topics with the same name and 
> configuration as the deleted topics.
> In this state kafka-1 and kafka-2 continually generate log messages similar 
> to:
> {{[2022-07-14 13:07:49,118] WARN [ReplicaFetcher replicaId=2, leaderId=0, 
> fetcherId=0] Received INCONSISTENT_TOPIC_ID from the leader for partition 
> test-039-0. This error may be returned transiently when the partition is 
> being created or deleted, but it is not expected to persist. 
> (kafka.server.ReplicaFetcherThread)}}
> Topics that have had replicas created on kafka-0 are under-replicated with 
> kafka-0 missing from the ISR list. Performing a rolling restart of each 
> broker in turn does not resolve the problem, in fact more partitions are 
> listed as under-replicated, as before kafka-0 is missing from their ISR list.
> I also tried to re-create this with Kafka running in Kraft mode, but was 
> unable to do so. My test configuration was three brokers configured based on 
> /config/kraft/server.properties. All three brokers were part of the 
> controller quorum. Interestingly I see log lines like the following when 
> re-starting the broker that I stopped mid-reassignment:
> {{[2022-07-14 13:44:42,705] INFO Found stray log dir 
> Log(dir=/tmp/kraft-2/test-029-0, topicId=DMGA3zxyQqGUfeV6cmkcmg, 
> topic=test-029, partition=0, highWatermark=0, lastStableOffset=0, 
> logStartOffset=0, logEndOffset=0): the current replica assignment [I@530d4c70 
> does not contain the local brokerId 2. 
> (kafka.server.metadata.BrokerMetadataPublisher$)}}
> With later log lines showing the topic being deleted. Looking at the 
> corresponding code: KRaft mode explicitly checks that the topic ID on disk 
> matches the expected value, and deletes the directory if it does not.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to