-
Notifications
You must be signed in to change notification settings - Fork 1
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Race condition between ReplicaFetcherThread and RemotePartitionMetadataStore. #33
Comments
Stack trace of the error:
|
|
Thanks for reporting this. I am back from holidays so I am going to start looking into this. |
Testing with upstream + retention PR, I can reproduce a similar issue:
and second broker is not able to replicate. Steps to reproduce: Using:
Create topic with RF=2
Stop second broker:
Write to topic:
Check directories: Broker 0:
Broker 1:
Then, start Broker 1, observe logs and directory:
|
Update: there seem to be a misconfiguration on my environment (upstream) that lead to this issue. User topic was configured with RF of 2, but internal topics were not, TBRLMM was complaining:
By applying this fix (Aiven-Open/tiered-storage-for-apache-kafka@2a36ac0) I haven't been able to reproduce this issue, and replication started to work as expected: Screencast.from.2023-07-11.11-42-48.webmThere's still more scenarios to test about replication in general, but wanted to note here that at least this scenario is not reproducible with upstream. @HenryCaiHaiying could you confirm if the internal topics (__remote_log_metadata specifically) have the proper replication factor on your environment? |
Please, ignore my latest comments. The race condition seems to be possible in upstream as well -- following up in this issue: https://issues.apache.org/jira/browse/KAFKA-15181 |
Upstream PR at apache#14012 |
I was able to reproduce the similar ERROR: org.apache.kafka.server.log.remote.storage.RemoteStorageException: Couldn't build the state from remote store for partition: t3-0, currentLeaderEpoch: 1, leaderLocalLogStartOffset: 54522, leaderLogStartOffset: 0, epoch: 1as the previous remote log segment metadata was not found Under this test setup: Kafka node-3 died, launched a replacement broker: node-4 and reassign the partitions from node-3 to node-4. Node-4 broker starts up prints out the above ERROR repeatedly. In my test up, the topic has a shorter retention.ms with only a few hours (for both local and remote), because there was no new incoming data to the topic, all remote segments were deleted. When PrimaryConsumerTask is handling DELETE_SEGMENT_FINISHED metadata event, it removes idToSegmentMetadata for the given segment in RemoteLogMetadataCache. This caused the lookup failed in the following line in TopicBasedRemoteLogMetadataManager:
Not sure about the best way to fix this, maybe keeping remote log segment retention really long? Or always keep the last segment in RemoteLogMetadataCache? |
New upstream bug is at apache#14127 |
[2023-06-26 22:11:04,980] ERROR [ReplicaFetcher replicaId=1, leaderId=0, fetcherId=0] Error building remote log auxiliary state for topic4-0 (kafka.server.ReplicaFetcherThread)
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: No resource found for partition: IWXjwq-vSsS8cu9DkGj6EQ:topic4-0
The text was updated successfully, but these errors were encountered: