redpanda-data / redpanda

Redpanda is a streaming data platform for developers. Kafka API compatible. 10x faster. No ZooKeeper. No JVM!
https://redpanda.com
9.58k stars 582 forks source link

Stuck shards during high partition count topic creation #15591

Open travisdowns opened 10 months ago

travisdowns commented 10 months ago

Version & Environment

Redpanda version: 23.2.17

What went wrong?

When repeatedly creating a partition with a high number of partitions (22,800 in this case), some shards seem to get into a state where they cannot complete the creation of the replica state. Notes:

Here is a specific example of a node with shards in a bad state.

We create the 30 partition topic ttta using rpk topic create ttta -p30.

rpk cluster health shows some ttta partitions as under-replicated (ignore the other partitions):

$ rpk cluster health
CLUSTER HEALTH OVERVIEW
=======================
Healthy:                     false
Unhealthy reasons:           [leaderless_partitions under_replicated_partitions]
Controller ID:               1
All nodes:                   [0 1 2]
Nodes down:                  []
Leaderless partitions:       [kafka/ttt/19 kafka/ttt/42 kafka/ttt/65 kafka/ttt/88 kafka/ttta/11 kafka/tttb/27 kafka/tttb/4 kafka/tttb/50 kafka/tttb/73 kafka/tttb/96]
Under-replicated partitions: [kafka/taddd/0 kafka/taddd/1 kafka/taddd/2 kafka/ttt/11 kafka/ttt/27 kafka/ttt/28 kafka/ttt/29 kafka/ttt/30 kafka/ttt/34 kafka/ttt/50 kafka/ttt/51 kafka/ttt/52 kafka/ttt/53 kafka/ttt/57 kafka/ttt/7 kafka/ttt/73 kafka/ttt/74 kafka/ttt/75 kafka/ttt/76 kafka/ttt/80 kafka/ttt/96 kafka/ttt/97 kafka/ttt/98 kafka/ttt/99 kafka/ttta/19 kafka/ttta/20 kafka/ttta/21 kafka/ttta/22 kafka/ttta/26 kafka/ttta/3 kafka/tttb/12 kafka/tttb/13 kafka/tttb/14 kafka/tttb/15 kafka/tttb/19 kafka/tttb/35 kafka/tttb/36 kafka/tttb/37 kafka/tttb/38 kafka/tttb/42 kafka/tttb/58 kafka/tttb/59 kafka/tttb/60 kafka/tttb/61 kafka/tttb/65 kafka/tttb/81 kafka/tttb/82 kafka/tttb/83 kafka/tttb/84 kafka/tttb/88]

This state is persistent.

We can look at the number of partitions per hon broker 1 (controller leader) and see that some shards are many fewer partitions:

$ curl -s http://localhost:9644/v1/partitions" | jq . | grep 'core":' | sort | uniq -c
     13     "core": 0,
     14     "core": 1,
     14     "core": 10,
      4     "core": 11,
     14     "core": 12,
     14     "core": 13,
     14     "core": 14,
     14     "core": 15,
     14     "core": 16,
     14     "core": 17,
     14     "core": 18,
      4     "core": 19,
     14     "core": 2,
     14     "core": 20,
     14     "core": 21,
     14     "core": 22,
     14     "core": 3,
      4     "core": 4,
      4     "core": 5,
      4     "core": 6,
     14     "core": 7,
     14     "core": 8,
     14     "core": 9,

All the cores with 4 partitions are stuck: not being able to reconcile new replicas. The 4 partitions they are do have are from topics like __consumer_offsets which existed before the problem manifested.

We can look at a specific partition kafka/ttta/3 which was had a replica assigned on shard 11 on all 3 brokers. On brokers 0 and 2, where shard 11 is unstuck, the partition table reports the partition as done, like so:

kubectl exec -it -c redpanda -n redpanda rp-clrjeb2e4n4gt1b99bu0-0 -- bash -c "curl -s http://localhost:9644/v1/partitions/kafka/ttta/3" | jq .
{
  "ns": "kafka",
  "topic": "ttta",
  "partition_id": 3,
  "status": "done",
  "leader_id": 0,
  "raft_group_id": 1021434,
  "replicas": [
    {
      "node_id": 0,
      "core": 11
    },
    {
      "node_id": 2,
      "core": 11
    },
    {
      "node_id": 1,
      "core": 11
    }
  ]
}

On broker 1, however, the partition is shown as "in_progress":

$ kubectl exec -it -c redpanda -n redpanda rp-clrjeb2e4n4gt1b99bu0-1 -- bash -c "curl -s http://localhost:9644/v1/partitions/kafka/ttta/3" | jq .
{
  "ns": "kafka",
  "topic": "ttta",
  "partition_id": 3,
  "status": "in_progress",
  "leader_id": 0,
  "raft_group_id": 1021434,
  "replicas": [
    {
      "node_id": 0,
      "core": 11
    },
    {
      "node_id": 2,
      "core": 11
    },
    {
      "node_id": 1,
      "core": 11
    }
  ]
}

Find here the full TRACE logs for the period including the creation of the ttta topic on broker 1 (controller). You can see that after ttta/3 is assigned to shard 11 there are not log lines at all regarding any reconciliation happening in cluster_backend, unlike unaffected partitions which have them (the only log lines are a result of a leader being elected for this partition on one of the other two brokers, and corresponding updates to the leader table.

This partition is still available since it is healthy on the other two brokers, but shard 19 is stock on both of brokers 0 and 1, so partitions on that shard (for example, ttta/11) are leaderless and unavailable.

What should have happened instead?

Partitions should reconcile and become available.

How to reproduce the issue?

  1. Repeatedly create and delete a 22,800 partition topic checking for stuck in_progress topics on the leader.

JIRA Link: CORE-1656

mmaslankaprv commented 10 months ago

From the logs attached it seems that controller_backend is not active at all on affected shards. It may be the case that some of the partitions failed to stop. I will try to reproduce that issue as the log doesn't contain entries from the moment when the problem occurred (i.e. creation/deletion of 22000 partitions topic)

travisdowns commented 10 months ago

@mmaslankaprv I learned overnight that simply creating/destroying the 23k partitions is not enough, this was working fine when run 100s of times in a loop. So the actual thing I was doing was a short load test omb_validation_test.py::OMBValidationTest.test_max_partitions) in between and I guess this is somehow relevant. Will provide more details soon.

mmaslankaprv commented 10 months ago

this may be related: https://github.com/redpanda-data/redpanda/issues/15392

ztlpn commented 10 months ago

Interesting thing is that these partitions are seen in the logs (grep for test-topic-) and each shard hosts 300-400 partitions of a single topic:

This suggests that when a topic is deleted and partitions are stopped, there is a chance that on one of the shards they will get stuck waiting for some common resource (as opposed to partitions getting stuck independently).

travisdowns commented 10 months ago

Interesting thing is that these partitions are seen in the logs (grep for test-topic-) and each shard hosts 300-400 partitions of a single topic:

It did strike me that there might be some sort of deadlock, e.g., via ssg exhaustion or some other route.

I think I was able to reproduce this with a simpler setup and will provide more details soon.

mmaslankaprv commented 9 months ago

@travisdowns let us know if you will have any updates here

travisdowns commented 8 months ago

@mmaslankaprv - we ended discussing this a bit on slack and I think the current status is that you have the timed watchdog checked in to give more info when delta application stalls?