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

Seeing occasional segfaults with latest release #4195

Closed
4 of 7 tasks
wmorgan6796 opened this issue Feb 17, 2023 · 5 comments · Fixed by #4254
Closed
4 of 7 tasks

Seeing occasional segfaults with latest release #4195

wmorgan6796 opened this issue Feb 17, 2023 · 5 comments · Fixed by #4254

Comments

@wmorgan6796
Copy link

wmorgan6796 commented Feb 17, 2023

Description

We've recently done a large scale rollout of librdkafka 2.0.2 (1000+ nodes) and we're seeing occasional segmentation faults on the library. We've noticed the segmentation faults seem to occur quite a bit more when there's heavy rebalances going on in the cluster for a consumer group.

Some important notes:
Our services are 100% modern C++17 compiled statically using GCC 12.2 on Linux Ubuntu Focal.

Stack trace here:

#0  __GI_raise (sig=sig@entry=6) at ../sysdeps/unix/sysv/linux/raise.c:50
#1  0x00007f965dc9b859 in __GI_abort () at abort.c:79
#2  0x00000000004b9294 in rd_kafka_crash.cold () at /usr/include/boost/throw_exception.hpp:70
#3  0x0000000001175999 in rd_kafka_toppar_set_fetch_state ()
#4  0x0000000001175a77 in rd_kafka_toppar_offset_retry ()
#5  0x0000000001177d74 in rd_kafka_toppar_broker_delegate ()
#6  0x00000000011bedb5 in rd_kafka_toppar_broker_update ()
#7  0x00000000011beee4 in rd_kafka_toppar_delegate_to_leader ()
#8  0x000000000120f8cd in rd_kafka_toppar_fetch_decide ()
#9  0x00000000011349a7 in rd_kafka_broker_consumer_serve ()
#10 0x0000000001135164 in rd_kafka_broker_serve ()
#11 0x00000000011357dd in rd_kafka_broker_thread_main ()
#12 0x00007f965ee1b6da in start_thread (arg=<optimized out>) at pthread_create.c:474
#13 0x00007f965dd98133 in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95

How to reproduce

I have not been able to reliable reproduce the issue locally and therefore don't have much more to go on then what is above from the segfault. I'm not sure if there is something wrong with the library or if its a smoking gun elsewhere.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

  • librdkafka version (release number or git tag): v2.0.2
  • Apache Kafka version: 2.8.1
  • librdkafka client configuration: <REPLACE with e.g., message.timeout.ms=123, auto.reset.offset=earliest, ..>
  • Operating system: Ubuntu 20.04
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
    Global Consumer Config
===== Global Kafka Consumer Config =====

	allow.auto.create.topics = false
	api.version.fallback.ms = 0
	api.version.request = true
	api.version.request.timeout.ms = 10000
	auto.commit.interval.ms = 5000
	batch.num.messages = 10000
	batch.size = 1000000
	broker.address.family = any
	broker.address.ttl = 1000
	broker.version.fallback = 0.10.0
	builtin.features = gzip,snappy,ssl,sasl,regex,lz4,sasl_gssapi,sasl_plain,sasl_scram,zstd,sasl_oauthbearer
	check.crcs = false
	client.id = <REDACTED>
	client.rack = ofb1
	client.software.name = librdkafka
	compression.codec = none
	connections.max.idle.ms = 0
	coordinator.query.interval.ms = 600000
	debug =
	default_topic_conf = 0x7f09fb27c600
	delivery.report.only.error = false
	enable.auto.commit = false
	enable.auto.offset.store = false
	enable.gapless.guarantee = false
	enable.idempotence = false
	enable.partition.eof = false
	enable.random.seed = true
	enable.sasl.oauthbearer.unsecure.jwt = false
	enable.sparse.connections = true
	enable.ssl.certificate.verification = true
	enable_sasl_queue = false
	enabled_events = 0
	fetch.error.backoff.ms = 500
	fetch.max.bytes = 52428800
	fetch.message.max.bytes = 1048576
	fetch.min.bytes = 1
	fetch.wait.max.ms = 500
	group.id = <REDACTED>
	group.protocol.type = consumer
	heartbeat.interval.ms = 3000
	internal.termination.signal = 0
	isolation.level = read_committed
	log.connection.close = true
	log.queue = false
	log.thread.name = true
	log_cb = 0x110dab0
	log_level = 6
	max.in.flight.requests.per.connection = 1000000
	max.poll.interval.ms = 300000
	message.copy.max.bytes = 65535
	message.max.bytes = 1000000
	message.send.max.retries = 2147483647
	metadata.broker.list = <REDACTED>
	metadata.max.age.ms = 900000
	metadata.request.timeout.ms = 10
	offset.store.method = broker
	open_cb = 0x1169bf0
	partition.assignment.strategy = range,roundrobin
	queue.buffering.backpressure.threshold = 1
	queue.buffering.max.kbytes = 1048576
	queue.buffering.max.messages = 100000
	queue.buffering.max.ms = 5
	queued.max.messages.kbytes = 65536
	queued.min.messages = 100000
	receive.message.max.bytes = 100000000
	reconnect.backoff.jitter.ms = 0
	reconnect.backoff.max.ms = 10000
	reconnect.backoff.ms = 100
	retry.backoff.ms = 100
	sasl.kerberos.kinit.cmd = kinit -R -t "%{sasl.kerberos.keytab}" -k %{sasl.kerberos.principal} || kinit -t "%{sasl.kerberos.keytab}" -k %{sasl.kerberos.principal}
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.principal = kafkaclient
	sasl.kerberos.service.name = kafka
	sasl.mechanisms = GSSAPI
	sasl.oauthbearer.method = default
	security.protocol = plaintext
	session.timeout.ms = 45000
	socket.blocking.max.ms = 1000
	socket.connection.setup.timeout.ms = 30000
	socket.keepalive.enable = false
	socket.max.fails = 1
	socket.nagle.disable = false
	socket.receive.buffer.bytes = 0
	socket.send.buffer.bytes = 0
	socket.timeout.ms = 60000
	socket_cb = 0x1128550
	ssl.ca.certificate.stores = Root
	ssl.endpoint.identification.algorithm = https
	ssl.engine.id = dynamic
	statistics.interval.ms = 300000
	sticky.partitioning.linger.ms = 10
	test.mock.broker.rtt = 0
	test.mock.num.brokers = 0
	topic.metadata.propagation.max.ms = 30000
	topic.metadata.refresh.fast.cnt = 10
	topic.metadata.refresh.fast.interval.ms = 250
	topic.metadata.refresh.interval.ms = 300000
	topic.metadata.refresh.sparse = true
	transaction.timeout.ms = 60000

Topic Consumer Config

===== Topic Kafka Consumer Config =====

	auto.commit.enable = true
	auto.commit.interval.ms = 60000
	auto.offset.reset = largest
	compression.codec = inherit
	compression.level = -1
	consume.callback.max.messages = 0
	message.timeout.ms = 300000
	offset.store.method = broker
	offset.store.path = .
	offset.store.sync.interval.ms = -1
	partitioner = consistent_random
	produce.offset.report = false
	queuing.strategy = fifo
	request.required.acks = -1
	request.timeout.ms = 30000

Producer Config

===== Global Kafka Producer Config =====

	allow.auto.create.topics = false
	api.version.fallback.ms = 0
	api.version.request = true
	api.version.request.timeout.ms = 10000
	auto.commit.interval.ms = 5000
	batch.num.messages = 1000000
	batch.size = 1000000
	broker.address.family = any
	broker.address.ttl = 1000
	broker.version.fallback = 0.10.0
	builtin.features = gzip,snappy,ssl,sasl,regex,lz4,sasl_gssapi,sasl_plain,sasl_scram,zstd,sasl_oauthbearer
	check.crcs = false
	client.id = <REDACTED>
	client.rack =
	client.software.name = librdkafka
	compression.codec = lz4
	connections.max.idle.ms = 0
	coordinator.query.interval.ms = 600000
	debug =
	default_topic_conf = 0x7f09fb27c480
	delivery.report.only.error = false
	enable.auto.commit = true
	enable.auto.offset.store = true
	enable.gapless.guarantee = false
	enable.idempotence = false
	enable.partition.eof = false
	enable.random.seed = true
	enable.sasl.oauthbearer.unsecure.jwt = false
	enable.sparse.connections = true
	enable.ssl.certificate.verification = true
	enable_sasl_queue = false
	enabled_events = 0
	fetch.error.backoff.ms = 500
	fetch.max.bytes = 52428800
	fetch.message.max.bytes = 1048576
	fetch.min.bytes = 1
	fetch.wait.max.ms = 500
	group.protocol.type = consumer
	heartbeat.interval.ms = 3000
	internal.termination.signal = 0
	isolation.level = read_committed
	log.connection.close = true
	log.queue = false
	log.thread.name = true
	log_cb = 0x110dab0
	log_level = 6
	max.in.flight.requests.per.connection = 1000000
	max.poll.interval.ms = 300000
	message.copy.max.bytes = 65535
	message.max.bytes = 1000000
	message.send.max.retries = 2147483647
	metadata.broker.list = <REDACTED>
	metadata.max.age.ms = 900000
	metadata.request.timeout.ms = 10
	offset.store.method = broker
	open_cb = 0x1169bf0
	partition.assignment.strategy = range,roundrobin
	queue.buffering.backpressure.threshold = 1
	queue.buffering.max.kbytes = 8000000
	queue.buffering.max.messages = 500000
	queue.buffering.max.ms = 5000
	queued.max.messages.kbytes = 65536
	queued.min.messages = 100000
	receive.message.max.bytes = 100000000
	reconnect.backoff.jitter.ms = 0
	reconnect.backoff.max.ms = 10000
	reconnect.backoff.ms = 100
	retry.backoff.ms = 100
	sasl.kerberos.kinit.cmd = kinit -R -t "%{sasl.kerberos.keytab}" -k %{sasl.kerberos.principal} || kinit -t "%{sasl.kerberos.keytab}" -k %{sasl.kerberos.principal}
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.principal = kafkaclient
	sasl.kerberos.service.name = kafka
	sasl.mechanisms = GSSAPI
	sasl.oauthbearer.method = default
	security.protocol = plaintext
	session.timeout.ms = 45000
	socket.blocking.max.ms = 1000
	socket.connection.setup.timeout.ms = 30000
	socket.keepalive.enable = false
	socket.max.fails = 1
	socket.nagle.disable = false
	socket.receive.buffer.bytes = 0
	socket.send.buffer.bytes = 0
	socket.timeout.ms = 60000
	socket_cb = 0x1128550
	ssl.ca.certificate.stores = Root
	ssl.endpoint.identification.algorithm = https
	ssl.engine.id = dynamic
	statistics.interval.ms = 300000
	sticky.partitioning.linger.ms = 10
	test.mock.broker.rtt = 0
	test.mock.num.brokers = 0
	topic.metadata.propagation.max.ms = 30000
	topic.metadata.refresh.fast.cnt = 10
	topic.metadata.refresh.fast.interval.ms = 250
	topic.metadata.refresh.interval.ms = 300000
	topic.metadata.refresh.sparse = true
	transaction.timeout.ms = 60000


===== Topic Kafka Producer Config =====

	auto.commit.enable = true
	auto.commit.interval.ms = 60000
	auto.offset.reset = largest
	compression.codec = inherit
	compression.level = -1
	consume.callback.max.messages = 0
	message.timeout.ms = 300000
	offset.store.method = broker
	offset.store.path = .
	offset.store.sync.interval.ms = -1
	partitioner = consistent_random
	produce.offset.report = false
	queuing.strategy = fifo
	request.required.acks = 1
	request.timeout.ms = 30000
@emasab
Copy link
Contributor

emasab commented Feb 20, 2023

@wmorgan6796 Thanks for the report!
This assert is failing

        rd_kafka_assert(NULL,
                        thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread));

It's because rd_kafka_toppar_set_fetch_state isn't called from the main thread but from the toppar thread instead, in rd_kafka_toppar_offset_retry. We're going to fix this.

@emasab emasab added the bug label Feb 23, 2023
@wmorgan6796
Copy link
Author

@emasab whats the criticality of this bug? If the fix isn't already in progress, would like to take a crack at it so we can fix the seg faults occuring in our environments

@emasab
Copy link
Contributor

emasab commented Apr 13, 2023

@wmorgan6796 how often does it occur? Could you get the a debug log with "debug=consumer,cgrp,topic,fetch"? I have clear how to fix it, but it's difficult to create a test that fails this way so it could be helpful to have the sequence of events.

The scenario is that the follower lease expires and the consumer migrates back to the leader, this happens while it was doing a ListOffsets to get the earliest or latest offset, so it calls rd_kafka_toppar_offset_retry to retry the ListOffsets on the new broker.

@emasab
Copy link
Contributor

emasab commented Apr 13, 2023

Could reproduce it using a seek to latest while fetching from follower, while manually reducing rktp_lease_intvl

emasab added a commit that referenced this issue Apr 13, 2023
lease expires and the partition is waiting for
a list offsets result

closes #4195
emasab added a commit that referenced this issue Apr 13, 2023
lease expires and the partition is waiting for
a list offsets result

closes #4195
@wmorgan6796
Copy link
Author

Hey @emasab, I'm unsure what logs I can provide as per my company's policy sharing internal logs is quite difficult as they have to be cleansed by hand of any identifying information. I'm more than happy to test a build, but I'm not sure I can do more than that at this point. (since the implementation of the fix is already ready for review)

emasab added a commit that referenced this issue Apr 19, 2023
lease expires and the partition is waiting for
a list offsets result

closes #4195
emasab added a commit that referenced this issue Apr 19, 2023
and the partition is waiting for
a list offsets result

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

Successfully merging a pull request may close this issue.

2 participants