linkedin/Burrow

Burrow fails to start with Kafka Version 2.8.x

arushi315 opened this issue · 9 comments

Hi team,
I am using Burrow 1.3.6 which works fine with Kafka Version 2.7 but when I upgrade Kafka to version 2.8.1, burrow fails to start with the following error:

{"level":"debug","ts":1642027415.5584846,"msg":"Connected to broker at uem-kafka:9092 (unregistered)","name":"sarama"}

{"level":"debug","ts":1642027415.5632129,"msg":"client/brokers registered new broker #0 at uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642027415.5634372,"msg":"Successfully initialized new client","name":"sarama"}

{"level":"debug","ts":1642027415.5635583,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642027415.5732005,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642027415.5733514,"msg":"client/metadata retrying after 250ms... (3 attempts remaining)","name":"sarama"}

{"level":"info","ts":1642027415.6499023,"msg":"starting evaluations","type":"coordinator","name":"notifier"}

{"level":"debug","ts":1642027415.8234534,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642027415.8260822,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642027415.8263423,"msg":"client/metadata retrying after 250ms... (2 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642027416.076764,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642027416.0788453,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642027416.0788953,"msg":"client/metadata retrying after 250ms... (1 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642027416.3291106,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642027416.3314283,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

Burrow failed at January 12, 2022 at 10:43pm (UTC)

{"level":"error","ts":1642027416.3314698,"msg":"failed to get partition count","type":"module","coordinator":"consumer","class":"kafka","name":"local-kafka","topic":"__consumer_offsets","error":"kafka server: Request was for a topic or partition that does not exist on this broker."}

{"level":"debug","ts":1642027416.3315206,"msg":"Closing Client","name":"sarama"}

{"level":"error","ts":1642027416.3315384,"msg":"failed to start consumer","type":"module","coordinator":"consumer","class":"kafka","name":"local-kafka","error":"kafka server: Request was for a topic or partition that does not exist on this broker."}

{"level":"debug","ts":1642027416.3315632,"msg":"Close() called on already closed client","name":"sarama"}

Sample of docker-compose.yml for Kafka and Zookeeper service:

version: '3.7'
services:
  uem-kafka:
    image: bitnami/kafka:2.8.1
    ports:
      - '9092:9092'
    user: "1001"
    environment:
      - ALLOW_PLAINTEXT_LISTENER=yes
      - KAFKA_CFG_BROKER_ID=0
      - KAFKA_CFG_NUM_PARTITIONS=3
      - KAFKA_CFG_ZOOKEEPER_CONNECT=uem-zookeeper:2181
      - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT
      - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=INSIDE
      - "KAFKA_CFG_LISTENERS=INSIDE://uem-kafka:9094,OUTSIDE://uem-kafka:9092"
      - "KAFKA_CFG_ADVERTISED_LISTENERS=INSIDE://uem-kafka:9094,OUTSIDE://uem-kafka:9092"
      - KAFKA_CFG_AUTO_CREATE_TOPICS_ENABLE=false
    depends_on:
      - uem-zookeeper

  uem-zookeeper:
    image: bitnami/zookeeper:3.6.2
    container_name: uem-zookeeper
    ports:
      - '2181:2181'
    environment:
     - ZOO_SERVER_ID=0
     - ZOO_HEAP_SIZE=100
     - ALLOW_ANONYMOUS_LOGIN=yes
     - ZOO_4LW_COMMANDS_WHITELIST="*"
     - ZOO_SERVERS='0.0.0.0:2888:3888'

Is burrow with Kafka Version 2.8.x supported? Any suggestions on how to troubleshoot this further?
Please let me know if there is any other log or configuration required.

Are there any consumer connected to the new cluster yet? __consumer_offsets topic is only created if there are any consumers.

Hi @mangatmodi Thank you for your quick response on this.

There is no other client talking to Kafka at this moment. I am simply bringing up the Zookeeper, Kafka and Burrow service.
I have tested this with Burrow version 1.3.8 as well and in the same setup Burrow service runs fine with Kafka Version 2.7 but fails with Kafka Version 2.8.1.

Note that I have KAFKA_CFG_AUTO_CREATE_TOPICS_ENABLE=false for both the runs.

I performed another test - I created a console consumer and started consuming from __consumer_offsets to see how Kafka reacts between 2.7 and 2.8.1.

Run with Kafka 2.7:
Screen Shot 2022-01-13 at 3 05 32 PM

Log snippet from Kafka:

[2022-01-13 21:04:47,372] INFO Creating topic __consumer_offsets with configuration {compression.type=producer, cleanup.policy=compact, segment.bytes=104857600} and initial partition assignment Map(23 -> ArrayBuffer(0), 32 -> ArrayBuffer(0), 41 -> ArrayBuffer(0), 17 -> ArrayBuffer(0), 8 -> ArrayBuffer(0), 35 -> ArrayBuffer(0), 44 -> ArrayBuffer(0), 26 -> ArrayBuffer(0), 11 -> ArrayBuffer(0), 29 -> ArrayBuffer(0), 38 -> ArrayBuffer(0), 47 -> ArrayBuffer(0), 20 -> ArrayBuffer(0), 2 -> ArrayBuffer(0), 5 -> ArrayBuffer(0), 14 -> ArrayBuffer(0), 46 -> ArrayBuffer(0), 49 -> ArrayBuffer(0), 40 -> ArrayBuffer(0), 13 -> ArrayBuffer(0), 4 -> ArrayBuffer(0), 22 -> ArrayBuffer(0), 31 -> ArrayBuffer(0), 16 -> ArrayBuffer(0), 7 -> ArrayBuffer(0), 43 -> ArrayBuffer(0), 25 -> ArrayBuffer(0), 34 -> ArrayBuffer(0), 10 -> ArrayBuffer(0), 37 -> ArrayBuffer(0), 1 -> ArrayBuffer(0), 19 -> ArrayBuffer(0), 28 -> ArrayBuffer(0), 45 -> ArrayBuffer(0), 27 -> ArrayBuffer(0), 36 -> ArrayBuffer(0), 18 -> ArrayBuffer(0), 9 -> ArrayBuffer(0), 21 -> ArrayBuffer(0), 48 -> ArrayBuffer(0), 3 -> ArrayBuffer(0), 12 -> ArrayBuffer(0), 30 -> ArrayBuffer(0), 39 -> ArrayBuffer(0), 15 -> ArrayBuffer(0), 42 -> ArrayBuffer(0), 24 -> ArrayBuffer(0), 6 -> ArrayBuffer(0), 33 -> ArrayBuffer(0), 0 -> ArrayBuffer(0)) (kafka.zk.AdminZkClient)

[2022-01-13 21:04:47,392] INFO [KafkaApi-0] Auto creation of topic __consumer_offsets with 50 partitions and replication factor 1 is successful (kafka.server.KafkaApis)

Run with Kafka 2.8.1:
Screen Shot 2022-01-13 at 3 09 46 PM

Log snippet from Kafka:

[2022-01-13 21:09:06,004] INFO Creating topic __consumer_offsets with configuration {compression.type=producer, cleanup.policy=compact, segment.bytes=104857600} and initial partition assignment Map(23 -> ArrayBuffer(0), 32 -> ArrayBuffer(0), 41 -> ArrayBuffer(0), 17 -> ArrayBuffer(0), 8 -> ArrayBuffer(0), 35 -> ArrayBuffer(0), 44 -> ArrayBuffer(0), 26 -> ArrayBuffer(0), 11 -> ArrayBuffer(0), 29 -> ArrayBuffer(0), 38 -> ArrayBuffer(0), 47 -> ArrayBuffer(0), 20 -> ArrayBuffer(0), 2 -> ArrayBuffer(0), 5 -> ArrayBuffer(0), 14 -> ArrayBuffer(0), 46 -> ArrayBuffer(0), 49 -> ArrayBuffer(0), 40 -> ArrayBuffer(0), 13 -> ArrayBuffer(0), 4 -> ArrayBuffer(0), 22 -> ArrayBuffer(0), 31 -> ArrayBuffer(0), 16 -> ArrayBuffer(0), 7 -> ArrayBuffer(0), 43 -> ArrayBuffer(0), 25 -> ArrayBuffer(0), 34 -> ArrayBuffer(0), 10 -> ArrayBuffer(0), 37 -> ArrayBuffer(0), 1 -> ArrayBuffer(0), 19 -> ArrayBuffer(0), 28 -> ArrayBuffer(0), 45 -> ArrayBuffer(0), 27 -> ArrayBuffer(0), 36 -> ArrayBuffer(0), 18 -> ArrayBuffer(0), 9 -> ArrayBuffer(0), 21 -> ArrayBuffer(0), 48 -> ArrayBuffer(0), 3 -> ArrayBuffer(0), 12 -> ArrayBuffer(0), 30 -> ArrayBuffer(0), 39 -> ArrayBuffer(0), 15 -> ArrayBuffer(0), 42 -> ArrayBuffer(0), 24 -> ArrayBuffer(0), 6 -> ArrayBuffer(0), 33 -> ArrayBuffer(0), 0 -> ArrayBuffer(0)) (kafka.zk.AdminZkClient)

In both the cases __consumer_offsets is eventually created.

Now coming back to Burrow (version 1.4.0) -

Run with Kafka 2.7:

{"level":"debug","ts":1642108786.2094376,"msg":"Connected to broker at uem-kafka:9092 (unregistered)","name":"sarama"}

{"level":"debug","ts":1642108786.214034,"msg":"client/brokers registered new broker #0 at uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642108786.2141097,"msg":"Successfully initialized new client","name":"sarama"}

{"level":"debug","ts":1642108786.214155,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642108786.2637825,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642108786.2638936,"msg":"client/metadata retrying after 250ms... (3 attempts remaining)","name":"sarama"}

{"level":"info","ts":1642108786.2936275,"msg":"starting evaluations","type":"coordinator","name":"notifier"}

{"level":"debug","ts":1642108786.5140655,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642108786.5219843,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642108786.522035,"msg":"client/metadata retrying after 250ms... (2 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642108786.7730803,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642108786.782377,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642108786.7824466,"msg":"client/metadata retrying after 250ms... (1 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642108787.0329359,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"info","ts":1642108787.0380993,"msg":"starting consumers","type":"module","coordinator":"consumer","class":"kafka","name":"local-kafka","topic":"__consumer_offsets","count":50}

__consumer_offsets is eventually created in Kafka.

Run with Kafka 2.8.1:

{"level":"debug","ts":1642109643.8580437,"msg":"Connected to broker at uem-kafka:9092 (unregistered)","name":"sarama"}

{"level":"debug","ts":1642109643.8627472,"msg":"client/brokers registered new broker #0 at uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642109643.8627968,"msg":"Successfully initialized new client","name":"sarama"}

{"level":"debug","ts":1642109643.862842,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642109643.8712587,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642109643.8713126,"msg":"client/metadata retrying after 250ms... (3 attempts remaining)","name":"sarama"}

{"level":"info","ts":1642109643.9436781,"msg":"starting evaluations","type":"coordinator","name":"notifier"}

{"level":"debug","ts":1642109644.1217878,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642109644.1244755,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642109644.1245284,"msg":"client/metadata retrying after 250ms... (2 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642109644.3755918,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642109644.3778868,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"debug","ts":1642109644.377944,"msg":"client/metadata retrying after 250ms... (1 attempts remaining)","name":"sarama"}

{"level":"debug","ts":1642109644.6290717,"msg":"client/metadata fetching metadata for [__consumer_offsets] from broker uem-kafka:9092","name":"sarama"}

{"level":"debug","ts":1642109644.6317284,"msg":"client/metadata found some partitions to be leaderless","name":"sarama"}

{"level":"error","ts":1642109644.6317704,"msg":"failed to get partition count","type":"module","coordinator":"consumer","class":"kafka","name":"local-kafka","topic":"__consumer_offsets","error":"kafka server: Request was for a topic or partition that does not exist on this broker."}

{"level":"debug","ts":1642109644.631847,"msg":"Closing Client","name":"sarama"}

No log in Kafka about __consumer_offsets and never gets created.

Note I am pretty new to using Burrow so not quite sure if this is something we need to look in Burrow or Sarama at this point.

Hi team,
Any comment of this issue or any suggestion on how to troubleshoot this further?
Please let me know if any other info is required.

dnwe commented

As per IBM/sarama#2115 (comment)

Yes looks likely to be due to the change in behaviour in Kafka 2.8.0 onward KAFKA-9751 whereby the broker is asynchronously forwarding the auto create of __consumer_offsets onto the controller. So whereas in the past the topic creation would be attempted immediately and would just return LEADER_NOT_AVAILABLE until the partitions were created and leaders assigned, now it will instead keep returning UNKNOWN_TOPIC_OR_PARTITION until the topic has begun creation via the coordinator

So this is something we will want to look at addressing in Sarama

For Burrow this is only a problem when run against a completely brand new 2.8.x cluster that has never had any consumer group attached to it, because it wants to determine the number of partitions on the __consumer_offsets topic before initialising its per-partition consumers:

// Get a partition count for the consumption topic
partitions, err := client.Partitions(module.offsetsTopic)
if err != nil {
module.Log.Error("failed to get partition count",
zap.String("topic", module.offsetsTopic),
zap.String("error", err.Error()),
)
client.Close()
return err
}

Theoretically burrow could workaround the issue by not treating a single client.Partitions failure as fatal and instead adding a small number of backoff retries around that call

dnwe commented

Hmm, having investigated this some more it seems like a change in behaviour on the broker in 2.8.x onward means that __consumer_offsets is not getting created by a MetadataRequest alone and it's only a JoinGroup from a consumer group that causes it to spring into life

Thank you so much @dnwe for analyzing this and sharing the details.

You are right, we are observing this issue only for a brand new cluster.
As a workaround for the time being, we are thinking of adding a pre-task before starting Burrow. This pre-task would use console consumer and start consuming to __consumer_offsets topic until its created.

Let me know if there is anything I can share to investigate this further on Burrow or Sarama.

dnwe commented

Kafka dev mailing list agree this is a regression in the broker.

https://lists.apache.org/thread/cxzq6009t2vsyc05tcro77mov12zhpvm

Raised upstream as https://issues.apache.org/jira/browse/KAFKA-13612

The only workaround for Burrow would be to actively join a consumer group rather than using the old style consumer

Thanks @dnwe for looking into this and raising the ticket with Kafka, I will keep an eye on that.