Kafka
  1. Kafka
  2. KAFKA-1509

Restart of destination broker after unreplicated partition move leaves partitions without leader

    Details

    • Type: Bug Bug
    • Status: Open
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: 0.8.1.1
    • Fix Version/s: None
    • Component/s: None
    • Labels:

      Description

      This should be reasonably easy to reproduce.

      Make a Kafka cluster with a few machines.

      Create a topic with partitions on these machines. No replication.

      Bring up one more Kafka node.

      Move some or all of the partitions onto this new broker:

      kafka-reassign-partitions.sh --generate --zookeeper zk:2181 --topics-to-move-json-file move.json --broker-list <new broker>

      kafka-reassign-partitions.sh --zookeeper 36cfqd1.in.cfops.it:2181 --reassignment-json-file reassign.json --execute

      Wait until broker is the leader for all the partitions you moved.

      Send some data to the partitions. It all works.

      Shut down the broker that just received the data. Start it back up.

      Topic:test	PartitionCount:2	ReplicationFactor:1	Configs:
      	Topic: test	Partition: 0	Leader: -1	Replicas: 7	Isr: 
      	Topic: test	Partition: 1	Leader: -1	Replicas: 7	Isr: 
      

      Leader for topic test never gets elected even though this node is the only node that knows about the topic.

      Some logs:

      Jun 26 23:18:07 localhost kafka: INFO [Socket Server on Broker 7], Started (kafka.network.SocketServer)
      Jun 26 23:18:07 localhost kafka: INFO [Socket Server on Broker 7], Started (kafka.network.SocketServer)
      Jun 26 23:18:07 localhost kafka: INFO [ControllerEpochListener on 7]: Initialized controller epoch to 53 and zk version 52 (kafka.controller.ControllerEpochListener)
      Jun 26 23:18:07 localhost kafka: INFO Will not load MX4J, mx4j-tools.jar is not in the classpath (kafka.utils.Mx4jLoader$)
      Jun 26 23:18:07 localhost kafka: INFO Will not load MX4J, mx4j-tools.jar is not in the classpath (kafka.utils.Mx4jLoader$)
      Jun 26 23:18:07 localhost kafka: INFO [Controller 7]: Controller starting up (kafka.controller.KafkaController)
      Jun 26 23:18:07 localhost kafka: INFO conflict in /controller data: {"version":1,"brokerid":7,"timestamp":"1403824687354"} stored data: {"version":1,"brokerid":4,"timestamp":"1403297911725"} (kafka.utils.ZkUtils$)
      Jun 26 23:18:07 localhost kafka: INFO conflict in /controller data: {"version":1,"brokerid":7,"timestamp":"1403824687354"} stored data: {"version":1,"brokerid":4,"timestamp":"1403297911725"} (kafka.utils.ZkUtils$)
      Jun 26 23:18:07 localhost kafka: INFO [Controller 7]: Controller startup complete (kafka.controller.KafkaController)
      Jun 26 23:18:07 localhost kafka: INFO Registered broker 7 at path /brokers/ids/7 with address xxx:9092. (kafka.utils.ZkUtils$)
      Jun 26 23:18:07 localhost kafka: INFO Registered broker 7 at path /brokers/ids/7 with address xxx:9092. (kafka.utils.ZkUtils$)
      Jun 26 23:18:07 localhost kafka: INFO [Kafka Server 7], started (kafka.server.KafkaServer)
      Jun 26 23:18:07 localhost kafka: INFO [Kafka Server 7], started (kafka.server.KafkaServer)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:3) for partition [requests,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:1) for partition [requests,13] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:4,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:1,5) for partition [requests_ipv6,5] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:13,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:4,5) for partition [requests_stored,7] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:5,ISR:5,LeaderEpoch:17,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:5) for partition [requests,6] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:7,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:1,4) for partition [requests_ipv6,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:6,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:2,1) for partition [requests_ipv6,6] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:5,ISR:5,LeaderEpoch:17,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:5) for partition [requests,10] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:1,5) for partition [requests_stored,4] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:3,2) for partition [requests_stored,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:13,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:3,4) for partition [requests_stored,6] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:1) for partition [requests,14] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:5,ISR:5,LeaderEpoch:17,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:5) for partition [requests,2] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test3,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:2) for partition [requests,3] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:9,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:3,2) for partition [requests_ipv6,7] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:9,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:3,1) for partition [requests_ipv6,2] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:5,3) for partition [requests_ipv6,4] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:15,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:4) for partition [requests,5] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:1) for partition [requests,4] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:4,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:2,5) for partition [requests_ipv6,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:15,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:4) for partition [requests,9] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:4,2) for partition [requests_ipv6,3] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:15,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:2) for partition [requests,11] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:10,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:2,3) for partition [requests_stored,5] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:4,2) for partition [requests_error,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test3,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:5,4) for partition [requests_stored,3] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:17,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:4,3) for partition [requests_stored,2] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:2,ISR:2,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:2) for partition [requests,7] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:3,1) for partition [requests_error,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:11,ControllerEpoch:53),ReplicationFactor:2),AllReplicas:2,1) for partition [requests_stored,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:6,ISR:6,LeaderEpoch:21,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:6) for partition [requests,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:6,ISR:6,LeaderEpoch:24,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:6) for partition [requests,12] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:3,ISR:3,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:3) for partition [requests,8] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 70 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 received LeaderAndIsr request (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) correlation id 71 from controller 4 epoch 53 for partition [test,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 received LeaderAndIsr request (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) correlation id 71 from controller 4 epoch 53 for partition [test3,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 received LeaderAndIsr request (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) correlation id 71 from controller 4 epoch 53 for partition [test,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 received LeaderAndIsr request (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) correlation id 71 from controller 4 epoch 53 for partition [test3,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 handling LeaderAndIsr request correlationId 71 from controller 4 epoch 53 starting the become-follower transition for partition [test,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 handling LeaderAndIsr request correlationId 71 from controller 4 epoch 53 starting the become-follower transition for partition [test3,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 handling LeaderAndIsr request correlationId 71 from controller 4 epoch 53 starting the become-follower transition for partition [test,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 handling LeaderAndIsr request correlationId 71 from controller 4 epoch 53 starting the become-follower transition for partition [test3,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: ERROR Broker 7 aborted the become-follower state change with correlation id 71 from controller 4 epoch 53 for partition [test,0] since new leader -1 is not currently available (state.change.logger)
      Jun 26 23:18:07 localhost kafka: ERROR Broker 7 aborted the become-follower state change with correlation id 71 from controller 4 epoch 53 for partition [test3,1] since new leader -1 is not currently available (state.change.logger)
      Jun 26 23:18:07 localhost kafka: ERROR Broker 7 aborted the become-follower state change with correlation id 71 from controller 4 epoch 53 for partition [test,1] since new leader -1 is not currently available (state.change.logger)
      Jun 26 23:18:07 localhost kafka: ERROR Broker 7 aborted the become-follower state change with correlation id 71 from controller 4 epoch 53 for partition [test3,0] since new leader -1 is not currently available (state.change.logger)
      Jun 26 23:18:07 localhost kafka: INFO [ReplicaFetcherManager on broker 7] Removed fetcher for partitions  (kafka.server.ReplicaFetcherManager)
      Jun 26 23:18:07 localhost kafka: INFO [ReplicaFetcherManager on broker 7] Removed fetcher for partitions  (kafka.server.ReplicaFetcherManager)
      Jun 26 23:18:07 localhost kafka: INFO [ReplicaFetcherManager on broker 7] Added fetcher for partitions List() (kafka.server.ReplicaFetcherManager)
      Jun 26 23:18:07 localhost kafka: INFO [ReplicaFetcherManager on broker 7] Added fetcher for partitions List() (kafka.server.ReplicaFetcherManager)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 completed LeaderAndIsr request correlationId 71 from controller 4 epoch 53 for the become-follower transition for partition [test,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 completed LeaderAndIsr request correlationId 71 from controller 4 epoch 53 for the become-follower transition for partition [test3,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 completed LeaderAndIsr request correlationId 71 from controller 4 epoch 53 for the become-follower transition for partition [test,1] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 completed LeaderAndIsr request correlationId 71 from controller 4 epoch 53 for the become-follower transition for partition [test3,0] (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 71 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:8,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test3,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 71 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:5,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test,1] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 71 (state.change.logger)
      Jun 26 23:18:07 localhost kafka: TRACE Broker 7 cached leader info (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:14,ControllerEpoch:53),ReplicationFactor:1),AllReplicas:7) for partition [test3,0] in response to UpdateMetadata request sent by controller 4 epoch 53 with correlation id 71 (state.change.logger)
      
      1. controller2.log
        264 kB
        Albert Strasheim

        Activity

        Hide
        Guozhang Wang added a comment -

        Could you check if there is any exceptions/errors in the controller log?

        Show
        Guozhang Wang added a comment - Could you check if there is any exceptions/errors in the controller log?
        Hide
        Albert Strasheim added a comment -

        controller log. I tried a bunch of things (partition assignment, leader election) to make this fix itself, without luck.

        If there is some way to drop the broken topics by editing ZooKeeper without taking down this cluster, I'd love to know about it. I don't mind losing the topics (they were for testing).

        Show
        Albert Strasheim added a comment - controller log. I tried a bunch of things (partition assignment, leader election) to make this fix itself, without luck. If there is some way to drop the broken topics by editing ZooKeeper without taking down this cluster, I'd love to know about it. I don't mind losing the topics (they were for testing).
        Hide
        Albert Strasheim added a comment - - edited

        Workaround: patch up all the /brokers/topic/partitions/<part>/state entries in ZK with the leader and ISR values you want and then apply the same config with /usr/local/kafka/bin/kafka-preferred-replica-election.sh.

        Show
        Albert Strasheim added a comment - - edited Workaround: patch up all the /brokers/topic/partitions/<part>/state entries in ZK with the leader and ISR values you want and then apply the same config with /usr/local/kafka/bin/kafka-preferred-replica-election.sh.
        Hide
        Nicolae Marasoiu added a comment -

        Is a fix still needed for this, do you know?

        Show
        Nicolae Marasoiu added a comment - Is a fix still needed for this, do you know?
        Hide
        Albert Strasheim added a comment -

        Nicolae Marasoiu Yes, I think a fix is still needed.

        Show
        Albert Strasheim added a comment - Nicolae Marasoiu Yes, I think a fix is still needed.
        Hide
        Guozhang Wang added a comment -

        Yes this is still a valid issue, but could probably be a tricky issue also. I looked through the controller code, basically when a new broker startup the controller needs to try to use the offline-elector to elect the new leaders for those offline partitions which are hosted on the new broker to be also online partitions. But this process is somehow not executed, and instead the periodic preferred leader elector was executed later and failed the process since the new broker is not in the ISR yet.

        This could be correlated to some bugs in delete-topic logic, but more investigation is needed to find the right fix for this issue.

        Show
        Guozhang Wang added a comment - Yes this is still a valid issue, but could probably be a tricky issue also. I looked through the controller code, basically when a new broker startup the controller needs to try to use the offline-elector to elect the new leaders for those offline partitions which are hosted on the new broker to be also online partitions. But this process is somehow not executed, and instead the periodic preferred leader elector was executed later and failed the process since the new broker is not in the ISR yet. This could be correlated to some bugs in delete-topic logic, but more investigation is needed to find the right fix for this issue.

          People

          • Assignee:
            Unassigned
            Reporter:
            Albert Strasheim
          • Votes:
            1 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

            • Created:
              Updated:

              Development