Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

the machine of producer 's network io is 100MB/s #1313

Closed
2 of 8 tasks
andrewrong opened this issue Jul 10, 2017 · 14 comments
Closed
2 of 8 tasks

the machine of producer 's network io is 100MB/s #1313

andrewrong opened this issue Jul 10, 2017 · 14 comments

Comments

@andrewrong
Copy link

andrewrong commented Jul 10, 2017

Description

When the broker is unstable because the fullgc takes too long time, the machine of producer 's network io is 100MB/s, this is so big and it's not reasonable;

my producer config is

batch.num.messages:1000
queue.buffering.max.ms: 200ms
queue.buffering.max.kbytes: 100 * 1024
queue.buffering.max.messages:100000
metadata.broker.list:broker ip
message.max.bytes: 1024 * 1023
socket.blocking.max.ms: 200
broker.version.fallback: 0.8.2.0
message.send.max.retries: 3
max.in.flight.requests.per.connection: 1000
request.required.acks: 1

My broker's log

controller log:

[2017-07-06 15:11:07,583] INFO [SessionExpirationListener on 723], ZK expired; shut down all controller components and try to re-elect (kafka.controller.KafkaController$SessionExpirationListener)

state-change.log

[2017-07-06 15:11:22,629] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:726,ISR:726,LeaderEpoch:2,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:726,728) for partition [antispam_trustlevel,0] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:728,ISR:713,728,LeaderEpoch:20,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:728,713) for partition [royalty_info_log,2] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:713,ISR:713,LeaderEpoch:26,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:713,711) for partition [forum_buildcms_log,3] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:724,ISR:724,LeaderEpoch:198,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:723,724) for partition [pay_order_schedule_info,2] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:2081,ISR:2081,729,LeaderEpoch:0,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:2081,729) for partition [acm_etl_order,0] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:107048,ISR:107050,107048,LeaderEpoch:11,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:107048,107050) for partition [spot_http_deviceEvent_log,0] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)
[2017-07-06 15:11:22,788] TRACE Broker 723 cached leader info (LeaderAndIsrInfo:(Leader:726,ISR:729,726,LeaderEpoch:17,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:726,729) for partition [mobisqlstat_log,2] in response to UpdateMetadata request sent by controller 107049 epoch 19 with correlation id 140127 (state.change.logger)

server.log

[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 1399 from client LogAgent_default on partition [trade_timeout_service_stdout,0] failed due to Leader not local for partition [trade_timeout_service_stdout,0] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 840495808 from client on partition [mario_etl_acm_expose_v1,2] failed due to Leader not local for partition [mario_etl_acm_expose_v1,2] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 840473960 from client on partition [mario_etl_ump,2] failed due to Leader not local for partition [mario_etl_ump,2] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 836965689 from client on partition [mario_etl_mobile_page,0] failed due to Leader not local for partition [mario_etl_mobile_page,0] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 836993146 from client on partition [mario_etl_acm_expose_v1,2] failed due to Leader not local for partition [mario_etl_acm_expose_v1,2] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 836966136 from client on partition [mario_etl_mobile_page,0] failed due to Leader not local for partition [mario_etl_mobile_page,0] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [KafkaApi-723] Produce request with correlation id 836974316 from client on partition [mario_etl_acm_expose_v1,2] failed due to Leader not local for partition [mario_etl_acm_expose_v1,2] on broker 723 (kafka.server.KafkaApis)
[2017-07-06 15:11:23,517] WARN [Replica Manager on Broker 723]: Fetch request with correlation id 723374 from client cube_for_etl_gmv on partition [order_refer_v6.1,3] failed due to Leader not local for partition [order_refer_v6.1,3] on broker 723 (kafka.server.ReplicaManager)

My machine monitor about the network io

06/07/17-15:14 4.5M 9.1M 8.2K 9.4K 0.00 0.00
06/07/17-15:15 5.1M 39.3M 15.4K 30.0K 0.00 0.00
06/07/17-15:16 5.2M 39.0M 16.0K 29.9K 0.00 0.00
06/07/17-15:17 5.2M 41.2M 16.5K 31.3K 0.00 0.00
06/07/17-15:18 5.6M 74.5M 22.2K 54.0K 0.00 0.00
06/07/17-15:19 6.0M 109.7M 26.9K 77.6K 0.00 0.00
06/07/17-15:20 4.6M 13.7M 8.8K 12.6K 0.00 0.00
06/07/17-15:21 4.6M 8.7M 8.0K 9.2K 0.00 0.00
06/07/17-15:22 4.6M 8.7M 7.9K 9.2K 0.00 0.00
06/07/17-15:23 4.6M 8.6M 7.9K 9.2K 0.00 0.00
06/07/17-15:24 4.7M 8.9M 8.1K 9.4K 0.00 0.00
06/07/17-15:25 4.5M 8.6M 7.8K 9.1K 0.00 0.00
06/07/17-15:26 4.7M 8.9M 8.1K 9.4K 0.00 0.00
06/07/17-15:27 4.5M 8.6M 7.8K 9.1K 0.00 0.00
06/07/17-15:28 4.6M 8.8M 8.0K 9.3K 0.00 0.00
06/07/17-15:29 4.5M 8.5M 7.8K 9.0K 0.00 0.00
06/07/17-15:30 4.6M 8.7M 7.9K 9.2K 0.00 0.00
06/07/17-15:31 4.6M 8.7M 8.0K 9.2K 0.00 0.00
06/07/17-15:32 4.6M 8.8M 8.1K 9.3K 0.00 0.00

How to reproduce

Checklist

Please provide the following information:

  • librdkafka version (release number or git tag): 0.9.5
  • Apache Kafka version: 0.8.2.2
  • librdkafka client configuration:
  • Operating system: centos 6.5
  • Using the legacy Consumer
  • Using the high-level KafkaConsumer
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • [* ] Critical issue
@edenhill
Copy link
Contributor

Set the debug property to protocol to see what requests the client is sending.

@andrewrong
Copy link
Author

@edenhill the producer's debug log when I reboot my broker(10.17.107.11),

[2017-07-12 15:23:57.671260 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 0+69504/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.671562 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 69504+62264/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.689253 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Received ProduceResponse (v0, 51 bytes, CorrId 6225, rtt 19.33ms)
[2017-07-12 15:23:57.689683 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 131768+85432/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.690066 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Received ProduceResponse (v0, 51 bytes, CorrId 6226, rtt 19.00ms)
[2017-07-12 15:23:57.690273 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 217200+55024/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.691125 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 272224+52128/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.691484 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 324352+39096/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.691832 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 363448+46336/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.692176 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 409784+39096/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.692590 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 448880+50680/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.692958 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 499560+46336/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.693317 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 545896+46336/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.693709 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 592232+43440/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.694107 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 635672+53576/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.694541 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 689248+53576/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.694941 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 742824+50680/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.695373 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 793504+46336/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.695798 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 839840+50680/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.696217 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 890520+53576/987730 bytes, CorrId 6227)
[2017-07-12 15:23:57.696646 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent ProduceRequest (v0, 987730 bytes @ 944096, CorrId 6227)
[2017-07-12 15:23:57.696845 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 0+31662/97204 bytes, CorrId 6228)
[2017-07-12 15:23:57.697213 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 31662+46336/97204 bytes, CorrId 6228)
[2017-07-12 15:23:57.697585 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent ProduceRequest (v0, 97204 bytes @ 77998, CorrId 6228)
[2017-07-12 15:23:57.697975 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 0+53194/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.698225 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 53194+53576/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.698637 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 106770+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.699081 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 157450+53576/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.699476 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Received ProduceResponse (v0, 51 bytes, CorrId 6227, rtt 2.68ms)
[2017-07-12 15:23:57.700094 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Received ProduceResponse (v0, 51 bytes, CorrId 6228, rtt 2.37ms)
[2017-07-12 15:23:57.700541 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 211026+143352/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.700957 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 354378+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.701364 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 412298+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.701828 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 462978+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.702288 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 513658+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.702750 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 571578+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.703234 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 622258+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.703702 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 680178+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.704157 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 738098+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.704674 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 788778+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.705117 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 846698+57920/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.705581 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent partial ProduceRequest (v0, 904618+50680/976146 bytes, CorrId 6229)
[2017-07-12 15:23:57.706017 info] (/home/fenglin/LogAgent/src/biz/sender/Sender.h:39)[thrd:10.17.107.11:9092/711]: 10.17.107.11:9092/711: Sent ProduceRequest (v0, 976146 bytes @ 955298, CorrId 6229)

And the machine of producer's network io is 103MB/s

@andrewrong
Copy link
Author

@edenhill why the produceRequest is so much and the produceRequest corrid is same?

@edenhill
Copy link
Contributor

The socket is congested (probably due to slow broker) so the client can only send about 50k per attempt, thus the "Sent partial ProduceRequest".

@andrewrong
Copy link
Author

I found a phenomenon; I stop a broker, the machine of producer's network io increased

12/07/17-17:16:19 4.5M 8.6M 8.5K 9.8K 0.00 0.00
12/07/17-17:16:24 5.0M 36.0M 15.1K 27.9K 0.00 0.00
12/07/17-17:16:29 5.3M 56.0M 20.3K 41.6K 0.00 0.00
12/07/17-17:16:34 5.1M 48.1M 18.2K 36.2K 0.00 0.00
12/07/17-17:16:39 5.4M 58.7M 21.0K 43.5K 0.00 0.00
12/07/17-17:16:44 5.4M 63.3M 20.9K 46.6K 0.00 0.00
12/07/17-17:16:49 5.5M 61.1M 21.6K 45.1K 0.00 0.00
12/07/17-17:16:54 5.4M 60.2M 21.4K 44.4K 0.00 0.00
12/07/17-17:16:59 5.4M 59.9M 21.5K 44.2K 0.00 0.00
12/07/17-17:17:04 5.5M 60.2M 21.9K 44.5K 0.00 0.00
12/07/17-17:17:09 5.4M 60.1M 21.7K 44.4K 0.00 0.00
12/07/17-17:17:14 5.3M 60.2M 21.8K 44.3K 0.00 0.00

when i start the broker before closing, the network io is not suddenly increased,

[2017-07-12 17:18:27,222] INFO [Kafka Server 711], starting (kafka.server.KafkaServer)

image

the debug log is 200MB, this is so big, and the kafka's in network io also is increased;

@edenhill
Copy link
Contributor

The only way to know what is going on is checking the protocol debug, try to correlate how many bytes are sent to which broker when you are seeing this rate increase.
It could be that another broker is elected leader and the client sends retries to the new broker.

@andrewrong
Copy link
Author

@edenhill I find that rd_kafka_metadata_fast_leader_query is invalid

this is my debug log;

rdkafka.tar.gz

  1. "Starting fast leader query" appear per second,but it doesn't trigger the metedata request
  2. the last metadata request is triggered because periodic refresh, not the fast leader query

image

why????

@andrewrong
Copy link
Author

@edenhill I debug my program that I add some logs in your library; I find two problems
[thrd:10.15.10.49:9092/bootstrap]: 10.15.10.49: 9092/49: MessageSet with 999 message(s) encountered error: Broker: Not leader for partition (actions 0x4) this log shows

image

image

image

My debug is no_leader = 0, I don't know why?

@andrewrong
Copy link
Author

@edenhill Please give me a reply,thank you

@edenhill
Copy link
Contributor

Thank you for your detailed analysis, I will look into this soon.

@andrewrong
Copy link
Author

@edenhill Do you have any progress on this question?

@edenhill
Copy link
Contributor

Thanks for a great analysis, you are right on point:

  • toppar_leader_unavailable() sets the UNAVAILABLE flag but not does clear the rktp_leader pointer
  • rd_kafka_metadata_leader_query_tmr_cb() indeed checks the UNAVAILABLE flag, but then scans through the topic's partitions and sets no_leader again based on the partition's leader pointer, thus resulting in no_leader being set to 0 in this case.

The bug is the for-loop conditional which is i < rkt->rkt_partition_cnt || no_leader but should be !no_leader && i < rkt->rkt_partition_cnt

@edenhill
Copy link
Contributor

Can you try to reproduce this issue with the fix in place (get master branch)?

@andrewrong
Copy link
Author

@edenhill thank you very much, I will try it now;

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants