librdkafka icon indicating copy to clipboard operation
librdkafka copied to clipboard

Hang on rd_kafka_destroy()

Open pranasge opened this issue 3 years ago • 14 comments

Description

I am using partition queues and kept seeing application hanging on destructors. Seen similar issues #3007 and #3143 but i could reproduce on 1.7.0. Then created minimal test. Issue is very "racy" as I needed to add sleeps and std::cout to reproduce.

How to reproduce

I run this test on loop

		std::string err{};
		std::unique_ptr<RdKafka::Conf> conf{ RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL) };
		conf->set("bootstrap.servers", "localhost:9093,localhost:9094,localhost:9095", err);
		conf->set("auto.offset.reset", "error", err);
		conf->set("enable.auto.commit", "false", err);
		conf->set("enable.partition.eof", "true", err);
		conf->set("group.id", "1", err);
		conf->set("debug", "all", err);

		std::shared_ptr<RdKafka::KafkaConsumer> kafkaConsumer{ RdKafka::KafkaConsumer::create(conf.get(), err) };
		{
			std::vector<std::future<void>> futures{3};

			class PartitionQueue
			{
			public:
				std::shared_ptr<RdKafka::KafkaConsumer> kafkaConsumer;
				std::unique_ptr<RdKafka::TopicPartition> topicPartition;
				std::unique_ptr<RdKafka::Queue> queue;

				PartitionQueue(const std::shared_ptr<RdKafka::KafkaConsumer> & kafkaConsumer, const std::string & topic, int partition) :
					kafkaConsumer(kafkaConsumer),
					topicPartition(RdKafka::TopicPartition::create(topic, partition, RdKafka::Topic::OFFSET_BEGINNING))
				{
					std::vector<RdKafka::TopicPartition *> consumerPartitions;
					consumerPartitions.push_back(topicPartition.get());
					kafkaConsumer->incremental_assign(consumerPartitions);

					queue = std::unique_ptr<RdKafka::Queue>(kafkaConsumer->get_partition_queue(topicPartition.get()));
					queue->forward(nullptr);
				}
				~PartitionQueue()
				{
					std::vector<RdKafka::TopicPartition *> consumerPartitions;
					consumerPartitions.push_back(topicPartition.get());
					kafkaConsumer->incremental_unassign(consumerPartitions);
				}
			};

			for (int i = 0; i < 3; i++)
			{
				futures[i] = std::async(std::launch::async, [&, partitionQueue = std::make_unique<PartitionQueue>( kafkaConsumer, "storage", i )]() mutable {
					for (int x = 10; x > 0; x--)
					{
						auto msg = std::unique_ptr<RdKafka::Message>(partitionQueue->queue->consume(500));
						if (msg->err() == RdKafka::ErrorCode::ERR__PARTITION_EOF || msg->err() == RdKafka::ErrorCode::ERR__TIMED_OUT)
							break;
						std::cout << "Message Consumed " << std::endl;
					}

					sleep(1);
				});
			}

			for (auto & future : futures)
				future.get();
		}
		while (kafkaConsumer->outq_len())
			kafkaConsumer->poll(500);

and restart brokers incrementally (1, 2, 3, 1, 2,...).

Callstack:

Thread 4 (Thread 0x7f08926d5700 (LWP 24410)):
#0  futex_abstimed_wait_cancelable (private=0, abstime=0x7f08926ccbd0, expected=0, futex_word=0x5557025e92a4)
    at ../sysdeps/unix/sysv/linux/futex-internal.h:205
#1  __pthread_cond_wait_common (abstime=0x7f08926ccbd0, mutex=0x5557025e9250, cond=0x5557025e9278)
    at pthread_cond_wait.c:539
#2  __pthread_cond_timedwait (cond=0x5557025e9278, mutex=0x5557025e9250, abstime=0x7f08926ccbd0) at pthread_cond_wait.c:667
#3  0x00007f08953e5149 in cnd_timedwait (cond=<optimized out>, mutex=<optimized out>, time_point=<optimized out>)
    at cnd_timedwait.c:25
#4  0x0000555701052a3e in cnd_timedwait_abs (cnd=0x5557025e9278, mtx=0x5557025e9250, tspec=0x7f08926ccbd0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/tinycthread_extra.c:106
#5  0x0000555700faf420 in rd_kafka_q_pop_serve (rkq=0x5557025e9250, timeout_us=999, version=0,
    cb_type=RD_KAFKA_Q_CB_RETURN, callback=0x0, opaque=0x0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_queue.c:412
#6  0x0000555700faf4c0 in rd_kafka_q_pop (rkq=0x5557025e9250, timeout_us=999, version=0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_queue.c:436
#7  0x0000555700f3b04a in rd_kafka_broker_ops_serve (rkb=0x5557025e8610, timeout_us=999)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:3377
#8  0x0000555700f3b163 in rd_kafka_broker_ops_io_serve (rkb=0x5557025e8610, abs_timeout=930900612854)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:3433
#9  0x0000555700f44454 in rd_kafka_broker_consumer_serve (rkb=0x5557025e8610, abs_timeout=930900612854)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5023
#10 0x0000555700f446dc in rd_kafka_broker_serve (rkb=0x5557025e8610, timeout_ms=1)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5166
#11 0x0000555700f44923 in rd_kafka_broker_thread_main (arg=0x5557025e8610)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5222
#12 0x00007f08953d90a2 in start_thread (arg=<optimized out>) at pthread_create.c:483
#13 0x00007f0894fe14cf in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95

Thread 3 (Thread 0x7f0892ed6700 (LWP 24409)):
#0  futex_abstimed_wait_cancelable (private=0, abstime=0x7f0892ecdbd0, expected=0, futex_word=0x5557025e8414)
    at ../sysdeps/unix/sysv/linux/futex-internal.h:205
#1  __pthread_cond_wait_common (abstime=0x7f0892ecdbd0, mutex=0x5557025e83c0, cond=0x5557025e83e8)
    at pthread_cond_wait.c:539
#2  __pthread_cond_timedwait (cond=0x5557025e83e8, mutex=0x5557025e83c0, abstime=0x7f0892ecdbd0) at pthread_cond_wait.c:667
#3  0x00007f08953e5149 in cnd_timedwait (cond=<optimized out>, mutex=<optimized out>, time_point=<optimized out>)
    at cnd_timedwait.c:25
#4  0x0000555701052a3e in cnd_timedwait_abs (cnd=0x5557025e83e8, mtx=0x5557025e83c0, tspec=0x7f0892ecdbd0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/tinycthread_extra.c:106
#5  0x0000555700faf420 in rd_kafka_q_pop_serve (rkq=0x5557025e83c0, timeout_us=1000, version=0,
    cb_type=RD_KAFKA_Q_CB_RETURN, callback=0x0, opaque=0x0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_queue.c:412
#6  0x0000555700faf4c0 in rd_kafka_q_pop (rkq=0x5557025e83c0, timeout_us=1000, version=0)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_queue.c:436
#7  0x0000555700f3b04a in rd_kafka_broker_ops_serve (rkb=0x5557025e7780, timeout_us=1000)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:3377
#8  0x0000555700f3b163 in rd_kafka_broker_ops_io_serve (rkb=0x5557025e7780, abs_timeout=930900613328)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:3433
#9  0x0000555700f44454 in rd_kafka_broker_consumer_serve (rkb=0x5557025e7780, abs_timeout=930900613327)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5023
#10 0x0000555700f446dc in rd_kafka_broker_serve (rkb=0x5557025e7780, timeout_ms=1)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5166
#11 0x0000555700f44923 in rd_kafka_broker_thread_main (arg=0x5557025e7780)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka_broker.c:5222
#12 0x00007f08953d90a2 in start_thread (arg=<optimized out>) at pthread_create.c:483
#13 0x00007f0894fe14cf in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95

Thread 2 (Thread 0x7f08946d9700 (LWP 24406)):
#0  0x00007f08953da495 in __GI___pthread_timedjoin_ex (threadid=139674793105152,
    thread_return=thread_return@entry=0x7f08946d0de0, abstime=abstime@entry=0x0, block=block@entry=true)
    at pthread_join_common.c:89
#1  0x00007f08953e4d37 in thrd_join (thr=<optimized out>, res=0x7f08946d0e18) at thrd_join.c:25
#2  0x0000555700f188ee in rd_kafka_destroy_internal (rk=0x5557025dc350)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka.c:1280
#3  0x0000555700f1cc63 in rd_kafka_thread_main (arg=0x5557025dc350)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka.c:2085
#4  0x00007f08953d90a2 in start_thread (arg=<optimized out>) at pthread_create.c:483
#5  0x00007f0894fe14cf in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95

Thread 1 (Thread 0x7f0894ee36c0 (LWP 24403)):
#0  0x00007f08953da495 in __GI___pthread_timedjoin_ex (threadid=139674826675968,
    thread_return=thread_return@entry=0x7ffe3d8c3a50, abstime=abstime@entry=0x0, block=block@entry=true)
    at pthread_join_common.c:89
#1  0x00007f08953e4d37 in thrd_join (thr=<optimized out>, res=0x7ffe3d8c3b9c) at thrd_join.c:25
#2  0x0000555700f181a0 in rd_kafka_destroy_app (rk=0x5557025dc350, flags=8)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka.c:1115
#3  0x0000555700f1824f in rd_kafka_destroy_flags (rk=0x5557025dc350, flags=8)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src/rdkafka.c:1132
#4  0x0000555700f0c67a in RdKafka::KafkaConsumerImpl::~KafkaConsumerImpl (this=0x5557025d8860, __in_chrg=<optimized out>,
    __vtt_parm=<optimized out>)
    at /mnt/c/Branches/mma/vcpkg/buildtrees/librdkafka/src/a46d1c52f5-4d89051189.clean/src-cpp/rdkafkacpp_int.h:1134
#5  0x0000555700f0c6e6 in RdKafka::KafkaConsumerImpl::~KafkaConsumerImpl (this=0x5557025d8860, __in_chrg=<optimized out>,
    __vtt_parm=<optimized out>)

Logs:

%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: ===== Received metadata (for 1 requested topics): refresh unavailable topics =====
%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: ClusterId: XDHMr1kwQcewkezVXX4k8Q, ControllerId: -1
%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: 0 brokers, 1 topics
%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9095/3:   Topic #0/1: storage with 0 partitions: Broker: Unknown topic or partition
%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: Error in metadata reply for topic storage (PartCnt 0): Broker: Unknown topic or partition
%7|1627303513.066|STATE|rdkafka#consumer-1| [thrd:main]: Topic storage changed state exists -> notexists
%5|1627303513.066|PARTCNT|rdkafka#consumer-1| [thrd:main]: Topic storage partition count changed from 3 to 0
%7|1627303513.066|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [0] no longer reported in metadata
%7|1627303513.066|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [0] is desired but no longer known: moving back on desired list
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegate to broker (none) (rktp 0x7f088c001700, term 0, ref 8)
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: no longer delegated to broker localhost:9095/3
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
%7|1627303513.066|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [0] 0x7f088c001700 from localhost:9095/3 to :0/internal (sending PARTITION_LEAVE to localhost:9095/3)
%7|1627303513.066|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [1] no longer reported in metadata
%7|1627303513.066|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [1] is desired but no longer known: moving back on desired list
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: delegate to broker (none) (rktp 0x7f088c0021e0, term 0, ref 6)
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: not updating broker: already on correct broker :0/internal
%7|1627303513.066|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [2] no longer reported in metadata
%7|1627303513.066|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [2] is desired but no longer known: moving back on desired list
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegate to broker (none) (rktp 0x7f088c0029a0, term 0, ref 2051)
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: no longer delegated to broker localhost:9094/2
%7|1627303513.066|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
%7|1627303513.066|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [2] 0x7f088c0029a0 from localhost:9094/2 to :0/internal (sending PARTITION_LEAVE to localhost:9094/2)
%7|1627303513.066|FETCH|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [2] in state active at offset 0 (2046/100000 msgs, 5119/65536 kb queued, opv 3) is not fetchable: forced removal
%7|1627303513.066|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: 1/1 requested topic(s) seen in metadata
%7|1627303513.066|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Topic storage [0]: leaving broker (0 messages in xmitq, next broker :0/internal, rktp 0x7f088c001700)
%7|1627303513.066|CONTROLLERID|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: ControllerId update 2 -> -1
%7|1627303513.066|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1627303513.066|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1627303513.066|FETCHADD|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Removed storage [2] from fetch list (0 entries, opv 3): forced removal
%7|1627303513.066|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [2]: leaving broker (0 messages in xmitq, next broker :0/internal, rktp 0x7f088c0029a0)
%7|1627303513.066|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: joining broker (rktp 0x7f088c001700, 0 message(s) queued)
%7|1627303513.066|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.066|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: joining broker (rktp 0x7f088c0029a0, 0 message(s) queued)
%7|1627303513.066|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.066|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1627303513.066|FETCH|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2] in state active at offset 0 (2046/100000 msgs, 5119/65536 kb queued, opv 3) is fetchable
%7|1627303513.066|FETCHADD|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Added storage [2] to fetch list (1 entries, opv 3, 0 messages queued): fetchable
%7|1627303513.108|RECV|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received FetchResponse (v11, 69 bytes, CorrId 12, rtt 502.29ms)
%7|1627303513.108|UNKTOPIC|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received Fetch response (error 0) for unknown topic storage [2]: ignoring
%7|1627303513.166|METADATA|rdkafka#consumer-1| [thrd:main]: Expired 1 entries from metadata cache (0 entries remain)
%7|1627303513.557|OFFSET|rdkafka#consumer-1| [thrd:main]: Topic storage [1]: timed offset query for BEGINNING in state offset-query
%7|1627303513.557|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [1]: backoff: (re)starting offset query timer for offset BEGINNING
%7|1627303513.557|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1627303513.557|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 3 partition(s)
%7|1627303513.557|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=3, wait_stop_cnt=0)
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 2 partition(s):
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1627303513.557|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1627303513.557|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_fetch_stop:2375: new version barrier v4
%7|1627303513.557|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [0] (v4)
%7|1627303513.557|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_pause_resume:2434: new version barrier v5
%7|1627303513.557|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [0] (v5)
%7|1627303513.557|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [0]
%7|1627303513.557|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [0] 0x7f088c001700 sending final LEAVE for removal by :0/internal
%7|1627303513.557|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [0] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1627303513.557|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1627303513.557|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 2 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1627303513.557|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f088c001700)
%7|1627303513.557|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: no next broker, failing 0 message(s) in partition queue
%7|1627303513.557|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.557|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op FETCH_STOP (v4) in fetch-state none (opv3)
%7|1627303513.557|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [0] in state none (v4)
%7|1627303513.557|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state none -> stopping
%7|1627303513.557|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [0]: offset store terminating
%7|1627303513.557|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state stopping -> stopped
%7|1627303513.557|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1627303513.557|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [0]: partition is not paused by library
%7|1627303513.557|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [0]
%7|1627303513.557|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [0]
%7|1627303513.557|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1627303513.557|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=2, wait_stop_cnt=0)
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 2 partition(s):
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1627303513.557|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1627303513.557|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1627303513.557|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 2 partition(s)
%7|1627303513.557|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=2, wait_stop_cnt=0)
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1627303513.557|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1627303513.557|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_fetch_stop:2375: new version barrier v4
%7|1627303513.557|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [1] (v4)
%7|1627303513.557|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_pause_resume:2434: new version barrier v5
%7|1627303513.557|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [1] (v5)
%7|1627303513.557|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [1]
%7|1627303513.557|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [1] 0x7f088c0021e0 sending final LEAVE for removal by :0/internal
%7|1627303513.557|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [1] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1627303513.557|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1627303513.557|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 1 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1627303513.557|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [1]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f088c0021e0)
%7|1627303513.557|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [1]: no next broker, failing 0 message(s) in partition queue
%7|1627303513.557|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.557|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op FETCH_STOP (v4) in fetch-state offset-query (opv3)
%7|1627303513.557|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [1] in state offset-query (v4)
%7|1627303513.557|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state offset-query -> stopping
%7|1627303513.557|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [1]: offset store terminating
%7|1627303513.557|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state stopping -> stopped
%7|1627303513.557|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1627303513.557|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [1]: partition is not paused by library
%7|1627303513.557|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [1]
%7|1627303513.557|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [1]
%7|1627303513.557|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1627303513.557|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1627303513.557|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1627303513.557|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.557|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1627303513.557|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1627303513.558|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 1 partition(s)
%7|1627303513.558|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
%7|1627303513.558|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1627303513.558|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1627303513.558|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_fetch_stop:2375: new version barrier v4
%7|1627303513.558|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [2] (v4)
%7|1627303513.558|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_pause_resume:2434: new version barrier v5
%7|1627303513.558|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [2] (v5)
%7|1627303513.558|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [2]
%7|1627303513.558|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [2] 0x7f088c0029a0 sending final LEAVE for removal by :0/internal
%7|1627303513.558|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [2] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1627303513.558|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1627303513.558|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 0 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1627303513.558|FETCH|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2] in state active at offset 0 (2046/100000 msgs, 5119/65536 kb queued, opv 3) is not fetchable: forced removal
%7|1627303513.558|FETCHADD|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Removed storage [2] from fetch list (0 entries, opv 3): forced removal
%7|1627303513.558|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f088c0029a0)
%7|1627303513.558|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: no next broker, failing 0 message(s) in partition queue
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.558|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op FETCH_STOP (v4) in fetch-state active (opv3)
%7|1627303513.558|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [2] in state active (v4)
%7|1627303513.558|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state active -> stopping
%7|1627303513.558|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [2]: offset store terminating
%7|1627303513.558|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state stopping -> stopped
%7|1627303513.558|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:app]: Terminating instance (destroy flags NoConsumerClose (0x8))
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:app]: Terminating consumer group handler
%7|1627303513.558|CLOSE|rdkafka#consumer-1| [thrd:app]: Closing consumer
%7|1627303513.558|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [2]: partition is not paused by library
%7|1627303513.558|CLOSE|rdkafka#consumer-1| [thrd:app]: Disabling and purging temporary queue to quench close events
%7|1627303513.558|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [2]
%7|1627303513.558|CLOSE|rdkafka#consumer-1| [thrd:app]: Consumer closed
%7|1627303513.558|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [2]
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:app]: Interrupting timers
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:app]: Sending TERMINATE to internal main thread
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:app]: Joining internal main thread
%7|1627303513.558|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1627303513.558|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
%7|1627303513.558|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1627303513.558|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op TERMINATE in state up (join-state init)
%7|1627303513.558|CGRPTERM|rdkafka#consumer-1| [thrd:main]: Terminating group "1" in state up with 0 partition(s)
%7|1627303513.558|CLEARASSIGN|rdkafka#consumer-1| [thrd:main]: No current assignment to clear
%7|1627303513.558|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
%7|1627303513.558|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1627303513.558|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1627303513.558|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state up -> term (join-state init)
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1627303513.558|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Group "1": clearing group assignment
%7|1627303513.558|COORDCLEAR|rdkafka#consumer-1| [thrd:main]: Group "1" broker localhost:9095/3 is no longer coordinator
%7|1627303513.558|NODENAME|rdkafka#consumer-1| [thrd:main]: GroupCoordinator/3: Broker nodename changed from "localhost:9095" to ""
%7|1627303513.558|NODEID|rdkafka#consumer-1| [thrd:main]: GroupCoordinator/3: Broker nodeid changed from 3 to -1
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:main]: Internal main thread terminating
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Destroy internal
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Removing all topics
%7|1627303513.558|TOPPARREMOVE|rdkafka#consumer-1| [thrd:main]: Removing toppar storage [-1] 0x7f088c001130
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: storage [-1]: 0x7f088c001130 DESTROY_FINAL
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9095/3
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9094/2
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Closing connection due to nodename change (after 1492ms in state UP) (_TRANSPORT)
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9093/bootstrap
%7|1627303513.558|TERM|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received TERMINATE op in state UP: 9 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state UP -> DOWN
%7|1627303513.558|TERM|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Received TERMINATE op in state UP: 3 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1627303513.558|TERM|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Client is terminating (after 1496ms in state UP) (_DESTROY)
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state UP -> DOWN
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Client is terminating (after 1501ms in state INIT) (_DESTROY)
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state INIT -> DOWN
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Client is terminating (after 997ms in state UP) (_DESTROY)
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Updating 0 buffers on connection reset
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.558|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Handle is terminating in state DOWN: 1 refcnts (0x5557025e6c78), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.558|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to GroupCoordinator
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state UP -> DOWN
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:main]: Purging reply queue
%7|1627303513.558|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: terminating: broker still has 9 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1627303513.558|BRKTERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state DOWN: 8 refcnts (0x5557025e7b08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.558|CONNECT|rdkafka#consumer-1| [thrd:GroupCoordinator]: Cluster connection already in progress: broker down
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state DOWN -> INIT
%7|1627303513.558|METADATA|rdkafka#consumer-1| [thrd:GroupCoordinator]: Hinted cache of 1/1 topic(s) being queried
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1627303513.558|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state DOWN: 2 refcnts (0x5557025e8998), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:main]: Decommissioning internal broker
%7|1627303513.558|METADATA|rdkafka#consumer-1| [thrd:GroupCoordinator]: Skipping metadata refresh of 1 topic(s): broker down: no usable brokers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:main]: Join 5 broker thread(s)
%7|1627303513.558|TERM|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1627303513.558|TERMINATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 2 refcnts (0x5557025e2d08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.558|FAIL|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Client is terminating (after 1502ms in state INIT) (_DESTROY)
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Broker changed state INIT -> DOWN
%7|1627303513.558|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state DOWN -> INIT
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1627303513.558|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1627303513.558|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
%7|1627303513.559|BRKTERM|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.559|TERMINATE|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Handle is terminating in state DOWN: 1 refcnts (0x5557025e5f28), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.559|FAIL|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1627303513.558|TERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1627303513.559|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Client is terminating (after 0ms in state INIT) (_DESTROY)
%7|1627303513.559|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state INIT -> DOWN
%7|1627303513.559|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1627303513.559|BRKTERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1627303513.559|TERMINATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 1 refcnts (0x5557025e2d08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.559|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1627303513.559|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
%7|1627303513.559|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 8 refcnts (0x5557025e7b08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.559|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x5557025e8998), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.560|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 8 refcnts (0x5557025e7b08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.560|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x5557025e8998), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.561|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 8 refcnts (0x5557025e7b08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303513.561|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x5557025e8998), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
....
%7|1627303518.996|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x5557025e8998), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1627303518.997|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 8 refcnts (0x5557025e7b08), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf

IMPORTANT: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed.

Checklist

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

Please provide the following information:

  • [x] librdkafka version (release number or git tag): REF 77a013b7a2611f7bdc091afa1e56b1a46d1c52f5 #v1.7.0
  • [x] Apache Kafka version: docker.io/bitnami/kafka:2.7.0-debian-10-r35
  • [x] librdkafka client configuration: can be seen in test
  • [x] Operating system: Debian GNU/Linux 10 (buster)
  • [x] Provide logs (with debug=all as necessary) from librdkafka
  • [ ] Provide broker log excerpts
  • [ ] Critical issue

pranasge avatar Jul 26 '21 13:07 pranasge

This typically happens when there is an outstanding librdkafka object reference, see https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#termination

Also try to reproduce on latest master, thanks.

edenhill avatar Apr 08 '22 10:04 edenhill

From the test it does not seem like there is any object references besides RdKafka::KafkaConsumer and RdKafka::Conf at the end of this test. And I just reproduced this tests with 0bfb95048cc03fd6d82f3a3a09965c24fb088b08.

%7|1649683356.619|TERMINATE|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Handle is terminating in state DOWN: 1 refcnts (0x55f719acd7d8), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1649683356.619|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1649683356.619|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 1 buffers
%7|1649683356.619|OFFSET|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Offset reply for topic storage [0] (v3 vs v5)
%7|1649683356.619|REQERR|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: ListOffsetsRequest failed: Local: Broker handle destroyed: actions Permanent
%7|1649683356.619|OFFSET|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: OffsetRequest failed: Local: Broker handle destroyed (Permanent)
%7|1649683356.619|OFFSET|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Offset reply error for topic storage [0] (v3, Local: Broker handle destroyed): Permanent
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1649683356.619|FAIL|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Purging bufq with 1 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1649683356.619|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1649683356.619|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1649683356.619|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Updating 0 buffers on connection reset
%7|1649683356.619|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1649683356.619|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state DOWN: 2 refcnts (0x55f719acf478), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1649683356.619|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state DOWN -> INIT
%7|1649683356.619|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1649683356.619|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1649683356.619|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Handle is terminating in state DOWN: 1 refcnts (0x55f719ace5a8), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1649683356.619|FAIL|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1649683356.619|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1649683356.620|TERMINATE|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Handle is terminating in state INIT: 3 refcnts (0x7feb380043e8), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1649683356.620|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55f719ad0348), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1649683356.620|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 2 refcnts (0x55f719acf478), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf

For me it seems problem happens when broker tries to reconnect to shutting down client.

pranasge avatar Apr 11 '22 13:04 pranasge

@edenhill Should I open new issue for this?

pranasge avatar Apr 25 '22 11:04 pranasge

Could you try reproducing this on latest master with debug=all and provide the logs? Thanks

edenhill avatar Apr 26 '22 16:04 edenhill

No problem.

%7|1651057993.234|MEMBERID|rdkafka#consumer-1| [thrd:app]: Group "1": updating member id "(not-set)" -> ""
%7|1651057993.234|WAKEUPFD|rdkafka#consumer-1| [thrd:app]: GroupCoordinator: Enabled low-latency ops queue wake-ups
%7|1651057993.234|BROKER|rdkafka#consumer-1| [thrd:app]: GroupCoordinator: Added new broker with NodeId -1
%7|1651057993.234|BRKMAIN|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Enter main broker thread
%7|1651057993.235|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state init -> query-coord (join-state init)
%7|1651057993.235|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|WAKEUPFD|rdkafka#consumer-1| [thrd:app]: localhost:9093/bootstrap: Enabled low-latency ops queue wake-ups
%7|1651057993.235|BRKMAIN|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Enter main broker thread
%7|1651057993.235|BROKER|rdkafka#consumer-1| [thrd:app]: localhost:9093/bootstrap: Added new broker with NodeId -1
%7|1651057993.235|WAKEUPFD|rdkafka#consumer-1| [thrd:app]: localhost:9094/bootstrap: Enabled low-latency ops queue wake-ups
%7|1651057993.235|BRKMAIN|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enter main broker thread
%7|1651057993.235|BROKER|rdkafka#consumer-1| [thrd:app]: localhost:9094/bootstrap: Added new broker with NodeId -1
%7|1651057993.235|WAKEUPFD|rdkafka#consumer-1| [thrd:app]: localhost:9095/bootstrap: Enabled low-latency ops queue wake-ups
%7|1651057993.235|BRKMAIN|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/bootstrap: Enter main broker thread
%7|1651057993.235|BROKER|rdkafka#consumer-1| [thrd:app]: localhost:9095/bootstrap: Added new broker with NodeId -1
%7|1651057993.235|INIT|rdkafka#consumer-1| [thrd:app]: librdkafka v1.9.0-devel-O0 (0x10900ff) rdkafka#consumer-1 initialized (builtin.features snappy,sasl,regex,lz4,sasl_gssapi,sasl_plain,plugins, CMAKE GNU GNU PKGCONFIG HDRHISTOGRAM LIBDL PLUGINS C11THREADS CRC32C_HW SNAPPY SOCKEM, debug 0xfffff)
%7|1651057993.235|BRKMAIN|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/bootstrap: Enter main broker thread
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]: Client configuration:
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   client.software.version = 1.9.0-devel-O0
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   metadata.broker.list = localhost:9093,localhost:9094,localhost:9095
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   debug = generic,broker,topic,metadata,feature,queue,msg,protocol,cgrp,security,fetch,interceptor,plugin,consumer,admin,eos,mock,assignor,conf,all
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   opaque = 0x55d5056cbdb8
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   default_topic_conf = 0x55d5056e1920
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   ssl_key = [redacted]
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   group.id = 1
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   enable.auto.commit = false
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   enable.partition.eof = true
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]: Default topic configuration:
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:app]:   auto.offset.reset = error
%7|1651057993.235|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state query-coord (join-state init)
%7|1651057993.235|TOPIC|rdkafka#consumer-1| [thrd:main]: New local topic: storage
%7|1651057993.235|TOPPARNEW|rdkafka#consumer-1| [thrd:main]: NEW storage [-1] 0x7f73e8001160 refcnt 0x7f73e80011f0 (at rd_kafka_topic_new0:468)
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:main]: Topic "storage" configuration (default_topic_conf):
%7|1651057993.235|CONF|rdkafka#consumer-1| [thrd:main]:   auto.offset.reset = error
%7|1651057993.235|TOPPARNEW|rdkafka#consumer-1| [thrd:main]: NEW storage [0] 0x7f73e8001770 refcnt 0x7f73e8001800 (at rd_kafka_toppar_desired_add:633)
%7|1651057993.235|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [0]: marking as DESIRED
%7|1651057993.235|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [0]: adding to DESIRED list
%7|1651057993.235|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Added 1 partition(s) to assignment which now consists of 1 partition(s) where of 1 are in pending state and 0 are being queried
%7|1651057993.235|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
%7|1651057993.235|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057993.235|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057993.235|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057993.235|DUMP_PND|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057993.235|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.235|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.235|SRVPEND|rdkafka#consumer-1| [thrd:main]: Starting pending assigned partition storage [0] at offset BEGINNING
%7|1651057993.235|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v2
%7|1651057993.235|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [0] (v2)
%7|1651057993.235|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_fetch_start:2335: new version barrier v3
%7|1651057993.235|CONSUMER|rdkafka#consumer-1| [thrd:main]: Start consuming storage [0] at offset BEGINNING (v3)
%7|1651057993.235|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: Selected for cluster connection: coordinator query (broker has 0 connection attempt(s))
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received CONNECT op
%7|1651057993.235|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.235|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1651057993.235|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op PAUSE (v2) in fetch-state none (opv1)
%7|1651057993.235|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [0]: partition is not paused by library
%7|1651057993.235|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op FETCH_START (v3) in fetch-state none (opv2)
%7|1651057993.235|FETCH|rdkafka#consumer-1| [thrd:main]: Start fetch for storage [0] in state none at offset BEGINNING (v3)
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: broker in state TRY_CONNECT connecting
%7|1651057993.235|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057993.235|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1651057993.235|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state none -> offset-query
%7|1651057993.235|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [0]: offset reset (at offset BEGINNING, broker -1) to BEGINNING: update: Success
%7|1651057993.235|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [0]: no current leader for partition: (re)starting offset query timer for offset BEGINNING
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_JOIN in state query-coord (join-state init) for storage [0]
%7|1651057993.235|PARTADD|rdkafka#consumer-1| [thrd:main]: Group "1": add storage [0]
%7|1651057993.235|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state query-coord (join-state init)
%7|1651057993.235|TOPPARNEW|rdkafka#consumer-1| [thrd:main]: NEW storage [1] 0x7f73e80022f0 refcnt 0x7f73e8002380 (at rd_kafka_toppar_desired_add:633)
%7|1651057993.235|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [1]: marking as DESIRED
%7|1651057993.235|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [1]: adding to DESIRED list
%7|1651057993.235|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Added 1 partition(s) to assignment which now consists of 2 partition(s) where of 1 are in pending state and 0 are being queried
%7|1651057993.235|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
%7|1651057993.235|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 2 partition(s):
%7|1651057993.235|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057993.235|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057993.235|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057993.235|DUMP_PND|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057993.235|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.235|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.235|SRVPEND|rdkafka#consumer-1| [thrd:main]: Starting pending assigned partition storage [1] at offset BEGINNING
%7|1651057993.235|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v2
%7|1651057993.235|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [1] (v2)
%7|1651057993.235|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_fetch_start:2335: new version barrier v3
%7|1651057993.235|CONSUMER|rdkafka#consumer-1| [thrd:main]: Start consuming storage [1] at offset BEGINNING (v3)
%7|1651057993.235|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op PAUSE (v2) in fetch-state none (opv1)
%7|1651057993.235|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [1]: partition is not paused by library
%7|1651057993.235|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op FETCH_START (v3) in fetch-state none (opv2)
%7|1651057993.235|FETCH|rdkafka#consumer-1| [thrd:main]: Start fetch for storage [1] in state none at offset BEGINNING (v3)
%7|1651057993.235|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state none -> offset-query
%7|1651057993.235|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [1]: offset reset (at offset BEGINNING, broker -1) to BEGINNING: update: Success
%7|1651057993.235|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [1]: no current leader for partition: (re)starting offset query timer for offset BEGINNING
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_JOIN in state query-coord (join-state init) for storage [1]
%7|1651057993.235|PARTADD|rdkafka#consumer-1| [thrd:main]: Group "1": add storage [1]
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.235|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.235|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.236|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state query-coord (join-state init)
%7|1651057993.236|TOPPARNEW|rdkafka#consumer-1| [thrd:main]: NEW storage [2] 0x7f73e8002a60 refcnt 0x7f73e8002af0 (at rd_kafka_toppar_desired_add:633)
%7|1651057993.236|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [2]: marking as DESIRED
%7|1651057993.236|DESIRED|rdkafka#consumer-1| [thrd:main]: storage [2]: adding to DESIRED list
%7|1651057993.236|CONNECT|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Connecting to ipv4#127.0.0.1:9093 (plaintext) with socket 13
%7|1651057993.236|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Added 1 partition(s) to assignment which now consists of 3 partition(s) where of 1 are in pending state and 0 are being queried
%7|1651057993.236|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=2, wait_stop_cnt=0)
%7|1651057993.236|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 3 partition(s):
%7|1651057993.236|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057993.236|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057993.236|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057993.236|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057993.236|DUMP_PND|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057993.236|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.236|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.236|SRVPEND|rdkafka#consumer-1| [thrd:main]: Starting pending assigned partition storage [2] at offset BEGINNING
%7|1651057993.236|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v2
%7|1651057993.236|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [2] (v2)
%7|1651057993.236|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_fetch_start:2335: new version barrier v3
%7|1651057993.236|CONSUMER|rdkafka#consumer-1| [thrd:main]: Start consuming storage [2] at offset BEGINNING (v3)
%7|1651057993.236|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057993.236|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.236|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.236|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op PAUSE (v2) in fetch-state none (opv1)
%7|1651057993.236|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [2]: partition is not paused by library
%7|1651057993.236|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op FETCH_START (v3) in fetch-state none (opv2)
%7|1651057993.236|FETCH|rdkafka#consumer-1| [thrd:main]: Start fetch for storage [2] in state none at offset BEGINNING (v3)
%7|1651057993.236|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state none -> offset-query
%7|1651057993.236|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [2]: offset reset (at offset BEGINNING, broker -1) to BEGINNING: update: Success
%7|1651057993.236|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [2]: no current leader for partition: (re)starting offset query timer for offset BEGINNING
%7|1651057993.236|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.236|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.236|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_JOIN in state query-coord (join-state init) for storage [2]
%7|1651057993.236|PARTADD|rdkafka#consumer-1| [thrd:main]: Group "1": add storage [2]
%7|1651057993.236|CONNECT|rdkafka#consumer-1| [thrd:main]: Not selecting any broker for cluster connection: still suppressed for 49ms: coordinator query
%7|1651057993.236|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: Group "1": no broker available for coordinator query: intervaled in state query-coord
%7|1651057993.236|CONNECT|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Connected to ipv4#127.0.0.1:9093
%7|1651057993.236|CONNECTED|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Connected (#1)
%7|1651057993.236|FEATURE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updated enabled protocol features +ApiVersion to ApiVersion
%7|1651057993.236|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state CONNECT -> APIVERSION_QUERY
%7|1651057993.236|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1651057993.236|SEND|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Sent ApiVersionRequest (v3, 49 bytes @ 0, CorrId 1)
%7|1651057993.237|RECV|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received ApiVersionResponse (v3, 431 bytes, CorrId 1, rtt 1.62ms)
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker API support:
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Produce (0) Versions 0..9
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Fetch (1) Versions 0..12
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ListOffsets (2) Versions 0..7
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Metadata (3) Versions 0..11
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey LeaderAndIsr (4) Versions 0..5
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey StopReplica (5) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey UpdateMetadata (6) Versions 0..7
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ControlledShutdown (7) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey OffsetCommit (8) Versions 0..8
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey OffsetFetch (9) Versions 0..8
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey FindCoordinator (10) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey JoinGroup (11) Versions 0..7
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Heartbeat (12) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey LeaveGroup (13) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey SyncGroup (14) Versions 0..5
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeGroups (15) Versions 0..5
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ListGroups (16) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey SaslHandshake (17) Versions 0..1
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ApiVersion (18) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey CreateTopics (19) Versions 0..7
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DeleteTopics (20) Versions 0..6
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DeleteRecords (21) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey InitProducerId (22) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey OffsetForLeaderEpoch (23) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AddPartitionsToTxn (24) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AddOffsetsToTxn (25) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey EndTxn (26) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey WriteTxnMarkers (27) Versions 0..1
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey TxnOffsetCommit (28) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeAcls (29) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey CreateAcls (30) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DeleteAcls (31) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeConfigs (32) Versions 0..4
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterConfigs (33) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterReplicaLogDirs (34) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeLogDirs (35) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey SaslAuthenticate (36) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey CreatePartitions (37) Versions 0..3
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey CreateDelegationToken (38) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey RenewDelegationToken (39) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ExpireDelegationToken (40) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeDelegationToken (41) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DeleteGroups (42) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ElectLeadersRequest (43) Versions 0..2
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey IncrementalAlterConfigsRequest (44) Versions 0..1
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterPartitionReassignmentsRequest (45) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey ListPartitionReassignmentsRequest (46) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey OffsetDeleteRequest (47) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeClientQuotasRequest (48) Versions 0..1
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterClientQuotasRequest (49) Versions 0..1
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey DescribeUserScramCredentialsRequest (50) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterUserScramCredentialsRequest (51) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey AlterIsrRequest (56) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey UpdateFeaturesRequest (57) Versions 0..0
%7|1651057993.237|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Unknown-60? (60) Versions 0..0
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Unknown-61? (61) Versions 0..0
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Unknown-65? (65) Versions 0..0
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Unknown-66? (66) Versions 0..0
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:   ApiKey Unknown-67? (67) Versions 0..0
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature MsgVer1: Produce (2..2) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature MsgVer1: Fetch (2..2) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature MsgVer1
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature MsgVer2: Produce (3..3) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature MsgVer2: Fetch (4..4) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature MsgVer2
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature ApiVersion: ApiVersion (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature ApiVersion
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerGroupCoordinator: FindCoordinator (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature BrokerGroupCoordinator
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: FindCoordinator (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: OffsetCommit (1..2) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: OffsetFetch (1..1) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: JoinGroup (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: SyncGroup (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: Heartbeat (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature BrokerBalancedConsumer: LeaveGroup (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature BrokerBalancedConsumer
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature ThrottleTime: Produce (1..2) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature ThrottleTime: Fetch (1..2) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature ThrottleTime
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature Sasl: JoinGroup (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature Sasl
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature SaslHandshake: SaslHandshake (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature SaslHandshake
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature LZ4: FindCoordinator (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature LZ4
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature OffsetTime: ListOffsets (1..1) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature OffsetTime
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature IdempotentProducer: InitProducerId (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature IdempotentProducer
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature ZSTD: Produce (7..7) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature ZSTD: Fetch (10..10) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature ZSTD
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature SaslAuthReq: SaslHandshake (1..1) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap:  Feature SaslAuthReq: SaslAuthenticate (0..0) supported by broker
%7|1651057993.238|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Enabling feature SaslAuthReq
%7|1651057993.238|FEATURE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updated enabled protocol features to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime,MsgVer2,IdempotentProducer,ZSTD,SaslAuthReq
%7|1651057993.238|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state APIVERSION_QUERY -> UP
%7|1651057993.238|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1651057993.238|METADATA|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Hinted cache of 1/1 topic(s) being queried
%7|1651057993.238|METADATA|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Requesting metadata for 1/1 topics: connected
%7|1651057993.238|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: Group "1": querying for coordinator: intervaled in state query-coord
%7|1651057993.238|METADATA|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Request metadata for 1 topic(s): connected
%7|1651057993.238|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state query-coord -> wait-coord (join-state init)
%7|1651057993.238|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057993.238|SEND|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Sent MetadataRequest (v4, 35 bytes @ 0, CorrId 2)
%7|1651057993.238|SEND|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Sent FindCoordinatorRequest (v2, 25 bytes @ 0, CorrId 3)
%7|1651057993.239|RECV|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received MetadataResponse (v4, 237 bytes, CorrId 2, rtt 1.04ms)
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: ===== Received metadata (for 1 requested topics): connected =====
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: ClusterId: tlRh8oKOSOO_jV1WfuggcA, ControllerId: 3
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: 3 brokers, 1 topics
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap:   Broker #0/3: localhost:9094 NodeId 2
%7|1651057993.239|UPDATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/bootstrap: NodeId changed from -1 to 2
%7|1651057993.239|UPDATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Name changed from localhost:9094/bootstrap to localhost:9094/2
%7|1651057993.239|LEADER|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Mapped 0 partition(s) to broker
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap:   Broker #1/3: localhost:9095 NodeId 3
%7|1651057993.239|UPDATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/bootstrap: NodeId changed from -1 to 3
%7|1651057993.239|UPDATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Name changed from localhost:9095/bootstrap to localhost:9095/3
%7|1651057993.239|LEADER|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Mapped 0 partition(s) to broker
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap:   Broker #2/3: 127.0.0.1:9093 NodeId 1
%7|1651057993.239|WAKEUPFD|rdkafka#consumer-1| [thrd:main]: 127.0.0.1:9093/1: Enabled low-latency ops queue wake-ups
%7|1651057993.239|BROKER|rdkafka#consumer-1| [thrd:main]: 127.0.0.1:9093/1: Added new broker with NodeId 1
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap:   Topic #0/1: storage with 3 partitions
%7|1651057993.239|STATE|rdkafka#consumer-1| [thrd:main]: Topic storage changed state unknown -> exists
%7|1651057993.239|PARTCNT|rdkafka#consumer-1| [thrd:main]: Topic storage partition count changed from 0 to 3
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]:   Topic storage partition 0 Leader 2
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegate to broker localhost:9094/2 (rktp 0x7f73e8001770, term 0, ref 4)
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegating to broker localhost:9094/2 for partition with 0 messages (0 bytes) queued
%7|1651057993.239|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [0] 0x7f73e8001770 from (none) to localhost:9094/2 (sending PARTITION_JOIN to localhost:9094/2)
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]:   Topic storage partition 1 Leader 2
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: delegate to broker localhost:9094/2 (rktp 0x7f73e80022f0, term 0, ref 4)
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: delegating to broker localhost:9094/2 for partition with 0 messages (0 bytes) queued
%7|1651057993.239|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [1] 0x7f73e80022f0 from (none) to localhost:9094/2 (sending PARTITION_JOIN to localhost:9094/2)
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]:   Topic storage partition 2 Leader 3
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegate to broker localhost:9095/3 (rktp 0x7f73e8002a60, term 0, ref 4)
%7|1651057993.239|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegating to broker localhost:9095/3 for partition with 0 messages (0 bytes) queued
%7|1651057993.239|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [0]: joining broker (rktp 0x7f73e8001770, 0 message(s) queued)
%7|1651057993.239|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [2] 0x7f73e8002a60 from (none) to localhost:9095/3 (sending PARTITION_JOIN to localhost:9095/3)
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.239|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [1]: joining broker (rktp 0x7f73e80022f0, 0 message(s) queued)
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.239|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: 1/1 requested topic(s) seen in metadata
%7|1651057993.239|CLUSTERID|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: ClusterId update "" -> "tlRh8oKOSOO_jV1WfuggcA"
%7|1651057993.239|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Topic storage [2]: joining broker (rktp 0x7f73e8002a60, 0 message(s) queued)
%7|1651057993.239|CONTROLLERID|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: ControllerId update -1 -> 3
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.239|BRKMAIN|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Enter main broker thread
%7|1651057993.239|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057993.240|RECV|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received FindCoordinatorResponse (v2, 31 bytes, CorrId 3, rtt 2.25ms)
%7|1651057993.240|CGRPCOORD|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: Group "1" coordinator is localhost:9094 id 2
%7|1651057993.240|CGRPCOORD|rdkafka#consumer-1| [thrd:main]: Group "1" changing coordinator -1 -> 2
%7|1651057993.240|COORDSET|rdkafka#consumer-1| [thrd:main]: Group "1" coordinator set to broker localhost:9094/2
%7|1651057993.240|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state wait-coord -> wait-broker-transport (join-state init)
%7|1651057993.240|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057993.240|NODENAME|rdkafka#consumer-1| [thrd:main]: GroupCoordinator: Broker nodename changed from "" to "localhost:9094"
%7|1651057993.240|NODEID|rdkafka#consumer-1| [thrd:main]: GroupCoordinator: Broker nodeid changed from -1 to 2
%7|1651057993.240|CGRPQUERY|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: Group "1": querying for coordinator: intervaled in state wait-broker-transport
%7|1651057993.240|CONNECT|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Received CONNECT op
%7|1651057993.240|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.240|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057993.240|SEND|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Sent FindCoordinatorRequest (v2, 25 bytes @ 0, CorrId 4)
%7|1651057993.240|CONNECT|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: broker in state TRY_CONNECT connecting
%7|1651057993.240|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057993.240|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057993.240|CONNECT|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Connecting to ipv4#127.0.0.1:9094 (plaintext) with socket 16
%7|1651057993.240|CONNECT|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Connected to ipv4#127.0.0.1:9094
%7|1651057993.240|CONNECTED|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Connected (#1)
%7|1651057993.240|FEATURE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Updated enabled protocol features +ApiVersion to ApiVersion
%7|1651057993.240|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state CONNECT -> APIVERSION_QUERY
%7|1651057993.240|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057993.240|SEND|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Sent ApiVersionRequest (v3, 49 bytes @ 0, CorrId 1)
%7|1651057993.241|RECV|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received FindCoordinatorResponse (v2, 31 bytes, CorrId 4, rtt 1.13ms)
%7|1651057993.241|CGRPCOORD|rdkafka#consumer-1| [thrd:main]: localhost:9093/bootstrap: Group "1" coordinator is localhost:9094 id 2
%7|1651057993.243|RECV|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Received ApiVersionResponse (v3, 431 bytes, CorrId 1, rtt 2.79ms)
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Broker API support:
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Produce (0) Versions 0..9
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Fetch (1) Versions 0..12
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ListOffsets (2) Versions 0..7
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Metadata (3) Versions 0..11
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey LeaderAndIsr (4) Versions 0..5
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey StopReplica (5) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey UpdateMetadata (6) Versions 0..7
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ControlledShutdown (7) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey OffsetCommit (8) Versions 0..8
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey OffsetFetch (9) Versions 0..8
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey FindCoordinator (10) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey JoinGroup (11) Versions 0..7
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Heartbeat (12) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey LeaveGroup (13) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey SyncGroup (14) Versions 0..5
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeGroups (15) Versions 0..5
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ListGroups (16) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey SaslHandshake (17) Versions 0..1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ApiVersion (18) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey CreateTopics (19) Versions 0..7
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DeleteTopics (20) Versions 0..6
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DeleteRecords (21) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey InitProducerId (22) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey OffsetForLeaderEpoch (23) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AddPartitionsToTxn (24) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AddOffsetsToTxn (25) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey EndTxn (26) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey WriteTxnMarkers (27) Versions 0..1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey TxnOffsetCommit (28) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeAcls (29) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey CreateAcls (30) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DeleteAcls (31) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeConfigs (32) Versions 0..4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterConfigs (33) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterReplicaLogDirs (34) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeLogDirs (35) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey SaslAuthenticate (36) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey CreatePartitions (37) Versions 0..3
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey CreateDelegationToken (38) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey RenewDelegationToken (39) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ExpireDelegationToken (40) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeDelegationToken (41) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DeleteGroups (42) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ElectLeadersRequest (43) Versions 0..2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey IncrementalAlterConfigsRequest (44) Versions 0..1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterPartitionReassignmentsRequest (45) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey ListPartitionReassignmentsRequest (46) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey OffsetDeleteRequest (47) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeClientQuotasRequest (48) Versions 0..1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterClientQuotasRequest (49) Versions 0..1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey DescribeUserScramCredentialsRequest (50) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterUserScramCredentialsRequest (51) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey AlterIsrRequest (56) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey UpdateFeaturesRequest (57) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Unknown-60? (60) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Unknown-61? (61) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Unknown-65? (65) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Unknown-66? (66) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:   ApiKey Unknown-67? (67) Versions 0..0
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature MsgVer1: Produce (2..2) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature MsgVer1: Fetch (2..2) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature MsgVer1
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature MsgVer2: Produce (3..3) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature MsgVer2: Fetch (4..4) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature MsgVer2
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature ApiVersion: ApiVersion (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature ApiVersion
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerGroupCoordinator: FindCoordinator (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature BrokerGroupCoordinator
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: FindCoordinator (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: OffsetCommit (1..2) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: OffsetFetch (1..1) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: JoinGroup (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: SyncGroup (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: Heartbeat (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature BrokerBalancedConsumer: LeaveGroup (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature BrokerBalancedConsumer
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature ThrottleTime: Produce (1..2) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature ThrottleTime: Fetch (1..2) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature ThrottleTime
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature Sasl: JoinGroup (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature Sasl
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature SaslHandshake: SaslHandshake (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature SaslHandshake
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature LZ4: FindCoordinator (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature LZ4
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature OffsetTime: ListOffsets (1..1) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature OffsetTime
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature IdempotentProducer: InitProducerId (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature IdempotentProducer
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature ZSTD: Produce (7..7) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature ZSTD: Fetch (10..10) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature ZSTD
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature SaslAuthReq: SaslHandshake (1..1) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2:  Feature SaslAuthReq: SaslAuthenticate (0..0) supported by broker
%7|1651057993.243|APIVERSION|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Enabling feature SaslAuthReq
%7|1651057993.243|FEATURE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator/2: Updated enabled protocol features to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime,MsgVer2,IdempotentProducer,ZSTD,SaslAuthReq
%7|1651057993.243|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state APIVERSION_QUERY -> UP
%7|1651057993.243|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057993.243|METADATA|rdkafka#consumer-1| [thrd:GroupCoordinator]: Skipping metadata refresh of 1 topic(s): connected: already being requested
%7|1651057993.243|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state wait-broker-transport -> up (join-state init)
%7|1651057993.243|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057993.243|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=3, wait_stop_cnt=0)
%7|1651057993.243|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 3 partition(s):
%7|1651057993.243|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057993.243|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057993.243|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057993.243|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.243|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.243|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057993.243|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057993.735|OFFSET|rdkafka#consumer-1| [thrd:main]: Topic storage [0]: timed offset query for BEGINNING in state offset-query
%7|1651057993.735|OFFREQ|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Partition storage [0]: querying for logical offset BEGINNING (opv 3)
%7|1651057993.735|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state offset-query -> offset-wait
%7|1651057993.735|OFFSET|rdkafka#consumer-1| [thrd:main]: Topic storage [1]: timed offset query for BEGINNING in state offset-query
%7|1651057993.735|OFFREQ|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Partition storage [1]: querying for logical offset BEGINNING (opv 3)
%7|1651057993.735|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state offset-query -> offset-wait
%7|1651057993.735|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.735|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.735|CONNECT|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: broker in state TRY_CONNECT connecting
%7|1651057993.735|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057993.735|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.735|CONNECT|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Connecting to ipv4#127.0.0.1:9094 (plaintext) with socket 17
%7|1651057993.736|CONNECT|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Connected to ipv4#127.0.0.1:9094
%7|1651057993.736|CONNECTED|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Connected (#1)
%7|1651057993.736|FEATURE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Updated enabled protocol features +ApiVersion to ApiVersion
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state CONNECT -> APIVERSION_QUERY
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.736|SEND|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Sent ApiVersionRequest (v3, 49 bytes @ 0, CorrId 1)
%7|1651057993.736|OFFSET|rdkafka#consumer-1| [thrd:main]: Topic storage [2]: timed offset query for BEGINNING in state offset-query
%7|1651057993.736|OFFREQ|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Partition storage [2]: querying for logical offset BEGINNING (opv 3)
%7|1651057993.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state offset-query -> offset-wait
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.736|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: broker in state TRY_CONNECT connecting
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.736|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connecting to ipv4#127.0.0.1:9095 (plaintext) with socket 18
%7|1651057993.736|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT) (_TRANSPORT)
%3|1651057993.736|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT)
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state CONNECT -> DOWN
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.736|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1651057993.736|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 1 buffers
%7|1651057993.736|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.736|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.736|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.736|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 193ms
%7|1651057993.736|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply for topic storage [2] (v3 vs v3)
%7|1651057993.736|REQERR|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: ListOffsetsRequest failed: Local: Broker transport failure: explicit actions Retry
%7|1651057993.736|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: OffsetRequest failed: Local: Broker transport failure (Retry)
%7|1651057993.736|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply error for topic storage [2] (v3, Local: Operation in progress): Retry
%7|1651057993.736|RETRY|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Retrying ListOffsetsRequest (v0, 0 bytes, retry 0/2, prev CorrId 0) in 100ms
%7|1651057993.736|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 193ms
%7|1651057993.739|RECV|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received ApiVersionResponse (v3, 431 bytes, CorrId 1, rtt 3.04ms)
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker API support:
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Produce (0) Versions 0..9
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Fetch (1) Versions 0..12
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ListOffsets (2) Versions 0..7
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Metadata (3) Versions 0..11
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey LeaderAndIsr (4) Versions 0..5
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey StopReplica (5) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey UpdateMetadata (6) Versions 0..7
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ControlledShutdown (7) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey OffsetCommit (8) Versions 0..8
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey OffsetFetch (9) Versions 0..8
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey FindCoordinator (10) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey JoinGroup (11) Versions 0..7
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Heartbeat (12) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey LeaveGroup (13) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey SyncGroup (14) Versions 0..5
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeGroups (15) Versions 0..5
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ListGroups (16) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey SaslHandshake (17) Versions 0..1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ApiVersion (18) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey CreateTopics (19) Versions 0..7
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DeleteTopics (20) Versions 0..6
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DeleteRecords (21) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey InitProducerId (22) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey OffsetForLeaderEpoch (23) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AddPartitionsToTxn (24) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AddOffsetsToTxn (25) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey EndTxn (26) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey WriteTxnMarkers (27) Versions 0..1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey TxnOffsetCommit (28) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeAcls (29) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey CreateAcls (30) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DeleteAcls (31) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeConfigs (32) Versions 0..4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterConfigs (33) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterReplicaLogDirs (34) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeLogDirs (35) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey SaslAuthenticate (36) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey CreatePartitions (37) Versions 0..3
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey CreateDelegationToken (38) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey RenewDelegationToken (39) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ExpireDelegationToken (40) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeDelegationToken (41) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DeleteGroups (42) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ElectLeadersRequest (43) Versions 0..2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey IncrementalAlterConfigsRequest (44) Versions 0..1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterPartitionReassignmentsRequest (45) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey ListPartitionReassignmentsRequest (46) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey OffsetDeleteRequest (47) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeClientQuotasRequest (48) Versions 0..1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterClientQuotasRequest (49) Versions 0..1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey DescribeUserScramCredentialsRequest (50) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterUserScramCredentialsRequest (51) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey AlterIsrRequest (56) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey UpdateFeaturesRequest (57) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Unknown-60? (60) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Unknown-61? (61) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Unknown-65? (65) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Unknown-66? (66) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:   ApiKey Unknown-67? (67) Versions 0..0
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature MsgVer1: Produce (2..2) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature MsgVer1: Fetch (2..2) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature MsgVer1
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature MsgVer2: Produce (3..3) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature MsgVer2: Fetch (4..4) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature MsgVer2
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature ApiVersion: ApiVersion (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature ApiVersion
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerGroupCoordinator: FindCoordinator (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature BrokerGroupCoordinator
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: FindCoordinator (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: OffsetCommit (1..2) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: OffsetFetch (1..1) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: JoinGroup (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: SyncGroup (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: Heartbeat (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature BrokerBalancedConsumer: LeaveGroup (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature BrokerBalancedConsumer
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature ThrottleTime: Produce (1..2) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature ThrottleTime: Fetch (1..2) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature ThrottleTime
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature Sasl: JoinGroup (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature Sasl
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature SaslHandshake: SaslHandshake (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature SaslHandshake
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature LZ4: FindCoordinator (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature LZ4
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature OffsetTime: ListOffsets (1..1) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature OffsetTime
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature IdempotentProducer: InitProducerId (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature IdempotentProducer
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature ZSTD: Produce (7..7) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature ZSTD: Fetch (10..10) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature ZSTD
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature SaslAuthReq: SaslHandshake (1..1) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2:  Feature SaslAuthReq: SaslAuthenticate (0..0) supported by broker
%7|1651057993.739|APIVERSION|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Enabling feature SaslAuthReq
%7|1651057993.739|FEATURE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Updated enabled protocol features to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime,MsgVer2,IdempotentProducer,ZSTD,SaslAuthReq
%7|1651057993.739|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state APIVERSION_QUERY -> UP
%7|1651057993.739|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057993.739|METADATA|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Skipping metadata refresh of 1 topic(s): connected: already being requested
%7|1651057993.739|OFFSET|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: ListOffsetsRequest (v2, opv 3) for 1 topic(s) and 1 partition(s)
%7|1651057993.739|SEND|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Sent ListOffsetsRequest (v2, 55 bytes @ 0, CorrId 2)
%7|1651057993.739|OFFSET|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: ListOffsetsRequest (v2, opv 3) for 1 topic(s) and 1 partition(s)
%7|1651057993.739|SEND|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Sent ListOffsetsRequest (v2, 55 bytes @ 0, CorrId 3)
%7|1651057993.741|RECV|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received ListOffsetsResponse (v2, 43 bytes, CorrId 2, rtt 1.74ms)
%7|1651057993.741|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Offset reply for topic storage [0] (v3 vs v3)
%7|1651057993.741|REQERR|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: ListOffsetsRequest failed: Broker: Unknown topic or partition: explicit actions Permanent
%7|1651057993.741|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: OffsetRequest failed: Broker: Unknown topic or partition (Permanent)
%7|1651057993.741|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Offset reply error for topic storage [0] (v3, Broker: Unknown topic or partition): Permanent
%7|1651057993.741|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state offset-wait -> none
%7|1651057993.741|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [0]: offset reset (at offset BEGINNING, broker 2) to INVALID: failed to query logical offset: Broker: Unknown topic or partition
%7|1651057993.742|RECV|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received ListOffsetsResponse (v2, 43 bytes, CorrId 3, rtt 3.37ms)
%7|1651057993.742|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Offset reply for topic storage [1] (v3 vs v3)
%7|1651057993.742|REQERR|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: ListOffsetsRequest failed: Broker: Unknown topic or partition: explicit actions Permanent
%7|1651057993.742|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: OffsetRequest failed: Broker: Unknown topic or partition (Permanent)
%7|1651057993.742|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Offset reply error for topic storage [1] (v3, Broker: Unknown topic or partition): Permanent
%7|1651057993.742|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state offset-wait -> none
%7|1651057993.742|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [1]: offset reset (at offset BEGINNING, broker 2) to INVALID: failed to query logical offset: Broker: Unknown topic or partition
%7|1651057993.836|RETRY|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Moved 1 retry buffer(s) to output queue
%7|1651057993.929|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: broker in state TRY_CONNECT connecting
%7|1651057993.929|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057993.929|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.929|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connecting to ipv4#127.0.0.1:9095 (plaintext) with socket 18
%7|1651057993.929|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT, 1 identical error(s) suppressed) (_TRANSPORT): identical to last error
%3|1651057993.929|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT, 1 identical error(s) suppressed)
%7|1651057993.929|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state CONNECT -> DOWN
%7|1651057993.929|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.929|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1651057993.929|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 1 buffers
%7|1651057993.929|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1651057993.929|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1651057993.929|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.929|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state INIT -> TRY_CONNECT
%7|1651057993.929|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057993.929|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 714ms
%7|1651057993.929|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply for topic storage [2] (v3 vs v3)
%7|1651057993.930|REQERR|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: ListOffsetsRequest failed: Local: Broker transport failure: explicit actions Retry
%7|1651057993.930|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: OffsetRequest failed: Local: Broker transport failure (Retry)
%7|1651057993.930|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply error for topic storage [2] (v3, Local: Operation in progress): Retry
%7|1651057993.930|RETRY|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Retrying ListOffsetsRequest (v0, 0 bytes, retry 0/2, prev CorrId 0) in 100ms
%7|1651057993.930|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 714ms
%7|1651057994.030|RETRY|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Moved 1 retry buffer(s) to output queue
%7|1651057994.235|QRYLEADER|rdkafka#consumer-1| [thrd:main]: Topic storage [2]: broker is down: re-query
%7|1651057994.235|METADATA|rdkafka#consumer-1| [thrd:main]: Requesting metadata for 1/1 topics: refresh unavailable topics
%7|1651057994.235|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: Request metadata for 1 topic(s): refresh unavailable topics
%7|1651057994.235|SEND|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Sent MetadataRequest (v4, 35 bytes @ 0, CorrId 4)
%7|1651057994.237|RECV|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received MetadataResponse (v4, 56 bytes, CorrId 4, rtt 1.87ms)
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: ===== Received metadata (for 1 requested topics): refresh unavailable topics =====
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: ClusterId: tlRh8oKOSOO_jV1WfuggcA, ControllerId: -1
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: 0 brokers, 1 topics
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2:   Topic #0/1: storage with 0 partitions: Broker: Unknown topic or partition
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: Error in metadata reply for topic storage (PartCnt 0): Broker: Unknown topic or partition
%7|1651057994.237|STATE|rdkafka#consumer-1| [thrd:main]: Topic storage changed state exists -> notexists
%5|1651057994.237|PARTCNT|rdkafka#consumer-1| [thrd:main]: Topic storage partition count changed from 3 to 0
%7|1651057994.237|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [0] no longer reported in metadata
%7|1651057994.237|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [0] is desired but no longer known: moving back on desired list
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegate to broker (none) (rktp 0x7f73e8001770, term 0, ref 8)
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: no longer delegated to broker localhost:9094/2
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [0]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
%7|1651057994.237|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [0] 0x7f73e8001770 from localhost:9094/2 to :0/internal (sending PARTITION_LEAVE to localhost:9094/2)
%7|1651057994.237|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [1] no longer reported in metadata
%7|1651057994.237|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [1] is desired but no longer known: moving back on desired list
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: delegate to broker (none) (rktp 0x7f73e80022f0, term 0, ref 8)
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: no longer delegated to broker localhost:9094/2
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [1]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
%7|1651057994.237|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [1] 0x7f73e80022f0 from localhost:9094/2 to :0/internal (sending PARTITION_LEAVE to localhost:9094/2)
%7|1651057994.237|REMOVE|rdkafka#consumer-1| [thrd:main]: storage [2] no longer reported in metadata
%7|1651057994.237|DESIRED|rdkafka#consumer-1| [thrd:main]: Topic storage [2] is desired but no longer known: moving back on desired list
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegate to broker (none) (rktp 0x7f73e8002a60, term 0, ref 7)
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: no longer delegated to broker localhost:9095/3
%7|1651057994.237|BRKDELGT|rdkafka#consumer-1| [thrd:main]: storage [2]: delegating to broker :0/internal for partition with 0 messages (0 bytes) queued
%7|1651057994.237|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [2]: migrating to new broker: (re)starting offset query timer for offset BEGINNING
%7|1651057994.237|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state offset-wait -> offset-query
%7|1651057994.237|BRKMIGR|rdkafka#consumer-1| [thrd:main]: Migrating topic storage [2] 0x7f73e8002a60 from localhost:9095/3 to :0/internal (sending PARTITION_LEAVE to localhost:9095/3)
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Topic storage [2]: leaving broker (0 messages in xmitq, next broker :0/internal, rktp 0x7f73e8002a60)
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: joining broker (rktp 0x7f73e8002a60, 0 message(s) queued)
%7|1651057994.237|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 407ms
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.237|METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: 1/1 requested topic(s) seen in metadata
%7|1651057994.237|CONTROLLERID|rdkafka#consumer-1| [thrd:main]: localhost:9094/2: ControllerId update 3 -> -1
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [0]: leaving broker (0 messages in xmitq, next broker :0/internal, rktp 0x7f73e8001770)
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Topic storage [1]: leaving broker (0 messages in xmitq, next broker :0/internal, rktp 0x7f73e80022f0)
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: joining broker (rktp 0x7f73e8001770, 0 message(s) queued)
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.237|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [1]: joining broker (rktp 0x7f73e80022f0, 0 message(s) queued)
%7|1651057994.237|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.337|METADATA|rdkafka#consumer-1| [thrd:main]: Expired 1 entries from metadata cache (0 entries remain)
%7|1651057994.644|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: broker in state TRY_CONNECT connecting
%7|1651057994.644|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state TRY_CONNECT -> CONNECT
%7|1651057994.644|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.644|CONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connecting to ipv4#127.0.0.1:9095 (plaintext) with socket 18
%7|1651057994.644|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT) (_TRANSPORT): identical to last error: error log suppressed
%7|1651057994.644|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state CONNECT -> DOWN
%7|1651057994.644|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.644|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1651057994.644|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 1 buffers
%7|1651057994.644|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1651057994.644|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1651057994.644|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.644|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state INIT -> TRY_CONNECT
%7|1651057994.644|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.644|RECONNECT|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Delaying next reconnect by 2691ms
%7|1651057994.644|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply for topic storage [2] (v3 vs v3)
%7|1651057994.644|OFFSET|rdkafka#consumer-1| [thrd:main]: localhost:9095/3: Offset reply error for topic storage [2] (v3, Local: Outdated):
%7|1651057994.644|OFFSET|rdkafka#consumer-1| [thrd:main]: storage [2]: outdated offset response: offset query timer already scheduled for offset BEGINNING
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1651057994.736|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 3 partition(s)
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=3, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 2 partition(s):
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [0] offset BEGINNING
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_fetch_stop:2363: new version barrier v4
%7|1651057994.736|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [0] (v4)
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [0]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v5
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [0] (v5)
%7|1651057994.736|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [0]
%7|1651057994.736|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [0] 0x7f73e8001770 sending final LEAVE for removal by :0/internal
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [0] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1651057994.736|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 2 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op FETCH_STOP (v4) in fetch-state none (opv3)
%7|1651057994.736|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [0] in state none (v4)
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state none -> stopping
%7|1651057994.736|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [0]: offset store terminating
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [0] changed fetch state stopping -> stopped
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [0] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [0]: partition is not paused by library
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [0]
%7|1651057994.736|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [0]
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f73e8001770)
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [0]: no next broker, failing 0 message(s) in partition queue
%7|1651057994.736|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.736|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=2, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 2 partition(s):
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1651057994.736|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 2 partition(s)
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=2, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [1] offset BEGINNING
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_fetch_stop:2363: new version barrier v4
%7|1651057994.736|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [1] (v4)
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [1]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v5
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [1] (v5)
%7|1651057994.736|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [1]
%7|1651057994.736|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [1] 0x7f73e80022f0 sending final LEAVE for removal by :0/internal
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [1] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1651057994.736|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 1 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [1]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f73e80022f0)
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [1]: no next broker, failing 0 message(s) in partition queue
%7|1651057994.736|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op FETCH_STOP (v4) in fetch-state none (opv3)
%7|1651057994.736|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [1] in state none (v4)
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state none -> stopping
%7|1651057994.736|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [1]: offset store terminating
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [1] changed fetch state stopping -> stopped
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [1] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [1]: partition is not paused by library
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [1]
%7|1651057994.736|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [1]
%7|1651057994.736|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op ASSIGN in state up (join-state init)
%7|1651057994.736|REMOVEASSIGN|rdkafka#consumer-1| [thrd:main]: Removed 1 partition(s) (0 with outstanding offset queries) from assignment of 1 partition(s)
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 1 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]:  storage [2] offset BEGINNING
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_fetch_stop:2363: new version barrier v4
%7|1651057994.736|CONSUMER|rdkafka#consumer-1| [thrd:main]: Stop consuming storage [2] (v4)
%7|1651057994.736|BARRIER|rdkafka#consumer-1| [thrd:main]: storage [2]: rd_kafka_toppar_op_pause_resume:2424: new version barrier v5
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Resume storage [2] (v5)
%7|1651057994.736|DESP|rdkafka#consumer-1| [thrd:main]: Removing (un)desired topic storage [2]
%7|1651057994.736|BRKMIGR|rdkafka#consumer-1| [thrd:main]: storage [2] 0x7f73e8002a60 sending final LEAVE for removal by :0/internal
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Removing storage [2] from assignment (started=true, pending=false, queried=false, stored offset=INVALID)
%7|1651057994.736|REMOVE|rdkafka#consumer-1| [thrd:main]: Served 1 removed partition(s), with 0 offset(s) to commit
%7|1651057994.736|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Current assignment of 0 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 0 offset commits in progress
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x7f73e8002a60)
%7|1651057994.736|TOPBRK|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Topic storage [2]: no next broker, failing 0 message(s) in partition queue
%7|1651057994.736|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op FETCH_STOP (v4) in fetch-state offset-query (opv3)
%7|1651057994.736|FETCH|rdkafka#consumer-1| [thrd:main]: Stopping fetch for storage [2] in state offset-query (v4)
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state offset-query -> stopping
%7|1651057994.736|STORETERM|rdkafka#consumer-1| [thrd:main]: storage [2]: offset store terminating
%7|1651057994.736|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition storage [2] changed fetch state stopping -> stopped
%7|1651057994.736|OP|rdkafka#consumer-1| [thrd:main]: storage [2] received op PAUSE (v5) in fetch-state stopped (opv4)
%7|1651057994.736|RESUME|rdkafka#consumer-1| [thrd:main]: Not resuming storage [2]: partition is not paused by library
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op PARTITION_LEAVE in state up (join-state init) for storage [2]
%7|1651057994.736|PARTDEL|rdkafka#consumer-1| [thrd:main]: Group "1": delete storage [2]
%7|1651057994.736|STOPSERVE|rdkafka#consumer-1| [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%3|1651057994.736|ERROR|rdkafka#consumer-1| [thrd:app]: rdkafka#consumer-1: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT)
%3|1651057994.736|ERROR|rdkafka#consumer-1| [thrd:app]: rdkafka#consumer-1: localhost:9095/3: Connect to ipv4#127.0.0.1:9095 failed: Connection refused (after 0ms in state CONNECT, 1 identical error(s) suppressed)
%7|1651057994.736|DESTROY|rdkafka#consumer-1| [thrd:app]: Terminating instance (destroy flags NoConsumerClose (0x8))
%7|1651057994.736|TERMINATE|rdkafka#consumer-1| [thrd:app]: Terminating consumer group handler
%7|1651057994.736|CLOSE|rdkafka#consumer-1| [thrd:app]: Closing consumer
%7|1651057994.736|CLOSE|rdkafka#consumer-1| [thrd:app]: Disabling and purging temporary queue to quench close events
%7|1651057994.736|CLOSE|rdkafka#consumer-1| [thrd:app]: Consumer closed
%7|1651057994.736|TERMINATE|rdkafka#consumer-1| [thrd:app]: Interrupting timers
%7|1651057994.736|TERMINATE|rdkafka#consumer-1| [thrd:app]: Sending TERMINATE to internal main thread
%7|1651057994.736|TERMINATE|rdkafka#consumer-1| [thrd:app]: Joining internal main thread
%7|1651057994.736|CGRPOP|rdkafka#consumer-1| [thrd:main]: Group "1" received op TERMINATE in state up (join-state init)
%7|1651057994.736|CGRPTERM|rdkafka#consumer-1| [thrd:main]: Terminating group "1" in state up with 0 partition(s)
%7|1651057994.736|CLEARASSIGN|rdkafka#consumer-1| [thrd:main]: No current assignment to clear
%7|1651057994.736|DUMP|rdkafka#consumer-1| [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
%7|1651057994.736|DUMP_ALL|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_PND|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_QRY|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|DUMP_REM|rdkafka#consumer-1| [thrd:main]: List with 0 partition(s):
%7|1651057994.736|ASSIGNDONE|rdkafka#consumer-1| [thrd:main]: Group "1": assignment operations done in join-state init (rebalance rejoin=false)
%7|1651057994.737|CGRPSTATE|rdkafka#consumer-1| [thrd:main]: Group "1" changed state up -> term (join-state init)
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057994.737|ASSIGNMENT|rdkafka#consumer-1| [thrd:main]: Group "1": clearing group assignment
%7|1651057994.737|COORDCLEAR|rdkafka#consumer-1| [thrd:main]: Group "1" broker localhost:9094/2 is no longer coordinator
%7|1651057994.737|NODENAME|rdkafka#consumer-1| [thrd:main]: GroupCoordinator/2: Broker nodename changed from "localhost:9094" to ""
%7|1651057994.737|NODEID|rdkafka#consumer-1| [thrd:main]: GroupCoordinator/2: Broker nodeid changed from 2 to -1
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:main]: Internal main thread terminating
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Destroy internal
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:main]: Broadcasting state change
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Removing all topics
%7|1651057994.737|TOPPARREMOVE|rdkafka#consumer-1| [thrd:main]: Removing toppar storage [-1] 0x7f73e8001160
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: storage [-1]: 0x7f73e8001160 DESTROY_FINAL
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to 127.0.0.1:9093/1
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Closing connection due to nodename change (after 1493ms in state UP) (_TRANSPORT)
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state UP -> DOWN
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9095/3
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|METADATA|rdkafka#consumer-1| [thrd:GroupCoordinator]: Requesting metadata for 1/1 topics: broker down
%7|1651057994.737|METADATA|rdkafka#consumer-1| [thrd:GroupCoordinator]: localhost:9094/2: Request metadata for 1 topic(s): broker down
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Client is terminating (after 1497ms in state INIT) (_DESTROY)
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Broker changed state INIT -> DOWN
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9094/2
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Received TERMINATE op in state TRY_CONNECT: 3 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Client is terminating (after 92ms in state TRY_CONNECT) (_DESTROY)
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state TRY_CONNECT -> DOWN
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 3 refcnts (0x55d5056e9a88), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Handle is terminating in state DOWN: 1 refcnts (0x7f73e80043d8), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state UP: 5 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 1 request(s) in retry+outbuf
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to localhost:9093/bootstrap
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:127.0.0.1:9093/1]: 127.0.0.1:9093/1: Updating 0 buffers on connection reset
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state DOWN -> INIT
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Received TERMINATE op in state UP: 4 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|DESTROY|rdkafka#consumer-1| [thrd:main]: Sending TERMINATE to GroupCoordinator
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Broker changed state DOWN -> INIT
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Client is terminating (after 998ms in state UP) (_DESTROY)
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: Broadcasting state change
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Client is terminating (after 0ms in state INIT) (_DESTROY)
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state INIT -> DOWN
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:main]: Purging reply queue
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state UP -> DOWN
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:main]: Decommissioning internal broker
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Received TERMINATE op in state UP: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:main]: Join 6 broker thread(s)
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057994.737|TERM|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Client is terminating (after 1499ms in state UP) (_DESTROY)
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker changed state UP -> DOWN
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Purging bufq with 0 buffers
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Client is terminating (after 1502ms in state INIT) (_DESTROY)
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:GroupCoordinator]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Purging bufq with 0 buffers
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Broker changed state INIT -> DOWN
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd::0/internal]: Broadcasting state change
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: terminating: broker still has 4 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state DOWN: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|STATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Broker changed state DOWN -> INIT
%7|1651057994.737|BROADCAST|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: Broadcasting state change
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Handle is terminating in state DOWN: 1 refcnts (0x55d5056eb788), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:localhost:9093/bootstrap]: localhost:9093/bootstrap: Updating 0 buffers on connection reset
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 1 refcnts (0x55d5056e9a88), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|BRKTERM|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
%7|1651057994.737|TERMINATE|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Handle is terminating in state DOWN: 1 refcnts (0x55d5056ea9b8), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.737|FAIL|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
%7|1651057994.737|BUFQ|rdkafka#consumer-1| [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
%7|1651057994.738|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.738|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.739|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.739|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.740|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.740|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.741|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.742|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.743|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.743|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.744|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.744|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.745|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.745|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.746|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.746|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.747|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.747|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.748|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.749|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.750|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.750|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.751|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.751|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.752|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.752|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.753|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.753|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.754|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.754|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.755|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.755|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.756|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.756|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.758|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.758|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.759|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.759|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.760|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.760|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.761|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.761|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.762|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.762|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.763|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.763|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.764|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.764|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.766|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.766|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.767|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.767|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.768|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.768|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.769|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.769|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.770|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.770|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.771|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9095/bootstrap]: localhost:9095/3: Handle is terminating in state INIT: 2 refcnts (0x55d5056ed528), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
%7|1651057994.771|TERMINATE|rdkafka#consumer-1| [thrd:localhost:9094/bootstrap]: localhost:9094/2: Handle is terminating in state INIT: 3 refcnts (0x55d5056ec658), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf

pranasge avatar Apr 27 '22 11:04 pranasge

So I was able to replicate this similar issue in k8s. It happened in a dev cluster during a failure test where the node was shutdown then brought back up. There were multiple apps with the same base image and using the same node-rdkafka (which uses librdkafka under the hood) and they all were in a stuck state trying to connect to the producer. The logs looked similar to the ones @pranasge showed. I was able to replicate this more than once so it's possible to get it in the state if you are trying.

To replicate here are the steps to follow:

  1. Spin up a k8s cluster using kind (follow the steps in this guide kind k8s cluster setup)
  2. Once the cluster is up deploy the app running the kafka producer (in my case I was using a nodejs app and I was using node-rdkafka as my kafka client).
  3. While the app is running check the node the app is running on in k8s kubectl get nodes
  4. Shutdown the node and after you shutdown the node disconnect the network connection to the kafka broker (In my case after I shutdown the node by stopping the container I just turned off my vpn which gave access to my kafka broker).
  5. Once the app has been migrated to the other node and is running you can check the logs and see if it connected (It shouldn't be connected because it doesn't have access to the broker so you can now turn the connection back on and then watch it fail to connect)

The app should be stuck in the producer connecting state and shouldn't really crash but it wouldn't do anything. A way to get it out is to just reboot the container and it should recover.

Node version: v14.19.1 noderdkafka_version: 2.12.0 librdkafka_version: 1.7.0

kind version: v0.12.0 go1.17.8 darwin/amd64

OS: Mac OS Big Sir Version 11.6 Docker Desktop Version: 4.5.0 (74594)

o2themar avatar Apr 27 '22 18:04 o2themar

I believe I can reproduce in a rather simple scenraio. when running the e2e test on node-rdkafka it happens in afterEach hook of the both.spec.js:282

In case you want to jump on a session on my machine - I'll keep the setup and try to dig into it further.

i modified the afterEach hook to look like this:

  afterEach(function(done) {
    console.log('after each started');
    var finished = 0;
    var called = false;

    function maybeDone(err) {
      console.log('why err', err);
      if (called) {
        return;
      }

      finished++;
      if (err) {
        called = true;
        return done(err);
      }

      if (finished === 2) {
        done();
      }
    }

    console.log('start disconnect');
    producer.disconnect(function(err) {
      maybeDone(err);
      consumer.disconnect(function(err) {
        maybeDone(err);
      });
    });
  });

the actual test:

  it('should be able to produce and consume messages: consumeLoop', function(done) {
    console.log('start test');
    var key = 'key';

    crypto.randomBytes(4096, function(ex, buffer) {
      console.log('rannd created');

      producer.setPollInterval(10);

      producer.once('delivery-report', function(err, report) {
        if (!err) {
          t.equal(topic, report.topic, 'invalid delivery-report topic');
          t.equal(key, report.key, 'invalid delivery-report key');
          t.ok(report.offset >= 0, 'invalid delivery-report offset');
        }
      });

      consumer.on('data', function(message) {
        console.log('on data');
        t.equal(buffer.toString(), message.value.toString(), 'invalid message value');
        t.equal(key, message.key, 'invalid message key');
        t.equal(topic, message.topic, 'invalid message topic');
        t.ok(message.offset >= 0, 'invalid message offset');
        done();
      });

      consumer.subscribe([topic]);
      consumer.consume();

      setTimeout(function() {
        console.log('PRODUCE - should be able to produce and consume messages: consumeLoop')
        producer.produce(topic, null, buffer, key);
      }, 2000);

    });
  });

the debug logs:

kafka-mocha#producer-17 connected to kafka server
rdkafka#consumer-16 connected to kafka server
start test
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Heartbeat (12) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey LeaveGroup (13) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey SyncGroup (14) Versions 0..5
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeGroups (15) Versions 0..5
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey ListGroups (16) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey SaslHandshake (17) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey ApiVersion (18) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey CreateTopics (19) Versions 0..7
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DeleteTopics (20) Versions 0..6
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DeleteRecords (21) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey InitProducerId (22) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey OffsetForLeaderEpoch (23) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AddPartitionsToTxn (24) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AddOffsetsToTxn (25) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey EndTxn (26) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey WriteTxnMarkers (27) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey TxnOffsetCommit (28) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeAcls (29) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey CreateAcls (30) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DeleteAcls (31) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeConfigs (32) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterConfigs (33) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterReplicaLogDirs (34) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeLogDirs (35) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey SaslAuthenticate (36) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey CreatePartitions (37) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey CreateDelegationToken (38) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey RenewDelegationToken (39) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey ExpireDelegationToken (40) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeDelegationToken (41) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DeleteGroups (42) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey ElectLeadersRequest (43) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey IncrementalAlterConfigsRequest (44) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterPartitionReassignmentsRequest (45) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey ListPartitionReassignmentsRequest (46) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey OffsetDeleteRequest (47) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeClientQuotasRequest (48) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterClientQuotasRequest (49) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey DescribeUserScramCredentialsRequest (50) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterUserScramCredentialsRequest (51) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey AlterIsrRequest (56) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey UpdateFeaturesRequest (57) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Unknown-60? (60) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Unknown-61? (61) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Unknown-65? (65) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Unknown-66? (66) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:   ApiKey Unknown-67? (67) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature MsgVer1: Produce (2..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature MsgVer1: Fetch (2..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature MsgVer1
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature MsgVer2: Produce (3..3) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature MsgVer2: Fetch (4..4) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature MsgVer2
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature ApiVersion: ApiVersion (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature ApiVersion
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerGroupCoordinator: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature BrokerGroupCoordinator
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: OffsetCommit (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: OffsetFetch (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: JoinGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: SyncGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: Heartbeat (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature BrokerBalancedConsumer: LeaveGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature BrokerBalancedConsumer
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature ThrottleTime: Produce (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature ThrottleTime: Fetch (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature ThrottleTime
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature Sasl: JoinGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature Sasl
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature SaslHandshake: SaslHandshake (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature SaslHandshake
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature LZ4: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature LZ4
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature OffsetTime: ListOffsets (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature OffsetTime
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature IdempotentProducer: InitProducerId (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature IdempotentProducer
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature ZSTD: Produce (7..7) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature ZSTD: Fetch (10..10) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature ZSTD
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature SaslAuthReq: SaslHandshake (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap:  Feature SaslAuthReq: SaslAuthenticate (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Enabling feature SaslAuthReq
{ severity: 7, fac: 'FEATURE' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Updated enabled protocol features to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime,MsgVer2,IdempotentProducer,ZSTD,SaslAuthReq
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Broker changed state APIVERSION_QUERY -> UP
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'METADATA' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Request metadata for brokers only: connected
{ severity: 7, fac: 'METADATA' } [thrd:app]: localhost:9092/bootstrap: Request metadata for all topics: application requested
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Sent MetadataRequest (v4, 30 bytes @ 0, CorrId 2)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Sent MetadataRequest (v4, 30 bytes @ 0, CorrId 3)
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Received MetadataResponse (v4, 61 bytes, CorrId 2, rtt 2.89ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: ===== Received metadata: connected =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: 1 brokers, 0 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap:   Broker #0/1: localhost:9092 NodeId 1
{ severity: 7, fac: 'UPDATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: NodeId changed from -1 to 1
{ severity: 7, fac: 'UPDATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Name changed from localhost:9092/bootstrap to localhost:9092/1
{ severity: 7, fac: 'LEADER' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Mapped 0 partition(s) to broker
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UP -> UPDATE
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UPDATE -> UP
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'CLUSTERID' } [thrd:main]: localhost:9092/1: ClusterId update "" -> "cP7EsIPWS4eZiKZcsK51kQ"
{ severity: 7, fac: 'CONTROLLERID' } [thrd:main]: localhost:9092/1: ControllerId update -1 -> 1
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received MetadataResponse (v4, 15216 bytes, CorrId 3, rtt 3.85ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: ===== Received metadata: application requested =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: 1 brokers, 67 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Broker #0/1: localhost:9092 NodeId 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #0/67: admin-test-topic-newparts2-1657239017952 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #1/67: admin-test-topic-newparts-1657237453679 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #2/67: admin-test-topic-1657238382469 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #3/67: admin-test-topic-newparts-1657237059590 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #4/67: transaction_output_1657237453721_382 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #5/67: test4 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #6/67: admin-test-topic-newparts2-1657239474141 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #7/67: admin-test-topic-2bdeleted-1657237453679 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #8/67: test2 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #9/67: admin-test-topic-newparts2-1657238856833 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #10/67: admin-test-topic-1657238169695 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #11/67: test6 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #12/67: admin-test-topic-2bdeleted-1657239017952 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #13/67: admin-test-topic-newparts-1657239474141 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #14/67: test with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #15/67: transaction_input_1657237453721_382 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #16/67: admin-test-topic-2bdeleted-1657238031002 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #17/67: admin-test-topic-newparts-1657238169695 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #18/67: __transaction_state with 50 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #19/67: admin-test-topic-2bdeleted-1657238257447 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #20/67: admin-test-topic-newparts2-1657236691954 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #21/67: admin-test-topic-1657237059590 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #22/67: admin-test-topic-newparts-1657238382469 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #23/67: test3 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #24/67: admin-test-topic-newparts-1657237285122 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #25/67: admin-test-topic-1657237285122 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #26/67: admin-test-topic-2bdeleted-1657236826605 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #27/67: admin-test-topic-newparts2-1657238169695 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #28/67: admin-test-topic-newparts2-1657236826605 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #29/67: admin-test-topic-2bdeleted-1657237285122 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #30/67: admin-test-topic-2bdeleted-1657238169695 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #31/67: test7 with 2 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #32/67: admin-test-topic-newparts-1657238277180 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #33/67: admin-test-topic-newparts-1657236691954 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #34/67: admin-test-topic-newparts2-1657239533960 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #35/67: admin-test-topic-1657238856833 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #36/67: admin-test-topic-newparts2-1657237059590 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #37/67: admin-test-topic-1657239474141 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #38/67: __consumer_offsets with 50 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #39/67: admin-test-topic-1657238257447 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #40/67: admin-test-topic-1657237453679 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #41/67: admin-test-topic-1657236691954 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #42/67: admin-test-topic-newparts2-1657237285122 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #43/67: admin-test-topic-1657236826605 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #44/67: admin-test-topic-newparts-1657239533960 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #45/67: admin-test-topic-2bdeleted-1657238382469 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #46/67: admin-test-topic-2bdeleted-1657238277180 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #47/67: admin-test-topic-1657239017952 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #48/67: admin-test-topic-2bdeleted-1657238856833 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #49/67: admin-test-topic-2bdeleted-1657236691954 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #50/67: admin-test-topic-newparts-1657236826605 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #51/67: admin-test-topic-1657239533960 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #52/67: admin-test-topic-2bdeleted-1657239533960 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #53/67: admin-test-topic-newparts2-1657237453679 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #54/67: admin-test-topic-2bdeleted-1657237059590 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #55/67: admin-test-topic-newparts2-1657238031002 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #56/67: admin-test-topic-newparts2-1657238382469 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #57/67: test5 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #58/67: admin-test-topic-newparts2-1657238257447 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #59/67: admin-test-topic-1657238277180 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #60/67: admin-test-topic-newparts-1657238856833 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #61/67: admin-test-topic-1657238031002 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #62/67: admin-test-topic-newparts-1657238031002 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #63/67: admin-test-topic-2bdeleted-1657239474141 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #64/67: admin-test-topic-newparts-1657238257447 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #65/67: admin-test-topic-newparts-1657239017952 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #66/67: admin-test-topic-newparts2-1657238277180 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: Absolute update of metadata cache with 67 topic(s)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: Caching full metadata with 1 broker(s) and 67 topic(s): application requested
{ severity: 7, fac: 'CGRPQUERY' } [thrd:main]: localhost:9092/bootstrap: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": querying for coordinator: intervaled in state query-coord
{ severity: 7, fac: 'METADATA' } [thrd:app]: localhost:9092/bootstrap: Request metadata for all topics: application requested
{ severity: 7, fac: 'CGRPSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed state query-coord -> wait-coord (join-state init)
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Sent MetadataRequest (v4, 26 bytes @ 0, CorrId 2)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Sent MetadataRequest (v4, 26 bytes @ 0, CorrId 3)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Sent FindCoordinatorRequest (v2, 80 bytes @ 0, CorrId 4)
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: Received MetadataResponse (v4, 61 bytes, CorrId 2, rtt 2.73ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: ===== Received metadata: connected =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap: 1 brokers, 0 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/bootstrap:   Broker #0/1: localhost:9092 NodeId 1
{ severity: 7, fac: 'UPDATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/bootstrap: NodeId changed from -1 to 1
{ severity: 7, fac: 'UPDATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Name changed from localhost:9092/bootstrap to localhost:9092/1
{ severity: 7, fac: 'LEADER' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Mapped 0 partition(s) to broker
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UP -> UPDATE
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UPDATE -> UP
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'CLUSTERID' } [thrd:main]: localhost:9092/1: ClusterId update "" -> "cP7EsIPWS4eZiKZcsK51kQ"
{ severity: 7, fac: 'CONTROLLERID' } [thrd:main]: localhost:9092/1: ControllerId update -1 -> 1
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received MetadataResponse (v4, 15216 bytes, CorrId 3, rtt 4.26ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: ===== Received metadata: application requested =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: 1 brokers, 67 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Broker #0/1: localhost:9092 NodeId 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #0/67: admin-test-topic-newparts2-1657239017952 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #1/67: admin-test-topic-newparts-1657237453679 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #2/67: admin-test-topic-1657238382469 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #3/67: admin-test-topic-newparts-1657237059590 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #4/67: transaction_output_1657237453721_382 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #5/67: test4 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #6/67: admin-test-topic-newparts2-1657239474141 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #7/67: admin-test-topic-2bdeleted-1657237453679 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #8/67: test2 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #9/67: admin-test-topic-newparts2-1657238856833 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #10/67: admin-test-topic-1657238169695 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #11/67: test6 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #12/67: admin-test-topic-2bdeleted-1657239017952 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #13/67: admin-test-topic-newparts-1657239474141 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #14/67: test with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #15/67: transaction_input_1657237453721_382 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #16/67: admin-test-topic-2bdeleted-1657238031002 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #17/67: admin-test-topic-newparts-1657238169695 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #18/67: __transaction_state with 50 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #19/67: admin-test-topic-2bdeleted-1657238257447 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #20/67: admin-test-topic-newparts2-1657236691954 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #21/67: admin-test-topic-1657237059590 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #22/67: admin-test-topic-newparts-1657238382469 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #23/67: test3 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #24/67: admin-test-topic-newparts-1657237285122 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #25/67: admin-test-topic-1657237285122 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #26/67: admin-test-topic-2bdeleted-1657236826605 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #27/67: admin-test-topic-newparts2-1657238169695 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #28/67: admin-test-topic-newparts2-1657236826605 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #29/67: admin-test-topic-2bdeleted-1657237285122 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #30/67: admin-test-topic-2bdeleted-1657238169695 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #31/67: test7 with 2 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #32/67: admin-test-topic-newparts-1657238277180 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #33/67: admin-test-topic-newparts-1657236691954 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #34/67: admin-test-topic-newparts2-1657239533960 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #35/67: admin-test-topic-1657238856833 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #36/67: admin-test-topic-newparts2-1657237059590 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #37/67: admin-test-topic-1657239474141 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #38/67: __consumer_offsets with 50 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #39/67: admin-test-topic-1657238257447 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #40/67: admin-test-topic-1657237453679 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #41/67: admin-test-topic-1657236691954 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #42/67: admin-test-topic-newparts2-1657237285122 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #43/67: admin-test-topic-1657236826605 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #44/67: admin-test-topic-newparts-1657239533960 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #45/67: admin-test-topic-2bdeleted-1657238382469 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #46/67: admin-test-topic-2bdeleted-1657238277180 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #47/67: admin-test-topic-1657239017952 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #48/67: admin-test-topic-2bdeleted-1657238856833 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #49/67: admin-test-topic-2bdeleted-1657236691954 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #50/67: admin-test-topic-newparts-1657236826605 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #51/67: admin-test-topic-1657239533960 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #52/67: admin-test-topic-2bdeleted-1657239533960 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #53/67: admin-test-topic-newparts2-1657237453679 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #54/67: admin-test-topic-2bdeleted-1657237059590 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #55/67: admin-test-topic-newparts2-1657238031002 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #56/67: admin-test-topic-newparts2-1657238382469 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #57/67: test5 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #58/67: admin-test-topic-newparts2-1657238257447 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #59/67: admin-test-topic-1657238277180 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #60/67: admin-test-topic-newparts-1657238856833 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #61/67: admin-test-topic-1657238031002 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #62/67: admin-test-topic-newparts-1657238031002 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #63/67: admin-test-topic-2bdeleted-1657239474141 with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #64/67: admin-test-topic-newparts-1657238257447 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #65/67: admin-test-topic-newparts-1657239017952 with 20 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1:   Topic #66/67: admin-test-topic-newparts2-1657238277180 with 4 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: Absolute update of metadata cache with 67 topic(s)
{ severity: 7, fac: 'METADATA' } [thrd:main]: localhost:9092/1: Caching full metadata with 1 broker(s) and 67 topic(s): application requested
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received FindCoordinatorResponse (v2, 31 bytes, CorrId 4, rtt 5.25ms)
{ severity: 7, fac: 'CGRPCOORD' } [thrd:main]: localhost:9092/1: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" coordinator is localhost:9092 id 1
{ severity: 7, fac: 'CGRPCOORD' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changing coordinator -1 -> 1
{ severity: 7, fac: 'COORDSET' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" coordinator set to broker localhost:9092/1
{ severity: 7, fac: 'CGRPSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed state wait-coord -> wait-broker-transport (join-state init)
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'NODENAME' } [thrd:main]: GroupCoordinator: Broker nodename changed from "" to "localhost:9092"
{ severity: 7, fac: 'NODEID' } [thrd:main]: GroupCoordinator: Broker nodeid changed from -1 to 1
{ severity: 7, fac: 'CGRPQUERY' } [thrd:main]: localhost:9092/1: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": querying for coordinator: intervaled in state wait-broker-transport
{ severity: 7, fac: 'CONNECT' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received CONNECT op
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state INIT -> TRY_CONNECT
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'CONNECT' } [thrd:GroupCoordinator]: GroupCoordinator/1: broker in state TRY_CONNECT connecting
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state TRY_CONNECT -> CONNECT
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FindCoordinatorRequest (v2, 80 bytes @ 0, CorrId 5)
{ severity: 7, fac: 'CONNECT' } [thrd:GroupCoordinator]: GroupCoordinator/1: Connecting to ipv6#[::1]:9092 (plaintext) with socket 42
{ severity: 7, fac: 'CONNECT' } [thrd:GroupCoordinator]: GroupCoordinator/1: Connected to ipv6#[::1]:9092
{ severity: 7, fac: 'CONNECTED' } [thrd:GroupCoordinator]: GroupCoordinator/1: Connected (#1)
{ severity: 7, fac: 'FEATURE' } [thrd:GroupCoordinator]: GroupCoordinator/1: Updated enabled protocol features +ApiVersion to ApiVersion
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state CONNECT -> APIVERSION_QUERY
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent ApiVersionRequest (v3, 40 bytes @ 0, CorrId 1)
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received FindCoordinatorResponse (v2, 31 bytes, CorrId 5, rtt 2.15ms)
{ severity: 7, fac: 'CGRPCOORD' } [thrd:main]: localhost:9092/1: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" coordinator is localhost:9092 id 1
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received ApiVersionResponse (v3, 431 bytes, CorrId 1, rtt 2.97ms)
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Broker API support:
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Produce (0) Versions 0..9
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Fetch (1) Versions 0..13
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ListOffsets (2) Versions 0..7
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Metadata (3) Versions 0..12
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey LeaderAndIsr (4) Versions 0..5
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey StopReplica (5) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey UpdateMetadata (6) Versions 0..7
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ControlledShutdown (7) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey OffsetCommit (8) Versions 0..8
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey OffsetFetch (9) Versions 0..8
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey FindCoordinator (10) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey JoinGroup (11) Versions 0..7
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Heartbeat (12) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey LeaveGroup (13) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey SyncGroup (14) Versions 0..5
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeGroups (15) Versions 0..5
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ListGroups (16) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey SaslHandshake (17) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ApiVersion (18) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey CreateTopics (19) Versions 0..7
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DeleteTopics (20) Versions 0..6
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DeleteRecords (21) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey InitProducerId (22) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey OffsetForLeaderEpoch (23) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AddPartitionsToTxn (24) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AddOffsetsToTxn (25) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey EndTxn (26) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey WriteTxnMarkers (27) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey TxnOffsetCommit (28) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeAcls (29) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey CreateAcls (30) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DeleteAcls (31) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeConfigs (32) Versions 0..4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterConfigs (33) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterReplicaLogDirs (34) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeLogDirs (35) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey SaslAuthenticate (36) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey CreatePartitions (37) Versions 0..3
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey CreateDelegationToken (38) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey RenewDelegationToken (39) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ExpireDelegationToken (40) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeDelegationToken (41) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DeleteGroups (42) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ElectLeadersRequest (43) Versions 0..2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey IncrementalAlterConfigsRequest (44) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterPartitionReassignmentsRequest (45) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey ListPartitionReassignmentsRequest (46) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey OffsetDeleteRequest (47) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeClientQuotasRequest (48) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterClientQuotasRequest (49) Versions 0..1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey DescribeUserScramCredentialsRequest (50) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterUserScramCredentialsRequest (51) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey AlterIsrRequest (56) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey UpdateFeaturesRequest (57) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Unknown-60? (60) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Unknown-61? (61) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Unknown-65? (65) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Unknown-66? (66) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:   ApiKey Unknown-67? (67) Versions 0..0
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature MsgVer1: Produce (2..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature MsgVer1: Fetch (2..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature MsgVer1
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature MsgVer2: Produce (3..3) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature MsgVer2: Fetch (4..4) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature MsgVer2
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature ApiVersion: ApiVersion (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature ApiVersion
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerGroupCoordinator: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature BrokerGroupCoordinator
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: OffsetCommit (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: OffsetFetch (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: JoinGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: SyncGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: Heartbeat (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature BrokerBalancedConsumer: LeaveGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature BrokerBalancedConsumer
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature ThrottleTime: Produce (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature ThrottleTime: Fetch (1..2) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature ThrottleTime
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature Sasl: JoinGroup (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature Sasl
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature SaslHandshake: SaslHandshake (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature SaslHandshake
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature LZ4: FindCoordinator (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature LZ4
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature OffsetTime: ListOffsets (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature OffsetTime
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature IdempotentProducer: InitProducerId (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature IdempotentProducer
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature ZSTD: Produce (7..7) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature ZSTD: Fetch (10..10) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature ZSTD
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature SaslAuthReq: SaslHandshake (1..1) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1:  Feature SaslAuthReq: SaslAuthenticate (0..0) supported by broker
{ severity: 7, fac: 'APIVERSION' } [thrd:GroupCoordinator]: GroupCoordinator/1: Enabling feature SaslAuthReq
{ severity: 7, fac: 'FEATURE' } [thrd:GroupCoordinator]: GroupCoordinator/1: Updated enabled protocol features to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime,MsgVer2,IdempotentProducer,ZSTD,SaslAuthReq
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state APIVERSION_QUERY -> UP
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'METADATA' } [thrd:GroupCoordinator]: GroupCoordinator/1: Request metadata for brokers only: connected
{ severity: 7, fac: 'CGRPSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed state wait-broker-transport -> up (join-state init)
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'ASSIGNDONE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": assignment operations done in join-state init (rebalance rejoin=false)
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent MetadataRequest (v4, 26 bytes @ 0, CorrId 2)
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received MetadataResponse (v4, 61 bytes, CorrId 2, rtt 2.13ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: ===== Received metadata: connected =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: 1 brokers, 0 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1:   Broker #0/1: localhost:9092 NodeId 1
rannd created
{ severity: 7, fac: 'CGRPOP' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" received op SUBSCRIBE in state up (join-state init)
{ severity: 7, fac: 'SUBSCRIBE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": subscribe to new subscription of 1 topics (join-state init)
{ severity: 7, fac: 'JOIN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": join with 0 subscribed topic(s)
{ severity: 7, fac: 'CGRPMETADATA' } [thrd:main]: consumer join: metadata for subscription is up to date (22ms old)
{ severity: 7, fac: 'SUBSCRIPTION' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": effective subscription list changed from 0 to 1 topic(s):
{ severity: 7, fac: 'SUBSCRIPTION' } [thrd:main]:  Topic test with 1 partition(s)
{ severity: 7, fac: 'JOIN' } [thrd:main]: localhost:9092/1: Joining group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" with 1 subscribed topic(s) and member id ""
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state init -> wait-join (state up)
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent JoinGroupRequest (v5, 172 bytes @ 0, CorrId 3)
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received JoinGroupResponse (v5, 64 bytes, CorrId 3, rtt 2.56ms)
{ severity: 7, fac: 'JOINGROUP' } [thrd:main]: JoinGroup response: GenerationId -1, Protocol , LeaderId , my MemberId rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d, member metadata count 0: Broker: Group member needs a valid member ID
{ severity: 7, fac: 'REQERR' } [thrd:main]: GroupCoordinator/1: JoinGroupRequest failed: Broker: Group member needs a valid member ID: explicit actions Ignore
{ severity: 7, fac: 'MEMBERID' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": updating member id "" -> "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d"
{ severity: 7, fac: 'REJOIN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": Rejoining group without an assignment: JoinGroup error: Broker: Group member needs a valid member ID
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-join -> init (state up)
{ severity: 7, fac: 'JOIN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": join with 1 subscribed topic(s)
{ severity: 7, fac: 'CGRPMETADATA' } [thrd:main]: consumer join: metadata for subscription is up to date (25ms old)
{ severity: 7, fac: 'JOIN' } [thrd:main]: localhost:9092/1: Joining group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" with 1 subscribed topic(s) and member id "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d"
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state init -> wait-join (state up)
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent JoinGroupRequest (v5, 216 bytes @ 0, CorrId 4)
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received JoinGroupResponse (v5, 185 bytes, CorrId 4, rtt 3.35ms)
{ severity: 7, fac: 'JOINGROUP' } [thrd:main]: JoinGroup response: GenerationId 1, Protocol range, LeaderId rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d (me), my MemberId rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d, member metadata count 1: (no error)
{ severity: 7, fac: 'JOINGROUP' } [thrd:main]: I am elected leader for group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" with 1 member(s)
{ severity: 7, fac: 'GRPLEADER' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": resetting group leader info: JoinGroup response clean-up
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-join -> wait-metadata (state up)
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: Request metadata for 1 topic(s): partition assignor
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent MetadataRequest (v4, 32 bytes @ 0, CorrId 5)
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received MetadataResponse (v4, 100 bytes, CorrId 5, rtt 1.73ms)
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: ===== Received metadata (for 1 requested topics): partition assignor =====
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: ClusterId: cP7EsIPWS4eZiKZcsK51kQ, ControllerId: 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: 1 brokers, 1 topics
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1:   Broker #0/1: localhost:9092 NodeId 1
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1:   Topic #0/1: test with 1 partitions
{ severity: 7, fac: 'METADATA' } [thrd:main]: GroupCoordinator/1: 1/1 requested topic(s) seen in metadata
{ severity: 7, fac: 'ASSIGN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" running range assignor for 1 member(s) and 1 eligible subscribed topic(s):
{ severity: 7, fac: 'ASSIGN' } [thrd:main]:  Member "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d" (me) with 0 owned partition(s) and 1 subscribed topic(s):
{ severity: 7, fac: 'ASSIGN' } [thrd:main]:   test [-1]
{ severity: 7, fac: 'ASSIGN' } [thrd:main]: range: Topic test with 1 partition(s) and 1 subscribing member(s)
{ severity: 7, fac: 'ASSIGN' } [thrd:main]: range: Member "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d": assigned topic test partitions 0..0
{ severity: 7, fac: 'ASSIGN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" range assignment for 1 member(s) finished in 0.009ms:
{ severity: 7, fac: 'ASSIGN' } [thrd:main]:  Member "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d" (me) assigned 1 partition(s):
{ severity: 7, fac: 'ASSIGN' } [thrd:main]:   test [0]
{ severity: 7, fac: 'ASSIGNOR' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": "range" assignor run for 1 member(s)
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-metadata -> wait-sync (state up)
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent SyncGroupRequest (v3, 209 bytes @ 0, CorrId 6)
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received SyncGroupResponse (v3, 34 bytes, CorrId 6, rtt 3.42ms)
{ severity: 7, fac: 'SYNCGROUP' } [thrd:main]: SyncGroup response: Success (24 bytes of MemberState data)
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]:  test [0] offset INVALID
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-sync -> wait-assign-call (state up)
{ severity: 7, fac: 'ASSIGN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": new assignment of 1 partition(s) in join-state wait-assign-call
{ severity: 7, fac: 'CLEARASSIGN' } [thrd:main]: No current assignment to clear
{ severity: 7, fac: 'TOPIC' } [thrd:main]: New local topic: test
{ severity: 7, fac: 'TOPPARNEW' } [thrd:main]: NEW test [-1] 0x103807c60 refcnt 0x103807cf0 (at rd_kafka_topic_new0:465)
{ severity: 7, fac: 'STATE' } [thrd:main]: Topic test changed state unknown -> exists
{ severity: 7, fac: 'PARTCNT' } [thrd:main]: Topic test partition count changed from 0 to 1
{ severity: 7, fac: 'TOPPARNEW' } [thrd:main]: NEW test [0] 0x1038052d0 refcnt 0x103805360 (at rd_kafka_topic_partition_cnt_update:798)
{ severity: 7, fac: 'METADATA' } [thrd:main]:   Topic test partition 0 Leader 1
{ severity: 7, fac: 'BRKDELGT' } [thrd:main]: test [0]: delegate to broker localhost:9092/1 (rktp 0x1038052d0, term 0, ref 2)
{ severity: 7, fac: 'BRKDELGT' } [thrd:main]: test [0]: delegating to broker localhost:9092/1 for partition with 0 messages (0 bytes) queued
{ severity: 7, fac: 'BRKMIGR' } [thrd:main]: Migrating topic test [0] 0x1038052d0 from (none) to localhost:9092/1 (sending PARTITION_JOIN to localhost:9092/1)
{ severity: 7, fac: 'CONF' } [thrd:main]: Topic "test" configuration (default_topic_conf):
{ severity: 7, fac: 'CONF' } [thrd:main]:   auto.offset.reset = largest
{ severity: 7, fac: 'DESIRED' } [thrd:main]: test [0]: marking as DESIRED
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Added 1 partition(s) to assignment which now consists of 1 partition(s) where of 1 are in pending state and 0 are being queried
{ severity: 7, fac: 'PAUSE' } [thrd:main]: Resuming fetchers for 1 assigned partition(s): assign called
{ severity: 7, fac: 'RESUME' } [thrd:main]: Library resuming 1 partition(s)
{ severity: 7, fac: 'BARRIER' } [thrd:main]: test [0]: rd_kafka_toppar_op_pause_resume:2449: new version barrier v2
{ severity: 7, fac: 'RESUME' } [thrd:main]: Resume test [0] (v2)
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-assign-call -> steady (state up)
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]:  test [0] offset STORED
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]:  test [0] offset STORED
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'SRVPEND' } [thrd:main]: Querying committed offset for pending assigned partition test [0]
{ severity: 7, fac: 'OFFSETFETCH' } [thrd:main]: Fetching committed offsets for 1 pending partition(s) in assignment
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: joining broker (rktp 0x1038052d0, 0 message(s) queued)
{ severity: 7, fac: 'OFFSET' } [thrd:main]: GroupCoordinator/1: OffsetFetchRequest(v7) for 1/1 partition(s)
{ severity: 7, fac: 'OFFSET' } [thrd:main]: GroupCoordinator/1: Fetch committed offsets for 1/1 partition(s)
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Current assignment of 1 partition(s) with 1 pending adds, 1 offset queries, 0 partitions awaiting stop and 0 offset commits in progress
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": setting group assignment to 1 partition(s)
{ severity: 7, fac: 'GRPASSIGNMENT' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'GRPASSIGNMENT' } [thrd:main]:  test [0] offset STORED
{ severity: 7, fac: 'HEARTBEAT' } [thrd:main]: GroupCoordinator/1: Heartbeat for group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" generation id 1
{ severity: 7, fac: 'OP' } [thrd:main]: test [0] received op PAUSE (v2) in fetch-state none (opv1)
{ severity: 7, fac: 'RESUME' } [thrd:main]: Not resuming test [0]: partition is not paused by library
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent OffsetFetchRequest (v7, 93 bytes @ 0, CorrId 7)
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent HeartbeatRequest (v3, 131 bytes @ 0, CorrId 8)
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received OffsetFetchResponse (v7, 36 bytes, CorrId 7, rtt 2.02ms)
{ severity: 7, fac: 'OFFSETFETCH' } [thrd:main]: GroupCoordinator/1: OffsetFetchResponse: test [0] offset -1, metadata 0 byte(s): NO_ERROR
{ severity: 7, fac: 'OFFFETCH' } [thrd:main]: GroupCoordinator/1: OffsetFetch for 1/1 partition(s) (0 unstable partition(s)) returned Success
{ severity: 7, fac: 'OFFSETFETCH' } [thrd:main]: Adding test [0] back to pending list with offset INVALID
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]:  test [0] offset STORED
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]:  test [0] offset INVALID
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'SRVPEND' } [thrd:main]: Starting pending assigned partition test [0] at offset INVALID
{ severity: 7, fac: 'BARRIER' } [thrd:main]: test [0]: rd_kafka_toppar_op_pause_resume:2449: new version barrier v3
{ severity: 7, fac: 'RESUME' } [thrd:main]: Resume test [0] (v3)
{ severity: 7, fac: 'BARRIER' } [thrd:main]: test [0]: rd_kafka_toppar_op_fetch_start:2363: new version barrier v4
{ severity: 7, fac: 'CONSUMER' } [thrd:main]: Start consuming test [0] at offset INVALID (v4)
{ severity: 7, fac: 'ASSIGNDONE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": assignment operations done in join-state steady (rebalance rejoin=false)
{ severity: 7, fac: 'OP' } [thrd:main]: test [0] received op PAUSE (v3) in fetch-state none (opv2)
{ severity: 7, fac: 'RESUME' } [thrd:main]: Not resuming test [0]: partition is not paused by library
{ severity: 7, fac: 'OP' } [thrd:main]: test [0] received op FETCH_START (v4) in fetch-state none (opv3)
{ severity: 7, fac: 'FETCH' } [thrd:main]: Start fetch for test [0] in state none at offset INVALID (v4)
{ severity: 7, fac: 'PARTSTATE' } [thrd:main]: Partition test [0] changed fetch state none -> offset-query
{ severity: 7, fac: 'OFFSET' } [thrd:main]: test [0]: offset reset (at offset INVALID) to END: no previously committed offset available: Local: No offset stored
{ severity: 7, fac: 'OFFSET' } [thrd:main]: test [0]: backoff: (re)starting offset query timer for offset END
{ severity: 7, fac: 'CGRPOP' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" received op PARTITION_JOIN in state up (join-state steady) for test [0]
{ severity: 7, fac: 'PARTADD' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": add test [0]
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received HeartbeatResponse (v3, 6 bytes, CorrId 8, rtt 2.56ms)
{ severity: 7, fac: 'OFFSET' } [thrd:main]: Topic test [0]: timed offset query for END in state offset-query
{ severity: 7, fac: 'OFFREQ' } [thrd:main]: localhost:9092/1: Partition test [0]: querying for logical offset END (opv 4)
{ severity: 7, fac: 'PARTSTATE' } [thrd:main]: Partition test [0] changed fetch state offset-query -> offset-wait
{ severity: 7, fac: 'OFFSET' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: ListOffsetsRequest (v2, opv 4) for 1 topic(s) and 1 partition(s)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent ListOffsetsRequest (v2, 52 bytes @ 0, CorrId 6)
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received ListOffsetsResponse (v2, 40 bytes, CorrId 6, rtt 2.03ms)
{ severity: 7, fac: 'OFFSET' } [thrd:main]: localhost:9092/1: Offset reply for topic test [0] (v4 vs v4)
{ severity: 7, fac: 'OFFSET' } [thrd:main]: Offset END request for test [0] returned offset 10138 (10138)
{ severity: 7, fac: 'PARTSTATE' } [thrd:main]: Partition test [0] changed fetch state offset-wait -> active
{ severity: 7, fac: 'FETCH' } [thrd:main]: Partition test [0] start fetching at offset 10138
{ severity: 7, fac: 'WAKEUP' } [thrd:main]: localhost:9092/1: Wake-up
{ severity: 7, fac: 'FETCHDEC' } [thrd:localhost:9092/bootstrap]: Topic test [0]: fetch decide: updating to version 4 (was 0) at offset 10138 (was 0)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0] in state active at offset 10138 (0/100000 msgs, 0/65536 kb queued, opv 4) is fetchable
{ severity: 7, fac: 'FETCHADD' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Added test [0] to fetch list (1 entries, opv 4, 0 messages queued): fetchable
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch topic test [0] at offset 10138 (v4)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch 1/1/1 toppar(s)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FetchRequest (v11, 94 bytes @ 0, CorrId 7)
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received FetchResponse (v11, 66 bytes, CorrId 7, rtt 1002.05ms)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0] MessageSet size 0, error "Success", MaxOffset 10138, LSO 10138, Ver 4/4
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch topic test [0] at offset 10138 (v4)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch 1/1/1 toppar(s)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FetchRequest (v11, 94 bytes @ 0, CorrId 8)
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
PRODUCE - should be able to produce and consume messages: consumeLoop
{ severity: 7, fac: 'TOPIC' } [thrd:app]: New local topic: test
{ severity: 7, fac: 'TOPPARNEW' } [thrd:app]: NEW test [-1] 0x102f1f5d0 refcnt 0x102f1f660 (at rd_kafka_topic_new0:465)
{ severity: 7, fac: 'STATE' } [thrd:app]: Topic test changed state unknown -> exists
{ severity: 7, fac: 'PARTCNT' } [thrd:app]: Topic test partition count changed from 0 to 1
{ severity: 7, fac: 'TOPPARNEW' } [thrd:app]: NEW test [0] 0x102f1fb40 refcnt 0x102f1fbd0 (at rd_kafka_topic_partition_cnt_update:798)
{ severity: 7, fac: 'METADATA' } [thrd:app]:   Topic test partition 0 Leader 1
{ severity: 7, fac: 'BRKDELGT' } [thrd:app]: test [0]: delegate to broker localhost:9092/1 (rktp 0x102f1fb40, term 0, ref 2)
{ severity: 7, fac: 'BRKDELGT' } [thrd:app]: test [0]: delegating to broker localhost:9092/1 for partition with 0 messages (0 bytes) queued
{ severity: 7, fac: 'BRKMIGR' } [thrd:app]: Migrating topic test [0] 0x102f1fb40 from (none) to localhost:9092/1 (sending PARTITION_JOIN to localhost:9092/1)
{ severity: 7, fac: 'PARTCNT' } [thrd:app]: Partitioning 0 unassigned messages in topic test to 1 partitions
{ severity: 7, fac: 'UAS' } [thrd:app]: 0/0 messages were partitioned in topic test
{ severity: 7, fac: 'CONF' } [thrd:app]: Topic "test" configuration (default_topic_conf):
{ severity: 7, fac: 'CONF' } [thrd:app]:   produce.offset.report = true
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: joining broker (rktp 0x102f1fb40, 0 message(s) queued)
{ severity: 7, fac: 'FETCHADD' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Added test [0] to active list (1 entries, opv 0, 0 messages queued): joining
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'TOPPAR' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: test [0] 1 message(s) in xmit queue (1 added from partition queue)
{ severity: 7, fac: 'TOPPAR' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: test [0] 1 message(s) in xmit queue (0 added from partition queue)
{ severity: 7, fac: 'TOPPAR' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: test [0] 1 message(s) in xmit queue (0 added from partition queue)
{ severity: 7, fac: 'PRODUCE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: test [0]: Produce MessageSet with 1 message(s) (4169 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid}, uncompressed)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent ProduceRequest (v7, 4224 bytes @ 0, CorrId 4)
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received ProduceResponse (v7, 48 bytes, CorrId 4, rtt 3.04ms)
{ severity: 7, fac: 'MSGSET' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: test [0]: MessageSet with 1 message(s) (MsgId 0, BaseSeq -1) delivered
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received FetchResponse (v11, 4235 bytes, CorrId 8, rtt 890.97ms)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0] MessageSet size 4169, error "Success", MaxOffset 10139, LSO 10139, Ver 4/4
{ severity: 7, fac: 'CONSUME' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Enqueue 1 message(s) (4099 bytes, 1 ops) on test [0] fetch queue (qlen 0, v4, last_offset 10138, 0 ctrl msgs, uncompressed)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch topic test [0] at offset 10139 (v4)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch 1/1/1 toppar(s)
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FetchRequest (v11, 94 bytes @ 0, CorrId 9)
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
on data
    ✓ should be able to produce and consume messages: consumeLoop (2029ms)
after each started
start disconnect
{ severity: 7, fac: 'WAKEUP' } [thrd:app]: localhost:9092/1: Wake-up
{ severity: 7, fac: 'DESTROY' } [thrd:app]: Terminating instance (destroy flags none (0x0))
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Interrupting timers
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Sending TERMINATE to internal main thread
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Joining internal main thread
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Internal main thread terminating
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Destroy internal
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Removing all topics
{ severity: 7, fac: 'PARTCNT' } [thrd:main]: Topic test partition count changed from 1 to 0
{ severity: 7, fac: 'REMOVE' } [thrd:main]: test [0] no longer reported in metadata
{ severity: 7, fac: 'BRKMIGR' } [thrd:main]: test [0] 0x102f1fb40 sending final LEAVE for removal by localhost:9092/1
{ severity: 7, fac: 'TOPPARREMOVE' } [thrd:main]: Removing toppar test [-1] 0x102f1f5d0
{ severity: 7, fac: 'DESTROY' } [thrd:main]: test [-1]: 0x102f1f5d0 DESTROY_FINAL
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Sending TERMINATE to localhost:9092/1
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x102f1fb40)
{ severity: 7, fac: 'FETCHADD' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Removed test [0] from active list (0 entries, opv 0): leaving
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: no next broker, failing 0 message(s) in partition queue
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'TOPPARREMOVE' } [thrd:localhost:9092/bootstrap]: Removing toppar test [0] 0x102f1fb40
{ severity: 7, fac: 'DESTROY' } [thrd:localhost:9092/bootstrap]: test [0]: 0x102f1fb40 DESTROY_FINAL
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Purging reply queue
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Decommissioning internal broker
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Join 2 broker thread(s)
{ severity: 7, fac: 'TERM' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received TERMINATE op in state UP: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
{ severity: 7, fac: 'TERM' } [thrd::0/internal]: :0/internal: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
{ severity: 7, fac: 'FAIL' } [thrd::0/internal]: :0/internal: Client is terminating (after 2043ms in state INIT) (_DESTROY)
{ severity: 7, fac: 'STATE' } [thrd::0/internal]: :0/internal: Broker changed state INIT -> DOWN
{ severity: 7, fac: 'BROADCAST' } [thrd::0/internal]: Broadcasting state change
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BRKTERM' } [thrd::0/internal]: :0/internal: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'TERMINATE' } [thrd::0/internal]: :0/internal: Handle is terminating in state DOWN: 1 refcnts (0x106016ba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'FAIL' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Client is terminating (after 2035ms in state UP) (_DESTROY)
{ severity: 7, fac: 'FAIL' } [thrd::0/internal]: :0/internal: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UP -> DOWN
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BRKTERM' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state DOWN: 1 refcnts (0x1060179a0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'FAIL' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Updating 0 buffers on connection reset
why err null
{ severity: 7, fac: 'RECV' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received FetchResponse (v11, 66 bytes, CorrId 9, rtt 1001.84ms)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0] MessageSet size 0, error "Success", MaxOffset 10139, LSO 10139, Ver 4/4
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch topic test [0] at offset 10139 (v4)
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch 1/1/1 toppar(s)
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Closing consumer
{ severity: 7, fac: 'SEND' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FetchRequest (v11, 94 bytes @ 0, CorrId 10)
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Waiting for close events
{ severity: 7, fac: 'CGRPOP' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" received op TERMINATE in state up (join-state steady)
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Terminating group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" in state up with 1 partition(s)
{ severity: 7, fac: 'UNSUBSCRIBE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": unsubscribe from current subscription of size 1 (leave group=true, has joined=true, rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d, join-state steady)
{ severity: 7, fac: 'SUBSCRIPTION' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": clearing subscribed topics list (1)
{ severity: 7, fac: 'SUBSCRIPTION' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": effective subscription list changed from 1 to 0 topic(s):
{ severity: 7, fac: 'GRPLEADER' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": resetting group leader info: unsubscribe
{ severity: 7, fac: 'REBALANCE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" initiating rebalance (EAGER) in state up (join-state steady) with 1 assigned partition(s): unsubscribe
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state steady -> wait-unassign-call (state up)
{ severity: 7, fac: 'CLEARASSIGN' } [thrd:main]: Clearing current assignment of 1 partition(s)
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-unassign-call -> wait-unassign-to-complete (state up)
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 1 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]:  test [0] offset STORED
{ severity: 7, fac: 'BARRIER' } [thrd:main]: test [0]: rd_kafka_toppar_op_fetch_stop:2390: new version barrier v5
{ severity: 7, fac: 'CONSUMER' } [thrd:main]: Stop consuming test [0] (v5)
{ severity: 7, fac: 'BARRIER' } [thrd:main]: test [0]: rd_kafka_toppar_op_pause_resume:2449: new version barrier v6
{ severity: 7, fac: 'RESUME' } [thrd:main]: Resume test [0] (v6)
{ severity: 7, fac: 'DESP' } [thrd:main]: Removing (un)desired topic test [0]
{ severity: 7, fac: 'REMOVE' } [thrd:main]: Removing test [0] from assignment (started=true, pending=false, queried=false, stored offset=10139)
{ severity: 7, fac: 'REMOVE' } [thrd:main]: Served 1 removed partition(s), with 1 offset(s) to commit
{ severity: 7, fac: 'COMMIT' } [thrd:main]: GroupCoordinator/1: Committing offsets for 1 partition(s) with generation-id 1 in join-state wait-unassign-to-complete: unassigned partitions
{ severity: 7, fac: 'OFFSET' } [thrd:main]: GroupCoordinator/1: Enqueue OffsetCommitRequest(v7, 1/1 partition(s))): unassigned partitions
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Current assignment of 0 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 1 offset commits in progress
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": clearing group assignment
{ severity: 7, fac: 'CLEARASSIGN' } [thrd:main]: No current assignment to clear
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=1, wait_stop_cnt=1)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Current assignment of 0 partition(s) with 0 pending adds, 0 offset queries, 1 partitions awaiting stop and 1 offset commits in progress
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 1 toppar(s), assignment in progress, 1 commit(s) (state up, join-state wait-unassign-to-complete) before terminating
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 1 toppar(s), assignment in progress, 1 commit(s) (state up, join-state wait-unassign-to-complete) before terminating
{ severity: 7, fac: 'OP' } [thrd:main]: test [0] received op FETCH_STOP (v5) in fetch-state active (opv4)
{ severity: 7, fac: 'FETCH' } [thrd:main]: Stopping fetch for test [0] in state active (v5)
{ severity: 7, fac: 'PARTSTATE' } [thrd:main]: Partition test [0] changed fetch state active -> stopping
{ severity: 7, fac: 'STORETERM' } [thrd:main]: test [0]: offset store terminating
{ severity: 7, fac: 'PARTSTATE' } [thrd:main]: Partition test [0] changed fetch state stopping -> stopped
{ severity: 7, fac: 'OP' } [thrd:main]: test [0] received op PAUSE (v6) in fetch-state stopped (opv5)
{ severity: 7, fac: 'RESUME' } [thrd:main]: Not resuming test [0]: partition is not paused by library
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 1 toppar(s), assignment in progress, 1 commit(s) (state up, join-state wait-unassign-to-complete) before terminating
{ severity: 7, fac: 'CGRPOP' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" received op PARTITION_LEAVE in state up (join-state wait-unassign-to-complete) for test [0]
{ severity: 7, fac: 'PARTDEL' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": delete test [0]
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 0 toppar(s), assignment in progress, 1 commit(s) (state up, join-state wait-unassign-to-complete) before terminating
{ severity: 7, fac: 'STOPSERVE' } [thrd:main]: All partitions awaiting stop are now stopped: serving assignment
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Current assignment of 0 partition(s) with 0 pending adds, 0 offset queries, 0 partitions awaiting stop and 1 offset commits in progress
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 0 toppar(s), assignment in progress, 1 commit(s) (state up, join-state wait-unassign-to-complete) before terminating
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent OffsetCommitRequest (v7, 163 bytes @ 0, CorrId 9)
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received OffsetCommitResponse (v7, 24 bytes, CorrId 9, rtt 3.21ms)
{ severity: 7, fac: 'COMMIT' } [thrd:main]: GroupCoordinator/1: OffsetCommit for 1 partition(s) in join-state wait-unassign-to-complete: unassigned partitions: returned: Success
{ severity: 7, fac: 'DUMP' } [thrd:main]: Assignment dump (started_cnt=0, wait_stop_cnt=0)
{ severity: 7, fac: 'DUMP_ALL' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_PND' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_QRY' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'DUMP_REM' } [thrd:main]: List with 0 partition(s):
{ severity: 7, fac: 'ASSIGNDONE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": assignment operations done in join-state wait-unassign-to-complete (rebalance rejoin=false)
{ severity: 7, fac: 'UNASSIGN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": unassign done in state up (join-state wait-unassign-to-complete)
{ severity: 7, fac: 'MEMBERID' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": updating member id "rdkafka-af615a5e-0705-4a80-9166-4b3f9a0e351d" -> ""
{ severity: 7, fac: 'LEAVE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": leave (in state up)
{ severity: 7, fac: 'LEAVE' } [thrd:main]: localhost:9092/1: Leaving group
{ severity: 7, fac: 'NOREJOIN' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": Not rejoining group without an assignment: Unassignment done: no subscribed topics
{ severity: 7, fac: 'CGRPJOINSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed join state wait-unassign-to-complete -> init (state up)
{ severity: 7, fac: 'CGRPTERM' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": waiting for 0 toppar(s), 0 commit(s), wait-leave, (state up, join-state init) before terminating
{ severity: 7, fac: 'SEND' } [thrd:GroupCoordinator]: GroupCoordinator/1: Sent LeaveGroupRequest (v1, 125 bytes @ 0, CorrId 10)
{ severity: 7, fac: 'RECV' } [thrd:GroupCoordinator]: GroupCoordinator/1: Received LeaveGroupResponse (v1, 6 bytes, CorrId 10, rtt 4.18ms)
{ severity: 7, fac: 'LEAVEGROUP' } [thrd:main]: LeaveGroup response received in state up
{ severity: 7, fac: 'CGRPSTATE' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" changed state up -> term (join-state init)
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'ASSIGNMENT' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab": clearing group assignment
{ severity: 7, fac: 'COORDCLEAR' } [thrd:main]: Group "kafka-mocha-grp-f8688ddf734d7425174e36ee555df8fec180fbab" broker localhost:9092/1 is no longer coordinator
{ severity: 7, fac: 'NODENAME' } [thrd:main]: GroupCoordinator/1: Broker nodename changed from "localhost:9092" to ""
{ severity: 7, fac: 'NODEID' } [thrd:main]: GroupCoordinator/1: Broker nodeid changed from 1 to -1
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Consumer closed
{ severity: 7, fac: 'DESTROY' } [thrd:app]: Terminating instance (destroy flags NoConsumerClose (0x8))
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Terminating consumer group handler
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Closing consumer
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Disabling and purging temporary queue to quench close events
{ severity: 7, fac: 'CLOSE' } [thrd:app]: Consumer closed
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Interrupting timers
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Sending TERMINATE to internal main thread
{ severity: 7, fac: 'TERMINATE' } [thrd:app]: Joining internal main thread
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Internal main thread terminating
{ severity: 7, fac: 'FAIL' } [thrd:GroupCoordinator]: GroupCoordinator: Closing connection due to nodename change (after 3037ms in state UP) (_TRANSPORT)
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Destroy internal
{ severity: 7, fac: 'BROADCAST' } [thrd:main]: Broadcasting state change
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state UP -> DOWN
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Removing all topics
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'PARTCNT' } [thrd:main]: Topic test partition count changed from 1 to 0
{ severity: 7, fac: 'REMOVE' } [thrd:main]: test [0] no longer reported in metadata
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'BRKMIGR' } [thrd:main]: test [0] 0x1038052d0 sending final LEAVE for removal by localhost:9092/1
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BRKTERM' } [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'METADATA' } [thrd:GroupCoordinator]: Requesting metadata for 1/1 topics: broker down
{ severity: 7, fac: 'METADATA' } [thrd:GroupCoordinator]: localhost:9092/1: Request metadata for 1 topic(s): broker down
{ severity: 7, fac: 'TOPPARREMOVE' } [thrd:main]: Removing toppar test [-1] 0x103807c60
{ severity: 7, fac: 'DESTROY' } [thrd:main]: test [-1]: 0x103807c60 DESTROY_FINAL
{ severity: 7, fac: 'TERMINATE' } [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 3 refcnts (0x10702afa0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state DOWN -> INIT
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Sending TERMINATE to localhost:9092/1
{ severity: 7, fac: 'DESTROY' } [thrd:main]: Sending TERMINATE to GroupCoordinator
{ severity: 7, fac: 'FETCH' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0] in state stopped at offset 10138 (0/100000 msgs, 0/65536 kb queued, opv 4) is not fetchable: forced removal
{ severity: 7, fac: 'FETCHADD' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Removed test [0] from fetch list (0 entries, opv 4): forced removal
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: leaving broker (0 messages in xmitq, next broker (none), rktp 0x1038052d0)
{ severity: 7, fac: 'TOPBRK' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic test [0]: no next broker, failing 0 message(s) in partition queue
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'TERM' } [thrd:GroupCoordinator]: GroupCoordinator: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
{ severity: 7, fac: 'TERM' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Received TERMINATE op in state UP: 5 refcnts, 0 toppar(s), 0 active toppar(s), 1 outbufs, 1 waitresps, 0 retrybufs
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Purging reply queue
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Decommissioning internal broker
{ severity: 7, fac: 'FAIL' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Client is terminating (after 3043ms in state UP) (_DESTROY)
{ severity: 7, fac: 'TERMINATE' } [thrd:main]: Join 3 broker thread(s)
{ severity: 7, fac: 'TERM' } [thrd::0/internal]: :0/internal: Received TERMINATE op in state INIT: 2 refcnts, 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs
{ severity: 7, fac: 'FAIL' } [thrd:GroupCoordinator]: GroupCoordinator: Client is terminating (after 0ms in state INIT) (_DESTROY)
{ severity: 7, fac: 'STATE' } [thrd:GroupCoordinator]: GroupCoordinator: Broker changed state INIT -> DOWN
{ severity: 7, fac: 'BROADCAST' } [thrd:GroupCoordinator]: Broadcasting state change
{ severity: 7, fac: 'FAIL' } [thrd::0/internal]: :0/internal: Client is terminating (after 3052ms in state INIT) (_DESTROY)
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'STATE' } [thrd::0/internal]: :0/internal: Broker changed state INIT -> DOWN
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BRKTERM' } [thrd:GroupCoordinator]: GroupCoordinator: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'TERMINATE' } [thrd:GroupCoordinator]: GroupCoordinator: Handle is terminating in state DOWN: 1 refcnts (0x10702afa0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'FAIL' } [thrd:GroupCoordinator]: GroupCoordinator: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd:GroupCoordinator]: GroupCoordinator: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BROADCAST' } [thrd::0/internal]: Broadcasting state change
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state UP -> DOWN
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 1 buffers
{ severity: 7, fac: 'BRKTERM' } [thrd::0/internal]: :0/internal: terminating: broker still has 2 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Purging bufq with 1 buffers
{ severity: 7, fac: 'TERMINATE' } [thrd::0/internal]: :0/internal: Handle is terminating in state DOWN: 1 refcnts (0x10702bda0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'BUFQ' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Updating 0 buffers on connection reset
{ severity: 7, fac: 'FAIL' } [thrd::0/internal]: :0/internal: Broker handle is terminating (after 0ms in state DOWN) (_DESTROY)
{ severity: 7, fac: 'BRKTERM' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: terminating: broker still has 3 refcnt(s), 0 buffer(s), 0 partition(s)
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Purging bufq with 0 buffers
{ severity: 7, fac: 'BUFQ' } [thrd::0/internal]: :0/internal: Updating 0 buffers on connection reset
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state DOWN: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'STATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Broker changed state DOWN -> INIT
{ severity: 7, fac: 'BROADCAST' } [thrd:localhost:9092/bootstrap]: Broadcasting state change
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf
{ severity: 7, fac: 'TERMINATE' } [thrd:localhost:9092/bootstrap]: localhost:9092/1: Handle is terminating in state INIT: 2 refcnts (0x10702cba0), 0 toppar(s), 0 active toppar(s), 0 outbufs, 0 waitresps, 0 retrybufs: failed 0 request(s) in retry+outbuf

robinfehr avatar Jul 08 '22 00:07 robinfehr

@robinfehr Have you figured anything out? I am only able to reproduce it in k8s. The producer gets stuck in connecting state and cannot get out with a restart. I would like to look at the issue with you to see if I can help.

o2themar avatar Jul 14 '22 20:07 o2themar

hey @o2themar sure - here you can find the full report: https://github.com/edenhill/librdkafka/issues/3904

if you want, we can look into it on my machine and debug it. i sadly can only reproduce it on a mac machine, linux works just fine.

robinfehr avatar Jul 14 '22 20:07 robinfehr

I have found that RdKafka::Message destructor is probably not thread safe. When I was destroying messages in different thread than the one in which they were created, rdkafka would hang on destruction.

pavelgeiger avatar Jul 15 '22 05:07 pavelgeiger

In my example you can see that RdKafka::Message is consumed and destroyed on same thread. So that should be different problem.

pranasge avatar Jul 15 '22 11:07 pranasge

So here is the line that I determined where it was failing. The other library that I'm using that uses librdkafka printed out in the logs that it failed to resolve the connecting to the brokers. https://github.com/edenhill/librdkafka/blob/a913c75e1e09ef87232e9c7b044d8ab3ec404417/src/rdkafka_broker.c#L991

Here is the errors that are logged for this particular issue. The error it throws in noderdkafka is related to librdkafka. Its getting back a broker transport failure. The brokers are fine and its able to connect to them if i restart the app manually.

I replaced the below broker names with aliases JFYI. {"severity":3,"fac":"FAIL","message":"[thrd:broker3:9092/bootstrap]: broker3:9092/bootstrap: Failed to resolve 'broker3:9092': Try again (after 5004ms in state CONNECT)"} {"severity":3,"fac":"FAIL","message":"[thrd:broker2:9092/bootstrap]: broker2:9092/bootstrap: Failed to resolve 'broker2:9092': Try again (after 5004ms in state CONNECT)"} {"severity":3,"fac":"FAIL","message":"[thrd:broker1:9092/bootstrap]: broker1:9092/bootstrap: Failed to resolve 'broker1:9092': Try again (after 5004ms in state CONNECT)"} {"severity":3,"fac":"FAIL","message":"[thrd:broker3:9092/bootstrap]: broker3:9092/bootstrap: Failed to resolve 'broker3:9092': Try again (after 5004ms in state CONNECT, 1 identical error(s) suppressed)"} {"severity":3,"fac":"FAIL","message":"[thrd:broker2:9092/bootstrap]: broker2:9092/bootstrap: Failed to resolve 'broker2:9092': Try again (after 5004ms in state CONNECT, 1 identical error(s) suppressed)"} {"severity":3,"fac":"FAIL","message":"[thrd:broker1:9092/bootstrap]: broker1:9092/bootstrap: Failed to resolve 'broker1:9092': Try again (after 5004ms in state CONNECT, 1 identical error(s) suppressed)"} {"origin":"local","message":"broker transport failure","code":-195,"errno":-195,"stack":"Error: Local: Broker transport failure\n at Function.createLibrdkafkaError [as create] (/katsrc/node_modules/node-rdkafka/lib/error.js:454:10)\n at /katsrc/node_modules/node-rdkafka/lib/client.js:350:28"}

o2themar avatar Jul 15 '22 17:07 o2themar

I should clarify the above happens when the node that the application is running on in k8s goes down and the pod changes its status from Active to Running. There is probably network disconnect since the node went down but the pod doesn't change to another node because by default its set to only transition after 5 minutes. The application stays in this state until the node comes up and then gets restarted. Once it starts up the above error is what shows.

o2themar avatar Jul 18 '22 11:07 o2themar

For more clarfication on what is happening read this about what happens when a node is restarted. It could be the reassigning of ips that causes the brokers to get in a weird state when they start back up. There is also an update of the ip tables which might have an effect on being able to find the brokers.

o2themar avatar Jul 18 '22 11:07 o2themar

One of new releases fixed this issue for me.

pranasge avatar Mar 07 '23 09:03 pranasge

@pranasge Can you tell me which version of this problem was resolved ?

XiaoYiXiaoYang avatar Mar 22 '23 10:03 XiaoYiXiaoYang

I upgraded from 1.9.0 to 1.9.2 first and after that i did not see this problem reproduced. But I used 1.9.2 only for month and now with 2.0.2 i haven't seen this problem since first update.

pranasge avatar Mar 24 '23 09:03 pranasge