Skip to content

Fail to recover a partition which was temporarily delegated to the internal broker #4804

@bjosv

Description

@bjosv

Description

When a partition is absent from metadata it is delegated to the internal broker :0/internal.
If the partition comes back later, with the same leader as before, it's no longer delegated back from the internal broker.
This makes the partition unusable, causing producing and consuming from said partition to stall.

The issue is likely a result of this change 6584ed7 in v2.4.0 (PR #4680)

This change requires that the epoch has been changed (leader_epoch > rktp->rktp_leader_epoch) before calling rd_kafka_toppar_broker_update(), which is not the case in this situation.

This commit also affects the related testcase 107 which fails.
This related test scenario only seem to pass if the commit 6584ed7 is reverted.

Logs where the issue can be seen:

2024-08-01T03:25:05.966753578 kafka:7: STATE: [thrd:main]: Topic MYTOPIC changed state unknown -> exists
2024-08-01T03:25:05.966777471 kafka:7: METADATA: [thrd:main]:   Topic MYTOPIC partition 0 Leader 2 Epoch 0
2024-08-01T03:25:05.966785841 kafka:7: BROKER: [thrd:main]: MYTOPIC [0]: leader -1 epoch -1 -> leader 2 epoch 0
2024-08-01T03:25:05.966790507 kafka:7: BRKDELGT: [thrd:main]: MYTOPIC [0]: delegate to broker ssl://kf-2.kf.svc.cluster.local:9093/2 (rktp 0x7f2d7ea4cd00, term 0, ref 3)
2024-08-01T03:25:05.966794391 kafka:7: BRKDELGT: [thrd:main]: MYTOPIC [0]: delegating to broker ssl://kf-2.kf.svc.cluster.local:9093/2 for partition with 0 messages (0 bytes) queued
2024-08-01T03:25:05.970168433 kafka:7: METADATA: [thrd:main]:   Topic MYTOPIC partition 0 Leader 2 Epoch 0
2024-08-01T03:25:05.970836944 kafka:7: METADATA: [thrd:main]: ssl://kf-0.kf.svc.cluster.local:9093/0: 1/1 requested topic(s) seen in metadata (lookup by name)
2024-08-01T03:25:06.157250297 kafka:7: METADATA: [thrd:main]: Error in metadata reply for topic MYTOPIC (PartCnt 0): Broker: Unknown topic or partition
2024-08-01T03:25:06.157461696 kafka:7: STATE: [thrd:main]: Topic MYTOPIC changed state exists -> notexists
2024-08-01T03:25:06.158317757 kafka:7: BRKDELGT: [thrd:main]: MYTOPIC [0]: delegate to broker (none) (rktp 0x7f2d7ea4cd00, term 0, ref 4)
2024-08-01T03:25:06.158591549 kafka:7: BRKDELGT: [thrd:main]: MYTOPIC [0]: no longer delegated to broker ssl://kf-2.kf.svc.cluster.local:9093/2
2024-08-01T03:25:06.158816894 kafka:7: BRKDELGT: [thrd:main]: MYTOPIC [0]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
2024-08-01T03:25:06.159422291 kafka:7: METADATA: [thrd:main]: ssl://kf-2.kf.svc.cluster.local:9093/2: 1/1 requested topic(s) seen in metadata (lookup by name)
2024-08-01T03:25:06.160886844 kafka:7: METADATA: [thrd:main]: ssl://kf-2.kf.svc.cluster.local:9093/2: 1/1 requested topic(s) seen in metadata (lookup by name)
2024-08-01T03:25:06.171946064 kafka:7: METADATA: [thrd:main]: Error in metadata reply for topic MYTOPIC (PartCnt 0): Broker: Unknown topic or partition
2024-08-01T03:25:06.171950544 kafka:7: METADATA: [thrd:main]: ssl://kf-2.kf.svc.cluster.local:9093/2: 1/1 requested topic(s) seen in metadata (lookup by name)
2024-08-01T03:25:06.959820354 kafka:7: STATE: [thrd:main]: Topic MYTOPIC changed state notexists -> exists

Missing a BRKDELGT log here.
Should delegate topic MYTOPIC back from ":0/internal" to "ssl://kf-2.kf.svc.cluster.local:9093/2"

Running test 107:

cd tests
./interactive_broker_version.py 3.6.2 --scenario noautocreate
TESTS=0107 TEST_DEBUG=topic,metadata make

Checklist

  • librdkafka version (release number or git tag): v2.4.0
  • Apache Kafka version: 3.6.2
  • librdkafka client configuration: defaults
  • Operating system: SLES
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue

CC @emasab

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions