Closed travisdowns closed 1 year ago
Update: it does seem like something weird is going on with shard 9 for hours prior to the crash. Consider the live objects (heap allocated objects) graph prior to the crash:
The crash occurs at 4:23 near the far right of the chart. Shard 9 (the highest line) is accumulating objects at a rapid clip and there was an earlier similar case earlier: this first drop was when I stopped one test (so the clients disconnect) and started another one.
It seems like we are leaking/accumulating objects at least while clients remain connected.
Immediately before the crash the reads from disk skyrockets, consistent with the idea that we are doing a lot of reads from disk. Maybe the parser_errc::end_of_stream
lo line is just reflecting that there are a lot of reads and we get to the end of many segments
At the same time the batch cache hit rate plunges to 7% from 100%. So maybe memory pressure from the accumulation of objects has become so acute at this point that we don't hit in the batch cache anymore and that causes the heavy read activity, but the actual problem is the memory leak/accumulation.
Metrics covering the range of the crash with the relevant instance selected:
After the node came back up, it crashed again 2.5 hours later, with the symptoms being the same.
A different node also crashed, in between the two crashes of node-12, with the same symptoms, though the backtrace was different, as shown below:
2023-03-17T05:51:03.383364961Z stderr F Backtrace:
[Backtrace #0]
{23.1.1-arm64/libexec/redpanda} 0x506864b: void seastar::backtrace<seastar::backtrace_buffer::append_backtrace()::'lambda'(seastar::frame)>(seastar::backtrace_buffer::append_backtrace()::'lambda'(seastar::frame)&&) at /v/build/v_deps_build/seastar-prefix/src/seastar/include/seastar/util/backtrace.hh:59
(inlined by) seastar::backtrace_buffer::append_backtrace() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:754
(inlined by) seastar::print_with_backtrace(seastar::backtrace_buffer&, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:784
{23.1.1-arm64/libexec/redpanda} 0x50bbecb: seastar::print_with_backtrace(char const*, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:796
(inlined by) seastar::sigabrt_action() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3663
(inlined by) void seastar::install_oneshot_signal_handler<6, (void (*)())(&(seastar::sigabrt_action()))>()::'lambda'(int, siginfo_t*, void*)::operator()(int, siginfo_t*, void*) const at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3639
(inlined by) void seastar::install_oneshot_signal_handler<6, (void (*)())(&(seastar::sigabrt_action()))>()::'lambda'(int, siginfo_t*, void*)::__invoke(int, siginfo_t*, void*) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3635
LLVMSymbolizer: error reading file: No such file or directory
{linux-vdso.so.1} 0x85f: ?? at ??:0
LLVMSymbolizer: error reading file: No such file or directory
{/opt/redpanda/lib/libc.so.6} 0x86067: ?? at ??:0
{/opt/redpanda/lib/libc.so.6} 0x3e87f: ?? at ??:0
{/opt/redpanda/lib/libc.so.6} 0x2aef7: ?? at ??:0
{23.1.1-arm64/libexec/redpanda} 0x4fe4d27: seastar::memory::on_allocation_failure(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1821
(inlined by) seastar::memory::allocate(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1410
{23.1.1-arm64/libexec/redpanda} 0x4ff169f: __libc_malloc at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1860
{23.1.1-arm64/libexec/redpanda} 0x22d9573: seastar::temporary_buffer<char>::temporary_buffer(unsigned long) at /vectorized/include/seastar/core/temporary_buffer.hh:73
(inlined by) iobuf::create_new_fragment(unsigned long) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/bytes/iobuf.h:235
(inlined by) iobuf::append(char const*, unsigned long) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/bytes/iobuf.h:289
(inlined by) iobuf::append(seastar::temporary_buffer<char>) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/bytes/iobuf.h:307
(inlined by) iobuf::append(iobuf)::'lambda'(details::io_fragment*)::operator()(details::io_fragment*) const at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/bytes/iobuf.h:326
(inlined by) void boost::intrusive::list_impl<boost::intrusive::mhtraits<details::io_fragment, boost::intrusive::list_member_hook<boost::intrusive::link_mode<(boost::intrusive::link_mode_type)1> >, &(details::io_fragment::hook)>, unsigned long, false, void>::pop_front_and_dispose<iobuf::append(iobuf)::'lambda'(details::io_fragment*)>(iobuf::append(iobuf)::'lambda'(details::io_fragment*)) at /vectorized/include/boost/intrusive/list.hpp:357
{23.1.1-arm64/libexec/redpanda} 0x343c0bf: iobuf::append(iobuf) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/bytes/iobuf.h:325
(inlined by) kafka::response_writer::write(std::__1::optional<iobuf>&&) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/protocol/response_writer.h:184
{23.1.1-arm64/libexec/redpanda} 0x32f0303: kafka::response_writer::write(std::__1::optional<kafka::batch_reader>&) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/protocol/response_writer.h:209
(inlined by) kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192::operator()(kafka::fetchable_topic_response&, kafka::response_writer&) const::'lambda'(kafka::fetchable_partition_response&, kafka::response_writer&)::operator()(kafka::fetchable_partition_response&, kafka::response_writer&) const at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/vbuild/release/clang/src/v/kafka/protocol/schemata/protocol.cc:8033
(inlined by) unsigned int kafka::response_writer::write_array<std::__1::vector<kafka::fetchable_partition_response, std::__1::allocator<kafka::fetchable_partition_response> >, kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192::operator()(kafka::fetchable_topic_response&, kafka::response_writer&) const::'lambda'(kafka::fetchable_partition_response&, kafka::response_writer&)>(std::__1::vector<kafka::fetchable_partition_response, std::__1::allocator<kafka::fetchable_partition_response> >&, kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192::operator()(kafka::fetchable_topic_response&, kafka::response_writer&) const::'lambda'(kafka::fetchable_partition_response&, kafka::response_writer&)&&) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/protocol/response_writer.h:272
(inlined by) kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192::operator()(kafka::fetchable_topic_response&, kafka::response_writer&) const at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/vbuild/release/clang/src/v/kafka/protocol/schemata/protocol.cc:8008
(inlined by) unsigned int kafka::response_writer::write_array<std::__1::vector<kafka::fetchable_topic_response, std::__1::allocator<kafka::fetchable_topic_response> >, kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192>(std::__1::vector<kafka::fetchable_topic_response, std::__1::allocator<kafka::fetchable_topic_response> >&, kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >)::$_192&&) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/protocol/response_writer.h:272
(inlined by) kafka::fetch_response_data::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/vbuild/release/clang/src/v/kafka/protocol/schemata/protocol.cc:8005
{23.1.1-arm64/libexec/redpanda} 0x2a5522b: kafka::fetch_response::encode(kafka::response_writer&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true> >) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/protocol/fetch.h:185
(inlined by) seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > kafka::request_context::respond<kafka::fetch_response>(kafka::fetch_response) at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/server/request_context.h:198
(inlined by) kafka::op_context::send_response() && at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/server/handlers/fetch.cc:730
{23.1.1-arm64/libexec/redpanda} 0x2a5f833: kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()::operator()() const at /var/lib/buildkite-agent/builds/buildkite-arm64-builders-i-0ce0a1f3f61d30bef-1/redpanda/redpanda/src/v/kafka/server/handlers/fetch.cc:573
(inlined by) decltype(static_cast<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&>(fp)()) std::__1::__invoke<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&>(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&) at /vectorized/llvm/bin/../include/c++/v1/type_traits:3640
(inlined by) std::__1::invoke_result<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&>::type std::__1::invoke<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&>(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:93
(inlined by) auto seastar::internal::future_invoke<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::internal::monostate>(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::internal::monostate&&) at /vectorized/include/seastar/core/future.hh:1223
(inlined by) seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > seastar::future<void>::then_impl_nrvo<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'(), seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > >(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&&)::'lambda'(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&)::operator()(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&) const::'lambda'()::operator()() const at /vectorized/include/seastar/core/future.hh:1596
(inlined by) void seastar::futurize<seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > >::satisfy_with_result_of<seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > seastar::future<void>::then_impl_nrvo<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'(), seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > >(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&&)::'lambda'(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&)::operator()(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&) const::'lambda'()>(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&&) at /vectorized/include/seastar/core/future.hh:2134
(inlined by) seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > seastar::future<void>::then_impl_nrvo<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'(), seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > >(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&&)::'lambda'(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&)::operator()(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&) const at /vectorized/include/seastar/core/future.hh:1589
(inlined by) seastar::continuation<seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'(), seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > seastar::future<void>::then_impl_nrvo<kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'(), seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > > >(kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&&)::'lambda'(seastar::internal::promise_base_with_type<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >&&, kafka::handler_template<kafka::fetch_api, (short)4, (short)11, seastar::future<seastar::foreign_ptr<std::__1::unique_ptr<kafka::response, std::__1::default_delete<kafka::response> > > >, &(kafka::default_estimate_adaptor(unsigned long, kafka::connection_context&))>::handle(kafka::request_context, seastar::smp_service_group)::$_4::operator()(std::__1::unique_ptr<kafka::op_context, std::__1::default_delete<kafka::op_context> >&) const::'lambda0'()&, seastar::future_state<seastar::internal::monostate>&&), void>::run_and_dispose() at /vectorized/include/seastar/core/future.hh:781
{23.1.1-arm64/libexec/redpanda} 0x5081d83: seastar::reactor::run_tasks(seastar::reactor::task_queue&) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:2330
(inlined by) seastar::reactor::run_some_tasks() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:2737
{23.1.1-arm64/libexec/redpanda} 0x5084beb: seastar::reactor::do_run() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:2906
{23.1.1-arm64/libexec/redpanda} 0x50bd1ef: seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95::operator()() const at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:4140
(inlined by) decltype(static_cast<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95&>(fp)()) std::__1::__invoke<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95&) at /vectorized/llvm/bin/../include/c++/v1/type_traits:3640
(inlined by) void std::__1::__invoke_void_return_wrapper<void, true>::__call<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95&) at /vectorized/llvm/bin/../include/c++/v1/__functional/invoke.h:61
(inlined by) std::__1::__function::__alloc_func<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95, std::__1::allocator<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95>, void ()>::operator()() at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:180
(inlined by) std::__1::__function::__func<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95, std::__1::allocator<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_95>, void ()>::operator()() at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:354
{23.1.1-arm64/libexec/redpanda} 0x502fbdb: std::__1::__function::__value_func<void ()>::operator()() const at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:507
(inlined by) std::__1::function<void ()>::operator()() const at /vectorized/llvm/bin/../include/c++/v1/__functional/function.h:1184
(inlined by) seastar::posix_thread::start_routine(void*) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/posix.cc:73
I don't think that means anything: it's just going to be "some" 128K allocation that fails and in this case it's a buffer we are preparing to send back to the client.
Hi All, i have reached to the similar error on my redpanda clsuter.
i have attached the full log file. osp-redpanda-0_osp-redpanda.log
The problem is that the replicate batch flusher call fall behind on background flushes if produce load is high enough and the shard is saturated CPU-wise.
The flushes of the batches are actually happening (since every flush flushes the entire replicate batcher) but the background tasks (most of which will flush nothing) enqueued in replicate_batcher::cache_and_wait_for_result
accumulate without bound, eventually causing the shard to run out of memory. Above, the orange shard is consistently at 100% CPU and has a linear increase of waiting flush tasks. Some other shards also seem to be increasing but at a much slower rate and/or occasionally dropping in allocated objects, indicating a period where flushing task completion was outpacing creation.
Here's the memory sampling dump:
INFO 2023-04-10 20:07:16,939 [shard 11] main - application.cc:398 - Memory dump 124:
Heap dump 33257/33848613 sampled allocs (1 in 1017.7891271010615), 4478528/4799160320 sampled bytes (1 in 1071.593237778127)
Sites non-zero/total: 158/158
bytes: 777984, count: 3039, hash: 10246010162384358531, Backtrace: 0x52f766b 0x50690b7 0x236c133 0x236b3bf 0x2368ae3 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 622592, count: 19, hash: 654376531747597443, Backtrace: 0x52f766b 0x504d6e7 0x5069d8f 0x2281ff3 0x2368d03 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 491520, count: 30, hash: 15973778145999529982, Backtrace: 0x52f766b 0x505bbc3 0x504aa63 0x1dfce9f 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 240960, count: 3012, hash: 12810156384384336621, Backtrace: 0x52f766b 0x50690b7 0x2368ddf 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 230960, count: 2887, hash: 12263796589439293404, Backtrace: 0x52f766b 0x50690b7 0x22822cb 0x2368d03 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 200512, count: 3133, hash: 5311949658652777565, Backtrace: 0x52f766b 0x50690b7 0x236984f 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 196992, count: 3078, hash: 6416106912508247201, Backtrace: 0x52f766b 0x50690b7 0x2369623 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 195200, count: 3050, hash: 16430223063636387105, Backtrace: 0x52f766b 0x50690b7 0x23693a3 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 193088, count: 3017, hash: 13770575252087356513, Backtrace: 0x52f766b 0x50690b7 0x23694e3 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 192896, count: 3014, hash: 4076512258210121633, Backtrace: 0x52f766b 0x50690b7 0x2369123 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 188544, count: 2946, hash: 9289804037616933873, Backtrace: 0x52f766b 0x50690b7 0x2368f73 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 183424, count: 2866, hash: 16356732700079447777, Backtrace: 0x52f766b 0x50690b7 0x2369263 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ad1b37 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 98304, count: 3, hash: 18323502822051799186, Backtrace: 0x52f766b 0x504d6e7 0x5069d8f 0x2281ff3 0x2368d03 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ac8377 0x2ac24f3 0x2abf75f 0x283d167 0x282b97f 0x280f853 0x280f08b 0x280963f 0x282064b 0x1dbcf1f 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 90112, count: 11, hash: 15996598970894179544, Backtrace: 0x52f766b 0x50690b7 0x49095cf 0x49024d7 0x48df6e3 0x2d20ccf 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 57344, count: 7, hash: 4365568195673421876, Backtrace: 0x52f766b 0x50690b7 0x4908a43 0x49024c7 0x48df6e3 0x2d20ccf 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 49152, count: 6, hash: 5128355952192277476, Backtrace: 0x52f766b 0x50690b7 0x4904433 0x48fd1bf 0x48d38b7 0x48a45bf 0x48a582f 0x4812f37 0x222ef33 0x222f65b 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 32768, count: 1, hash: 4717383159171747977, Backtrace: 0x52f766b 0x504d6e7 0x5069d8f 0x490cd53 0x490ef63 0x47d893f 0x47d9427 0x21e668f 0x24106f7 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
bytes: 32768, count: 1, hash: 1166850261794791049, Backtrace: 0x52f766b 0x504d6e7 0x5069d8f 0x2281ff3 0x2368d03 0x23686e7 0x21e5d33 0x41ba0fb 0x41ffb8f 0x41ac8ff 0x41ac103 0x40d21b7 0x2b39023 0x2accdf7 0x2ac8377 0x2ac24f3 0x2abf75f 0x283d167 0x282b97f 0x280f853 0x280f08b 0x280963f 0x2805857 0x2821b63 0x1dbcf1f 0x50ea3bb 0x50ed223 0x512582b 0x5098213 /opt/redpanda/lib/libc.so.6+0x7d5c7 /opt/redpanda/lib/libc.so.6+0xe5d1b
All of the objects around the ~3000 count mark are associated with item
entries in replicate_batcher::_item_cache. The sampling rate is 0.1% so that means around 3 million objects of each type. Several of the other entries are also related, e.g., some of the big but low-count entries are associated with e.g., the backing array (circular_fifo
actually) for the lock object that the tasks queue on.
When the load stops, the tasks are able to drain (load comes off at 20:07:30):
@surikbs - I don't think it's the same issue based on the size of the objects in your memory diagnostics.
Can you open a new issue and post the full disagnostic output that occurs when the OOM occurs as text, not as a screenshot?
Hi @travisdowns i have created a new issue and attached the logs. https://github.com/redpanda-data/redpanda/issues/9995
Version & Environment
Redpanda version: 23.1.1
What went wrong?
During a load test, a redpanda node crashed with an out of memory error. The memory diagnostics indicate that the system was truly out of memory (it was not fragmentation) and failed while allocating a 128K buffer for logfile readahead. Most of the memory is used by small objects of 64-256 bytes.
Prior to the crash, shard 9 seems to go haywire, repeatedly issuing the following log lines (> 10,000 times) in rapid succession:
Here's the memory dump:
Here's the decoded backtrace at the crash (which occurs on shard 9 in the same code that issues the log message above):
It looks to be like there may be an infinite loop or other problem which prevents forward progress and eventually enough memory accumulates (e.g., via background jobs that cannot drain fast enough) that the OOM ensues.
What should have happened instead?
Not crash.
How to reproduce the issue?
Not sure. It occurrred under a high load test with both producers and consumers active.
Additional information
Full log attached. 0.log