Open dotnwat opened 7 months ago
Reactor stall decoding
tl;dr ZSTD_initDCtx_internal
probably touching a bunch of memory for some zstd scratch space and stalling out in page faults. kernel be faultin
{<kernel>} 0xffffffff9cb03977: handle_pte_fault+0x217
{<kernel>} 0xffffffff9cb05905: __handle_mm_fault+0x405
{<kernel>} 0xffffffff9cb05cc8: handle_mm_fault+0xd8
[nwatkins@fedora pstg]$ ~/src/seastar/scripts/seastar-addr2line -v -e amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda --kallsyms kallsysms/ubuntu-vm-05.pod-22.poc.local/proc/kallsyms
amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda: ELF 64-bit LSB pie executable, x86-64, version 1 (SYSV), dynamically linked, interpreter /opt/redpanda/lib/ld.so, for GNU/Linux 3.2.0, BuildID[sha1]=a373d6880ff95d17856b3a690f5a235aa77b9229, stripped
Reactor stalled for 33 ms on shard 9. Backtrace: 0x90460bf 0x904781d 0x42abf 0x857ea5a 0x84cd061 0x84cdd86 0x84cc913 0x2d08fd3 0x906a8ef 0x906e061 0x90c8a95 0x8ff76ff 0x91016 0x1166cf
kernel callstack: 0xffffffffffffff80 0xffffffff9c99194c 0xffffffff9c991bdd 0xffffffff9c8a3569 0xffffffff9c8a371e 0xffffffff9cb16530 0xffffffff9cafdcd9 0xffffffff9cb01dce 0xffffffff9cb03977 0xffffffff9cb05905 0xffffffff9cb05cc8 0xffffffff9c89f919 0xffffffff9d5bd877 0xffffffff9d600bb7
Reactor stalled for 33 ms on shard 9. Backtrace:
[Backtrace #0]
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x90460bf: void seastar::backtrace<seastar::backtrace_buffer::append_backtrace_oneline()::{lambda(seastar::frame)#1}>(seastar::backtrace_buffer::append_backtrace_oneline()::{lambda(seastar::frame)#1}&&) at /v/build/v_deps_build/seastar-prefix/src/seastar/include/seastar/util/backtrace.hh:64
(inlined by) seastar::backtrace_buffer::append_backtrace_oneline() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:838
(inlined by) seastar::print_with_backtrace(seastar::backtrace_buffer&, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:857
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x904781d: seastar::internal::cpu_stall_detector::generate_trace() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:1539
(inlined by) seastar::internal::cpu_stall_detector::maybe_report() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:1376
(inlined by) seastar::internal::cpu_stall_detector::on_signal() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:1396
(inlined by) seastar::reactor::block_notifier(int) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:1597
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x42abf: ?? ??:0
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x857ea5a: ZSTD_initDCtx_internal at /v/build/v_deps_build/zstd-prefix/src/zstd/lib/decompress/zstd_decompress.c:282
(inlined by) ZSTD_initStaticDCtx at /v/build/v_deps_build/zstd-prefix/src/zstd/lib/decompress/zstd_decompress.c:285
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x84cd061: static_dctx at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/compression/async_stream_zstd.cc:120
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x84cdd86: async_stream_zstd at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/compression/async_stream_zstd.cc:150
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x84cc913: std::__1::__unique_if<compression::async_stream_zstd>::__unique_single std::__1::make_unique[abi:v160004]<compression::async_stream_zstd, unsigned long&, int const&>(unsigned long&, int const&) at /vectorized/llvm/bin/../include/c++/v1/__memory/unique_ptr.h:686
(inlined by) compression::initialize_async_stream_zstd(unsigned long) at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/compression/async_stream_zstd.cc:60
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x2d08fd3: operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/redpanda/application.cc:526
(inlined by) seastar::future<void> seastar::futurize<void>::invoke<application::initialize(std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<scheduling_groups>)::$_6&>(application::initialize(std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<scheduling_groups>)::$_6&) at /vectorized/include/seastar/core/future.hh:2003
(inlined by) seastar::smp_message_queue::async_work_item<application::initialize(std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<YAML::Node>, std::__1::optional<scheduling_groups>)::$_6>::run_and_dispose() at /vectorized/include/seastar/core/smp.hh:263
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x906a8ef: 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x906e061: seastar::reactor::do_run() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3397
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x90c8a95: operator() 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x8ff76ff: 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x91016: ?? ??:0
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x1166cf: ?? ??:0
kernel callstack:
[Backtrace #1]
{<kernel>} 0xffffffffffffff80: 0xffffffffffffff80 (1061210960 bytes after last symbol)
{<kernel>} 0xffffffff9c99194c: smp_call_function_many_cond+0x13c
{<kernel>} 0xffffffff9c991bdd: on_each_cpu_cond_mask+0x1d
{<kernel>} 0xffffffff9c8a3569: native_flush_tlb_multi+0x59
{<kernel>} 0xffffffff9c8a371e: flush_tlb_mm_range+0xee
{<kernel>} 0xffffffff9cb16530: ptep_clear_flush+0x60
{<kernel>} 0xffffffff9cafdcd9: wp_page_copy+0x269
{<kernel>} 0xffffffff9cb01dce: do_wp_page+0xce
{<kernel>} 0xffffffff9cb03977: handle_pte_fault+0x217
{<kernel>} 0xffffffff9cb05905: __handle_mm_fault+0x405
{<kernel>} 0xffffffff9cb05cc8: handle_mm_fault+0xd8
{<kernel>} 0xffffffff9c89f919: do_user_addr_fault+0x1c9
{<kernel>} 0xffffffff9d5bd877: exc_page_fault+0x77
{<kernel>} 0xffffffff9d600bb7: asm_exc_page_fault+0x27
Looks like faulting in chunk cache allocations
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x8f9db07: seastar::memory::cpu_pages::free_span_no_merge(unsigned int, unsigned int) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:709
(inlined by) seastar::memory::cpu_pages::allocate_large_and_trim(unsigned int, bool) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:816
(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:912
(inlined by) seastar::memory::small_pool::add_more_objects() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1454
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x8fa1ccb: seastar::memory::small_pool::allocate() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1423
(inlined by) void* seastar::memory::allocate_from_small_pool<(seastar::memory::alignment_t)0>(unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1595
(inlined by) seastar::memory::allocate_aligned(unsigned long, unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:1687
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x8f92fde: seastar::internal::allocate_aligned_buffer_impl(unsigned long, unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:2254
(inlined by) seastar::internal::allocate_aligned_buffer_impl(unsigned long, unsigned long) at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/memory.cc:142
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x2ff25c1: std::__1::unique_ptr<char [], seastar::free_deleter> seastar::allocate_aligned_buffer<char>(unsigned long, unsigned long) at /vectorized/include/seastar/core/aligned_buffer.hh:45
(inlined by) segment_appender_chunk at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/storage/segment_appender_chunk.h:34
(inlined by) lw_shared_ptr_no_esft<const unsigned long &, const detail::base_named_type<unsigned long, storage::alignment_type, std::__1::integral_constant<bool, true> > &> at /vectorized/include/seastar/core/shared_ptr.hh:178
(inlined by) seastar::lw_shared_ptr<storage::segment_appender_chunk> seastar::lw_shared_ptr<storage::segment_appender_chunk>::make<unsigned long const&, detail::base_named_type<unsigned long, storage::alignment_type, std::__1::integral_constant<bool, true> > const&>(unsigned long const&, detail::base_named_type<unsigned long, storage::alignment_type, std::__1::integral_constant<bool, true> > const&) at /vectorized/include/seastar/core/shared_ptr.hh:282
(inlined by) seastar::lw_shared_ptr<storage::segment_appender_chunk> seastar::make_lw_shared<storage::segment_appender_chunk, unsigned long const&, detail::base_named_type<unsigned long, storage::alignment_type, std::__1::integral_constant<bool, true> > const&>(unsigned long const&, detail::base_named_type<unsigned long, storage::alignment_type, std::__1::integral_constant<bool, true> > const&) at /vectorized/include/seastar/core/shared_ptr.hh:443
(inlined by) operator() at /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-059c3321c337b010b-1/redpanda/redpanda/src/v/storage/chunk_cache.h:51
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x2ff2dec: seastar::future<void> seastar::futurize<void>::invoke<storage::internal::chunk_cache::start()::{lambda(unsigned long)#1}&, unsigned long const&>(storage::internal::chunk_cache::start()::{lambda(unsigned long)#1}&, unsigned long const&) at /vectorized/include/seastar/core/future.hh:2003
(inlined by) auto seastar::futurize_invoke<storage::internal::chunk_cache::start()::{lambda(unsigned long)#1}&, unsigned long const&>(storage::internal::chunk_cache::start()::{lambda(unsigned long)#1}&, unsigned long const&) at /vectorized/include/seastar/core/future.hh:2037
(inlined by) seastar::internal::do_for_each_state<boost::iterators::counting_iterator<unsigned long, boost::use_default, boost::use_default>, storage::internal::chunk_cache::start()::{lambda(unsigned long)#1}>::run_and_dispose() at /vectorized/include/seastar/core/loop.hh:403
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x906a8ef: 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x906e061: seastar::reactor::do_run() at /v/build/v_deps_build/seastar-prefix/src/seastar/src/core/reactor.cc:3397
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x90c8a95: operator() 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x8ff76ff: 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
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x91016: ?? ??:0
{amd64/23.3.7-1-amd64/opt/redpanda/libexec/redpanda} 0x1166cf: ?? ??:0
kernel callstack:
[Backtrace #1]
{<kernel>} 0xffffffffffffff80: 0xffffffffffffff80 (1061210960 bytes after last symbol)
{<kernel>} 0xffffffff9c99194c: smp_call_function_many_cond+0x13c
{<kernel>} 0xffffffff9c991bdd: on_each_cpu_cond_mask+0x1d
{<kernel>} 0xffffffff9c8a3569: native_flush_tlb_multi+0x59
{<kernel>} 0xffffffff9c8a371e: flush_tlb_mm_range+0xee
{<kernel>} 0xffffffff9cb16530: ptep_clear_flush+0x60
{<kernel>} 0xffffffff9cafdcd9: wp_page_copy+0x269
{<kernel>} 0xffffffff9cb01dce: do_wp_page+0xce
{<kernel>} 0xffffffff9cb03977: handle_pte_fault+0x217
{<kernel>} 0xffffffff9cb05905: __handle_mm_fault+0x405
{<kernel>} 0xffffffff9cb05cc8: handle_mm_fault+0xd8
{<kernel>} 0xffffffff9c89f919: do_user_addr_fault+0x1c9
{<kernel>} 0xffffffff9d5bd877: exc_page_fault+0x77
{<kernel>} 0xffffffff9d600bb7: asm_exc_page_fault+0x27
Checked around 10 other random samples and they all seemed to be chunk cache allocations
Yes, I've seen something like this before. The chunk cache gets implicated because it's one of the main place we touch a substantial amount of memory at startup of a fresh cluster. We could just allocate the chunks in a more lazy fashion (but still allocate the memory) I suppose.
How do you know it's 200 faults? 50ms is extremely slow for a fault, these should often be around 1 to 10 microseconds maybe. Is it because there were 200 reactor stalls?
I think this occurs here either because something about the VM makes the machinery of handling allocation (e.g., page table manipulation way slower) or because we aren't getting huge pages. Normally we get 2MB huge pages which dramatically reduces the fault rate since we fault every 2MB not every 4K. There are counters for that should we access this system again.
How do you know it's 200 faults? 50ms is extremely slow for a fault, these should often be around 1 to 10 microseconds maybe. Is it because there were 200 reactor stalls?
Sorry, being sloppy in my writing. Right, not 200 faults. I meant, all the reactor stalls reported were around 50ms. My understanding is that not all reactor stalls are printed--that seastar throttles their printing--so that would be roughly 200 50ms reactor stalls. I don't think there is enough information there to know if that is what's going on.
One thing to note is that the initial chunk cache hydration happens in a seastar::do_for_each
loop so I'm a little surprised that it would be causing the reactor stall since after each chunk allocation there is a yield opportunity for the reactor. Perhaps it lots of faults as a result of memory starting to fragment and ranges splitting? bit mysterious
Notes from investigating a 20-30 second start-up delay in a virtual machine environment using fast, but non-local storage.
Summary
Open question:
Start-up traces
13 second delay, 15 in a second examined trace.
Some interesting observations here
11 second delay, 9 in a second examined trace.
JIRA Link: CORE-1890