redpanda-data / redpanda

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

Oversized allocation: 1994752 bytes #17778

Closed wzzzrd86 closed 4 months ago

wzzzrd86 commented 5 months ago

From topic scale testing.

From Customer running environment when testing 20K topics

Redpanda version;

23.3.11

Apr 10 11:57:14 redpanda-17 rpk[62990]: WARN 2024-04-10 11:57:14,592 [shard 8:main] seastar_memory - oversized allocation: 1994752 bytes. This is non-fatal, but could lead to latency and/or fragmentation issues. Please report: at 0x93848d3 0x8febe3c 0x8ff9110 0x780dbf8 0x45cb610 0x4322bf6 0x438ddd4 0x436d182 0x2cf2f5a 0x90b933f 0x90bcab1 0x91174e5 0x90467af /opt/redpanda/lib/libc.so.6+0x91016 /opt/redpanda/lib/libc.so.6+0x1166cf

Backtrace

Backtrace:
[Backtrace #0]
void seastar::backtrace<seastar::current_backtrace_tasklocal()::$_0>(seastar::current_backtrace_tasklocal()::$_0&&) at /v/build/v_deps_build/seastar-prefix/src/seastar/include/seastar/util/backtrace.hh:64
 (inlined by) seastar::current_backtrace_tasklocal() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/util/backtrace.cc:98
 (inlined by) seastar::current_tasktrace() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/util/backtrace.cc:149
 (inlined by) seastar::current_backtrace() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/util/backtrace.cc:182
seastar::memory::cpu_pages::warn_large_allocation(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:839
 (inlined by) seastar::memory::cpu_pages::check_large_allocation(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:904
 (inlined by) seastar::memory::cpu_pages::allocate_large(unsigned int, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:911
 (inlined by) seastar::memory::allocate_large(unsigned long, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1521
 (inlined by) seastar::memory::allocate_slowpath(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1645
seastar::memory::allocate(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1658
 (inlined by) operator new(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:2355
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<kafka::describe_configs_resource>::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<kafka::describe_configs_resource>>::pointer> std::__1::__allocate_at_least[abi:v160004]<std::__1::allocator<kafka::describe_configs_resource>>(std::__1::allocator<kafka::describe_configs_resource>&, unsigned long) at /vectorized/llvm/bin/../include/c++/v1/__memory/allocate_at_least.h:55
 (inlined by) std::__1::__split_buffer<kafka::describe_configs_resource, std::__1::allocator<kafka::describe_configs_resource>&>::__split_buffer(unsigned long, unsigned long, std::__1::allocator<kafka::describe_configs_resource>&) at /vectorized/llvm/bin/../include/c++/v1/__split_buffer:323
 (inlined by) std::__1::vector<kafka::describe_configs_resource, std::__1::allocator<kafka::describe_configs_resource>>::reserve(unsigned long) at /vectorized/llvm/bin/../include/c++/v1/vector:1503
 (inlined by) std::__1::vector<kafka::describe_configs_resource> kafka::protocol::decoder::do_read_array<std::__1::vector, kafka::describe_configs_request_data::decode_standard(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>)::$_0, kafka::describe_configs_resource>(int, kafka::describe_configs_request_data::decode_standard(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>)::$_0&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/protocol/wire.h:314
 (inlined by) std::__1::vector<kafka::describe_configs_resource> kafka::protocol::decoder::read_array<std::__1::vector, kafka::describe_configs_request_data::decode_standard(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>)::$_0, kafka::describe_configs_resource>(kafka::describe_configs_request_data::decode_standard(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>)::$_0&&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/protocol/wire.h:204
 (inlined by) kafka::describe_configs_request_data::decode_standard(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/vbuild/release/clang/src/v/kafka/protocol/schemata/protocol.cc:2760
kafka::describe_configs_request::decode(kafka::protocol::decoder&, detail::base_named_type<short, kafka::kafka_api_version, std::__1::integral_constant<bool, true>>) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/protocol/describe_configs.h:36
 (inlined by) kafka::handler_template<kafka::describe_configs_api, (short)0, (short)4, 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) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/server/handlers/describe_configs.cc:107
kafka::handler_base<false>::handle(kafka::request_context&&, seastar::smp_service_group) const at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/server/handlers/handler_interface.cc:88
kafka::process_generic(kafka::handler_interface const*, kafka::request_context&&, seastar::smp_service_group, kafka::session_resources const&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/server/requests.cc:112
 (inlined by) kafka::process_request(kafka::request_context&&, seastar::smp_service_group, kafka::session_resources const&) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/server/requests.cc:339
kafka::connection_context::dispatch_method_once(kafka::request_header, unsigned long) (.resume) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-0c83b6d177030809f-1/redpanda/redpanda/src/v/kafka/server/connection_context.cc:597
std::__1::coroutine_handle<seastar::internal::coroutine_traits_base<void>::promise_type>::resume[abi:v160004]() const at /vectorized/llvm/bin/../include/c++/v1/__coroutine/coroutine_handle.h:169
 (inlined by) seastar::internal::coroutine_traits_base<void>::promise_type::run_and_dispose() at /vectorized/include/seastar/core/coroutine.hh:125
seastar::reactor::run_tasks(seastar::reactor::task_queue&) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:2750
 (inlined by) seastar::reactor::run_some_tasks() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3213
seastar::reactor::do_run() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3397
seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2::operator()() const at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:4624
 (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:90
LLVMSymbolizer: error reading file: No such file or directory
?? at ??:0
?? at ??:0

JIRA Link: CORE-2318

michael-redpanda commented 4 months ago

describe_configs_request_data contains a vector of describe_configs_resource. Probalby need to do something similar to what was done in #17756