redpanda-data / redpanda

Redpanda is a streaming data platform for developers. Kafka API compatible. 10x faster. No ZooKeeper. No JVM!
https://redpanda.com
9.74k stars 592 forks source link

Oversize allocation in segment_appender::flush - Failed to allocate 5242880 bytes #12547

Closed StephanDollberg closed 1 year ago

StephanDollberg commented 1 year ago

Version & Environment

Redpanda version: 23.2.3

segment_appender::flush_ops_ is a std::vector. OOMed under load. Must have been hundredthousands of flush_ops as that struct is fairly small.

seastar::memory::on_allocation_failure(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:2054
seastar::memory::finish_allocation(void*, unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1579
 (inlined by) seastar::memory::allocate_slowpath(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1618
seastar::memory::allocate(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1629
 (inlined by) operator new(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:2318
void* std::__1::__libcpp_operator_new[abi:v160004]<unsigned long>(unsigned long) at /vectorized/llvm/bin/../include/c++/v1/new:266
 (inlined by) std::__1::__libcpp_allocate[abi:v160004](unsigned long, unsigned long) at /vectorized/llvm/bin/../include/c++/v1/new:292
 (inlined by) std::__1::allocator<storage::segment_appender::flush_op>::allocate[abi:v160004](unsigned long) at /vectorized/llvm/bin/../include/c++/v1/__memory/allocator.h:115
 (inlined by) std::__1::__allocation_result<std::__1::allocator_traits<std::__1::allocator<storage::segment_appender::flush_op> >::pointer> std::__1::__allocate_at_least[abi:v160004]<std::__1::allocator<storage::segment_appender::flush_op> >(std::__1::allocator<storage::segment_appender::flush_op>&, unsigned long) at /vectorized/llvm/bin/../include/c++/v1/__memory/allocate_at_least.h:55
 (inlined by) __split_buffer at /vectorized/llvm/bin/../include/c++/v1/__split_buffer:323
 (inlined by) void std::__1::vector<storage::segment_appender::flush_op, std::__1::allocator<storage::segment_appender::flush_op> >::__emplace_back_slow_path<unsigned long>(unsigned long&&) at /vectorized/llvm/bin/../include/c++/v1/vector:1580
 (inlined by) storage::segment_appender::flush_op& std::__1::vector<storage::segment_appender::flush_op, std::__1::allocator<storage::segment_appender::flush_op> >::emplace_back<unsigned long>(unsigned long&&) at /vectorized/llvm/bin/../include/c++/v1/vector:1603
 (inlined by) storage::segment_appender::flush() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/segment_appender.cc:537
storage::segment::do_flush() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/segment.cc:337
 (inlined by) operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/segment.cc:327
seastar::future<void> seastar::futurize<seastar::future<void> >::invoke<storage::segment::flush()::$_0, seastar::semaphore_units<seastar::semaphore_default_exception_factory, std::__1::chrono::steady_clock> >(storage::segment::flush()::$_0&&, seastar::semaphore_units<seastar::semaphore_default_exception_factory, std::__1::chrono::steady_clock>&&) at /vectorized/include/seastar/core/future.hh:1999
 (inlined by) seastar::future<void> seastar::future<seastar::semaphore_units<seastar::semaphore_default_exception_factory, std::__1::chrono::steady_clock> >::then_impl<storage::segment::flush()::$_0, seastar::future<void> >(storage::segment::flush()::$_0&&) at /vectorized/include/seastar/core/future.hh:1470
 (inlined by) seastar::future<void> seastar::future<seastar::semaphore_units<seastar::semaphore_default_exception_factory, std::__1::chrono::steady_clock> >::then<storage::segment::flush()::$_0, seastar::future<void> >(storage::segment::flush()::$_0&&) at /vectorized/include/seastar/core/future.hh:1396
 (inlined by) storage::segment::flush() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/segment.cc:326
storage::disk_log_impl::flush() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/disk_log_impl.cc:1148
storage::log::flush() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/storage/log.h:116
 (inlined by) raft::consensus::flush_log() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/consensus.cc:2480
raft::replicate_entries_stm::flush_log() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:57
raft::replicate_entries_stm::dispatch_single_retry(raft::vnode) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:181
 (inlined by) operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:154
 (inlined by) seastar::future<void> seastar::futurize<seastar::future<void> >::invoke<raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0>(raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0&&) at /vectorized/include/seastar/core/future.hh:1999
 (inlined by) auto seastar::futurize_invoke<raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0>(raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0&&) at /vectorized/include/seastar/core/future.hh:2030
 (inlined by) auto seastar::internal::invoke_func_with_gate<raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0>(seastar::gate&, raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0&&) at /vectorized/include/seastar/core/gate.hh:273
 (inlined by) auto seastar::with_gate<raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0>(seastar::gate&, raft::replicate_entries_stm::dispatch_one(raft::vnode)::$_0&&) at /vectorized/include/seastar/core/gate.hh:291
 (inlined by) raft::replicate_entries_stm::dispatch_one(raft::vnode) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:151
operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:300
 (inlined by) raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >)::$_2 std::__1::for_each[abi:v160004]<boost::range_detail::join_iterator<std::__1::__wrap_iter<raft::vnode*>, std::__1::__wrap_iter<raft::vnode*>, raft::vnode, raft::vnode&, boost::iterators::random_access_traversal_tag>, raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >)::$_2>(boost::range_detail::join_iterator<std::__1::__wrap_iter<raft::vnode*>, std::__1::__wrap_iter<raft::vnode*>, raft::vnode, raft::vnode&, boost::iterators::random_access_traversal_tag>, boost::range_detail::join_iterator<std::__1::__wrap_iter<raft::vnode*>, std::__1::__wrap_iter<raft::vnode*>, raft::vnode, raft::vnode&, boost::iterators::random_access_traversal_tag>, raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >)::$_2) at /vectorized/llvm/bin/../include/c++/v1/__algorithm/for_each.h:26
 (inlined by) void raft::group_configuration::for_each_broker_id<raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >)::$_2>(raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >)::$_2&&) const at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/group_configuration.h:458
 (inlined by) raft::replicate_entries_stm::apply(std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_entries_stm.cc:285
raft::replicate_batcher::do_flush(std::__1::vector<seastar::lw_shared_ptr<raft::replicate_batcher::item>, std::__1::allocator<seastar::lw_shared_ptr<raft::replicate_batcher::item> > >, raft::append_entries_request, std::__1::vector<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, std::__1::allocator<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > >, absl::lts_20220623::flat_hash_map<raft::vnode, detail::base_named_type<unsigned long, raft::follower_req_seq_tag, std::__1::integral_constant<bool, true> >, absl::lts_20220623::hash_internal::Hash<raft::vnode>, std::__1::equal_to<raft::vnode>, std::__1::allocator<std::__1::pair<raft::vnode const, detail::base_named_type<unsigned long, raft::follower_req_seq_tag, std::__1::integral_constant<bool, true> > > > >) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:326
raft::replicate_batcher::flush(seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>, bool) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:270
operator()<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> > at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:80
 (inlined by) seastar::future<void> seastar::futurize<seastar::future<void> >::invoke<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}, seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> >(raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}&&, seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock>&&) at /vectorized/include/seastar/core/future.hh:1999
 (inlined by) seastar::future<void> seastar::future<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> >::then_impl<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}, seastar::future<void> >(raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}&&) at /vectorized/include/seastar/core/future.hh:1470
 (inlined by) seastar::future<void> seastar::future<seastar::semaphore_units<seastar::named_semaphore_exception_factory, std::__1::chrono::steady_clock> >::then<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}, seastar::future<void> >(raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0::operator()() const::{lambda(auto:1)#1}&&) at /vectorized/include/seastar/core/future.hh:1396
 (inlined by) operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:79
 (inlined by) seastar::future<void> seastar::futurize<seastar::future<void> >::invoke<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0>(raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0&&) at /vectorized/include/seastar/core/future.hh:1999
 (inlined by) auto seastar::futurize_invoke<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0>(raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0&&) at /vectorized/include/seastar/core/future.hh:2030
 (inlined by) auto seastar::internal::invoke_func_with_gate<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0>(seastar::gate&, raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0&&) at /vectorized/include/seastar/core/gate.hh:273
 (inlined by) auto seastar::try_with_gate<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0>(seastar::gate&, raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0&&) at /vectorized/include/seastar/core/gate.hh:313
 (inlined by) auto ssx::spawn_with_gate_then<raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0>(seastar::gate&, raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >)::$_0&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/ssx/future-util.h:294
 (inlined by) raft::replicate_batcher::cache_and_wait_for_result(seastar::promise<void>, std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:77
raft::replicate_batcher::replicate(std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader, raft::consistency_level, std::__1::optional<std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000l> > >) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/replicate_batcher.cc:40
raft::consensus::do_replicate(std::__1::optional<detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> > >, model::record_batch_reader&&, raft::replicate_options) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/consensus.cc:795
 (inlined by) raft::consensus::replicate_in_stages(detail::base_named_type<long, model::model_raft_term_id_type, std::__1::integral_constant<bool, true> >, model::record_batch_reader&&, raft::replicate_options) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/raft/consensus.cc:753
kafka::group::store_offsets(kafka::offset_commit_request&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/kafka/server/group.cc:2443
kafka::group::handle_offset_commit(kafka::offset_commit_request&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/kafka/server/group.cc:2657
kafka::group_manager::offset_commit(kafka::offset_commit_request&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/kafka/server/group_manager.cc:1287
decltype (((std::declval<kafka::group_manager&>()).*(std::declval<kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&)>()))(std::declval<kafka::offset_commit_request>())) std::__1::__invoke[abi:v160004]<kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&), kafka::group_manager&, kafka::offset_commit_request, void>(kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&), kafka::group_manager&, kafka::offset_commit_request&&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:343
 (inlined by) std::__1::invoke_result<kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&), kafka::group_manager&, kafka::offset_commit_request>::type std::__1::invoke[abi:v160004]<kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&), kafka::group_manager&, kafka::offset_commit_request>(kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*&)(kafka::offset_commit_request&&), kafka::group_manager&, kafka::offset_commit_request&&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:539
 (inlined by) operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0f5ff99c2b18b4005-1/redpanda/redpanda/src/v/kafka/server/group_router.cc:95
decltype ((std::declval<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}>())(std::declval<kafka::group_manager&>())) std::__1::__invoke[abi:v160004]<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}, kafka::group_manager&>(kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&, kafka::group_manager&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:394
 (inlined by) decltype(auto) std::__1::__apply_tuple_impl[abi:v160004]<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}, std::__1::tuple<kafka::group_manager&>, 0ul>(kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&, std::__1::tuple<kafka::group_manager&>&&, std::__1::__tuple_indices<0ul>) at /vectorized/llvm/bin/../include/c++/v1/tuple:1816
 (inlined by) decltype(auto) std::__1::apply[abi:v160004]<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}, std::__1::tuple<kafka::group_manager&> >(kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&, std::__1::tuple<kafka::group_manager&>&&) at /vectorized/llvm/bin/../include/c++/v1/tuple:1825
 (inlined by) operator() at /vectorized/include/seastar/core/sharded.hh:458
 (inlined by) seastar::future<std::__1::tuple<std::exception_ptr, std::__1::unique_ptr<seastar::future<kafka::offset_commit_response>, std::__1::default_delete<seastar::future<kafka::offset_commit_response> > > > > seastar::futurize<seastar::future<std::__1::tuple<std::exception_ptr, std::__1::unique_ptr<seastar::future<kafka::offset_commit_response>, std::__1::default_delete<seastar::future<kafka::offset_commit_response> > > > > >::invoke<seastar::sharded<kafka::group_manager>::invoke_on<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}, , seastar::future<std::__1::tuple<std::exception_ptr, std::__1::unique_ptr<seastar::future<kafka::offset_commit_response>, std::__1::default_delete<seastar::future<kafka::offset_commit_response> > > > > >(unsigned int, seastar::smp_submit_to_options, kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&)::{lambda()#1}&>(kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&) at /vectorized/include/seastar/core/future.hh:1999
 (inlined by) seastar::smp_message_queue::async_work_item<seastar::sharded<kafka::group_manager>::invoke_on<kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}, , seastar::future<std::__1::tuple<std::exception_ptr, std::__1::unique_ptr<seastar::future<kafka::offset_commit_response>, std::__1::default_delete<seastar::future<kafka::offset_commit_response> > > > > >(unsigned int, seastar::smp_submit_to_options, kafka::group_router::route_stages<kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&)>(kafka::offset_commit_request, kafka::group::stages<kafka::offset_commit_response> (kafka::group_manager::*)(kafka::offset_commit_request&&))::{lambda()#1}::operator()()::{lambda(kafka::group_manager&)#1}&&)::{lambda()#1}>::run_and_dispose() at /vectorized/include/seastar/core/smp.hh:240
seastar::reactor::run_tasks(seastar::reactor::task_queue&) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:2660
 (inlined by) seastar::reactor::run_some_tasks() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3123
seastar::reactor::do_run() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3307
operator() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:4522
 (inlined by) decltype ((std::declval<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>())()) std::__1::__invoke[abi:v160004]<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:394
 (inlined by) void std::__1::__invoke_void_return_wrapper<void, true>::__call<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:487
 (inlined by) std::__1::__function::__alloc_func<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2, std::__1::allocator<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2>, void ()>::operator()[abi:v160004]() at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:185
 (inlined by) std::__1::__function::__func<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2, std::__1::allocator<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2>, void ()>::operator()() at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:356
std::__1::__function::__value_func<void ()>::operator()[abi:v160004]() const at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:510
 (inlined by) std::__1::function<void ()>::operator()() const at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:1156
 (inlined by) seastar::posix_thread::start_routine(void*) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/posix.cc:73
StephanDollberg commented 1 year ago

Switching to fragmented_vector or chunked_fifo not a 100% replacement as we use vector::erase here which neither currently support.

Maybe just use std::deque.

StephanDollberg commented 1 year ago

One further problem is that each partition replica has a segment_appender so with large partition counts we don't just want to prealloc 8k for all of them.

Hence, std::deque is out of the window (4k chunks by default in libc++). So will probably need to use fragmented_vector with an erase workaround.

There is further the question of whether there is actually a different bug that causes there to be 100k+ pending flush_ops.

StephanDollberg commented 1 year ago

I looked at the memory sampler dumps from the crash (i.e.: this is a fragmentation OOM).

There is a few interesting allocation sites though which all come from the replicate_batcher to segment_appender path. Most importantly all of them have kafka::group::handle_offset_commit in there path so this is likely related to an offset commits storm or something?