Kafka
  1. Kafka
  2. KAFKA-1365

Second Manual preferred replica leader election command always fails

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Critical Critical
    • Resolution: Fixed
    • Affects Version/s: 0.8.1
    • Fix Version/s: 0.8.1.1
    • Component/s: controller, tools
    • Labels:
      None

      Description

      After running kafka-preferred-replica-election.sh once, a second run will fail with "Preferred replica leader election currently in progress for ...".

      The /admin/preferred_replica_election key is never deleted from ZooKeeper, because the "isTriggeredByAutoRebalance" parameter to onPreferredReplicaElection (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L614) is used incorrectly. In the automatic case (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L1119), it is set to false. In the manual case (https://github.com/apache/kafka/blob/0ffec142a991849833d9767be07e895428ccaea1/core/src/main/scala/kafka/controller/KafkaController.scala#L1266) the parameter is not passed, so it defaults to true.

      1. KAFKA-1365.patch
        1 kB
        Guozhang Wang

        Issue Links

          Activity

          Hide
          Joel Koshy added a comment -

          Thanks for filing this issue and for debugging it. We will fix this in
          0.8.2. The automatic preferred replica leader election feature has other
          known issues (e.g., KAFKA-1305) that we won't get to in 0.8.1.1 so it should
          turned off (default) until 0.8.2.

          Show
          Joel Koshy added a comment - Thanks for filing this issue and for debugging it. We will fix this in 0.8.2. The automatic preferred replica leader election feature has other known issues (e.g., KAFKA-1305 ) that we won't get to in 0.8.1.1 so it should turned off (default) until 0.8.2.
          Hide
          Balaji Seshadri added a comment - - edited

          I can help fix this as its happening for us in DISH Upgrade,please direct me accordingly.

          Please see email from Bob.

          ----Original Message----
          From: Bello, Bob Bob.Bello@dish.com
          Sent: Tuesday, April 15, 2014 10:00 AM
          To: users@kafka.apache.org
          Cc: Bello, Bob
          Subject: RE: Kafka upgrade 0.8.0 to 0.8.1 - kafka-preferred-replica-election failure

          I performed another test. I build a single Kafka 0.8.1 Broker with a single ZK instance. Brand new, no topics.

          Upon start up of the Kafka broker, the zookeeper /admin node only contains "/admin/delete_topics".

          Even without creating a topic, I run a perfered replica election, and it's successful. After the run, I check the node for /admin/preferred_replica_election and it exists.

          [zk: tm1mwwm001:2181(CONNECTED) 0] get /admin/preferred_replica_election

          {"version":1,"partitions":[]}

          cZxid = 0x19 ctime = Tue Apr 15 09:53:22 MDT 2014 mZxid = 0x19 mtime = Tue Apr 15 09:53:22 MDT 2014 pZxid = 0x19 cversion = 0 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 29 numChildren = 0

          If I run the election again, I get the following error (the same error as my original post).

          Failed to start preferred replica election
          kafka.common.AdminCommandFailedException: Admin command failed
          at kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:115)
          at kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
          at kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)
          Caused by: kafka.admin.AdminOperationException: Preferred replica leader election currently in progress for Set(). Aborting operation
          at kafka.admin.PreferredReplicaLeaderElectionCommand$.writePreferredReplicaElectionData(PreferredReplicaLeaderElectionCommand.scala:101)
          at kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:113)
          ... 2 more

          The zookeeper log shows the following:

          2014-04-15 09:53:44 INFO server.PrepRequestProcessor - Got user-level KeeperException when processing sessionid:0x1456615e7770006 type:create cxid:0x2 zxid:0x1e txntype:-1 reqpath:n/a Error Path:/admin/preferred_replica_election Error:KeeperErrorCode = NodeExists for /admin/preferred_replica_election

          It looks like the preferred election process is not removing the node after completion.

          Bob Bello
          Middleware Applications Administrator
          Direct: 720-514-6605
          email: bob.bello@dish.com

          Show
          Balaji Seshadri added a comment - - edited I can help fix this as its happening for us in DISH Upgrade,please direct me accordingly. Please see email from Bob. ---- Original Message ---- From: Bello, Bob Bob.Bello@dish.com Sent: Tuesday, April 15, 2014 10:00 AM To: users@kafka.apache.org Cc: Bello, Bob Subject: RE: Kafka upgrade 0.8.0 to 0.8.1 - kafka-preferred-replica-election failure I performed another test. I build a single Kafka 0.8.1 Broker with a single ZK instance. Brand new, no topics. Upon start up of the Kafka broker, the zookeeper /admin node only contains "/admin/delete_topics". Even without creating a topic, I run a perfered replica election, and it's successful. After the run, I check the node for /admin/preferred_replica_election and it exists. [zk: tm1mwwm001:2181(CONNECTED) 0] get /admin/preferred_replica_election {"version":1,"partitions":[]} cZxid = 0x19 ctime = Tue Apr 15 09:53:22 MDT 2014 mZxid = 0x19 mtime = Tue Apr 15 09:53:22 MDT 2014 pZxid = 0x19 cversion = 0 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 29 numChildren = 0 If I run the election again, I get the following error (the same error as my original post). Failed to start preferred replica election kafka.common.AdminCommandFailedException: Admin command failed at kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:115) at kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60) at kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala) Caused by: kafka.admin.AdminOperationException: Preferred replica leader election currently in progress for Set(). Aborting operation at kafka.admin.PreferredReplicaLeaderElectionCommand$.writePreferredReplicaElectionData(PreferredReplicaLeaderElectionCommand.scala:101) at kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:113) ... 2 more The zookeeper log shows the following: 2014-04-15 09:53:44 INFO server.PrepRequestProcessor - Got user-level KeeperException when processing sessionid:0x1456615e7770006 type:create cxid:0x2 zxid:0x1e txntype:-1 reqpath:n/a Error Path:/admin/preferred_replica_election Error:KeeperErrorCode = NodeExists for /admin/preferred_replica_election It looks like the preferred election process is not removing the node after completion. Bob Bello Middleware Applications Administrator Direct: 720-514-6605 email: bob.bello@dish.com
          Hide
          Joel Koshy added a comment -

          Adding back 0.8.1.1

          Show
          Joel Koshy added a comment - Adding back 0.8.1.1
          Hide
          Guozhang Wang added a comment -

          Created reviewboard https://reviews.apache.org/r/20424/
          against branch origin/0.8.1

          Show
          Guozhang Wang added a comment - Created reviewboard https://reviews.apache.org/r/20424/ against branch origin/0.8.1
          Hide
          Guozhang Wang added a comment -

          Has double checked that this issue has been fixed in trunk. Submitted a patch for 0.8.1 only.

          Show
          Guozhang Wang added a comment - Has double checked that this issue has been fixed in trunk. Submitted a patch for 0.8.1 only.
          Hide
          Joel Koshy added a comment -

          Thanks for the patch. Committed to 0.8.1

          I tested locally as well, but Balaji Seshadri do you want to give this a spin and confirm?

          Show
          Joel Koshy added a comment - Thanks for the patch. Committed to 0.8.1 I tested locally as well, but Balaji Seshadri do you want to give this a spin and confirm?
          Hide
          BalajiSeshadri added a comment -

          We stopped the upgrade so we will not be able to test this.

          Thanks,

          Balaji

          Show
          BalajiSeshadri added a comment - We stopped the upgrade so we will not be able to test this. Thanks, Balaji
          Hide
          BalajiSeshadri added a comment -

          We will test the controller issue one in dev ,we will test this also at that time.

          Sorry for that.

          Show
          BalajiSeshadri added a comment - We will test the controller issue one in dev ,we will test this also at that time. Sorry for that.

            People

            • Assignee:
              Neha Narkhede
              Reporter:
              Ryan Berdeen
            • Votes:
              2 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development