Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-16157

Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

Attach filesAttach ScreenshotVotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 3.7.0
    • 3.7.0
    • jbod, kraft
    • None

    Description

      In a cluster with 4 brokers, `broker-1..broker-4` with 2 disks `d1` and `d2` in each broker, we perform the following operations:

       

      1. Create a topic `foo.test` with 10 partitions and RF 4. Let's assume the topic was created with id `rAujIqcjRbu_-E4UxgQT8Q`.
      2. Start a producer in the background to produce to `foo.test`.
      3. Break disk `d1` in `broker-1`. We simulate this by marking the log dir read-only.
      4. Delete topic `foo.test`
      5. Recreate topic `foo.test`. Let's assume the topic was created with id `bgdrsv-1QjCLFEqLOzVCHg`.
      6. Wait for 5 minutes
      7. Describe the recreated topic `foo.test`.

       

      We observe that `broker-1` is the leader and in-sync for few partitions

       

       

       
      Topic: foo.test TopicId: bgdrsv-1QjCLFEqLOzVCHg PartitionCount: 10      ReplicationFactor: 4    Configs: min.insync.replicas=1,unclean.leader.election.enable=false
              Topic: foo.test Partition: 0    Leader: 101     Replicas: 101,102,103,104       Isr: 101,102,103,104
              Topic: foo.test Partition: 1    Leader: 102     Replicas: 102,103,104,101       Isr: 102,103,104
              Topic: foo.test Partition: 2    Leader: 103     Replicas: 103,104,101,102       Isr: 103,104,102
              Topic: foo.test Partition: 3    Leader: 104     Replicas: 104,101,102,103       Isr: 104,102,103
              Topic: foo.test Partition: 4    Leader: 104     Replicas: 104,102,101,103       Isr: 104,102,103
              Topic: foo.test Partition: 5    Leader: 102     Replicas: 102,101,103,104       Isr: 102,103,104
              Topic: foo.test Partition: 6    Leader: 101     Replicas: 101,103,104,102       Isr: 101,103,104,102
              Topic: foo.test Partition: 7    Leader: 103     Replicas: 103,104,102,101       Isr: 103,104,102
              Topic: foo.test Partition: 8    Leader: 101     Replicas: 101,102,104,103       Isr: 101,102,104,103
              Topic: foo.test Partition: 9    Leader: 102     Replicas: 102,104,103,101       Isr: 102,104,103
      

       

       

      In this example, it is the leader of partitions `0, 6 and 8`.

       

      Consider `foo.test-8`. It is present in the following brokers/disks:

       

       

      $ fd foo.test-8
      broker-1/d1/foo.test-8/
      broker-2/d2/foo.test-8/
      broker-3/d2/foo.test-8/
      broker-4/d1/foo.test-8/

       

       

      `broker-1/d1` still refers to the topic id which is pending deletion because the log dir is marked offline.

       

       

      $ cat broker-1/d1/foo.test-8/partition.metadata
      version: 0
      topic_id: rAujIqcjRbu_-E4UxgQT8Q

       

       

      However, other brokers have the correct topic-id

       

       

      $ cat broker-2/d2/foo.test-8/partition.metadata
      version: 0
      topic_id: bgdrsv-1QjCLFEqLOzVCHg%

       

       

      Now, let's consider `foo.test-0`. We observe that the replica isn't present in `broker-1`:

      $ fd foo.test-0
      broker-2/d1/foo.test-0/
      broker-3/d1/foo.test-0/
      broker-4/d2/foo.test-0/

      In both cases, `broker-1` shouldn't be the leader or in-sync replica for the partitions.

       

      Attachments

        1. broker.log
          26 kB
          Gaurav Narula
        2. broker.log.1
          1.00 MB
          Gaurav Narula
        3. broker.log.10
          1.00 MB
          Gaurav Narula
        4. broker.log.2
          1.00 MB
          Gaurav Narula
        5. broker.log.3
          1.00 MB
          Gaurav Narula
        6. broker.log.4
          1.00 MB
          Gaurav Narula
        7. broker.log.5
          1.00 MB
          Gaurav Narula
        8. broker.log.6
          1.00 MB
          Gaurav Narula
        9. broker.log.7
          1.00 MB
          Gaurav Narula
        10. broker.log.8
          1.00 MB
          Gaurav Narula
        11. broker.log.9
          1.00 MB
          Gaurav Narula

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            gnarula Gaurav Narula
            gnarula Gaurav Narula
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment