Skip to content

Conversation

pgellert
Copy link
Owner

Controller frontend/backend integration of the quota store.

savex and others added 30 commits May 14, 2024 18:43
     This adds FileHandlers to boto3 loggers using
     the same filenames and levels that is used in ducktape

     Also, adds the logger name manually to easily identify
     the source of the message next to the module name
This will be used to uniquely identify new segments.
- remove CRTP from stm_raft_fixture to improve readability
- in raft_fixture::retry_with_leader assume unknown error types are not
to be retried
- in raft_fixture::retry_with_leader assume unknown errors are not
retryable
- in raft_fixture::retry_with_leader hold the function with do_with
- in raft_fixture expose base_directory and stop_and_recreate_nodes()
This is similar to change introduced in
8c3777b but this time for
kafka::offset.

Quoting from the previous commit message:

    model::offset::max() is often used to indicate "no upper bound" on
    operations. E.g. for tiered storage uploads[^1], for reading from local
    storage[^2], etc.

    We also do often convert from closed to opened offset intervals
    representations. E.g. committed offset to LSO and the other way around.

    When combined, these can result in unexpected behaviors. In particular,
    if on a read path the max offset is specified as model::offset_max() but
    at lower level this is converted into an exclusive offset by calling
    next_offset(model::offset::max()), the result is model::offset::min()
    aka -2^63.

    This is dangerous. Let's instead saturate the offset similar to how we
    saturate prev_offset.

    We also have a few cases where we just do `o + model::offset(1)`. These
    should be refactored to use next_offset too.

    This isn't fixing any existing known bug. Discovered this while trying
    to rewrite some logic related to tiered storage uploads.

    [^1]: https://github.com/redpanda-data/redpanda/blob/79bf7eed6e04da1d0987b5abd719c4b289dde761/src/v/archival/ntp_archiver_service.cc#L1656
    [^2]: https://github.com/redpanda-data/redpanda/blob/79bf7eed6e04da1d0987b5abd719c4b289dde761/src/v/cluster/migrations/tx_manager_migrator.cc#L219
Adds the url addressing style to the `s3_imposter` fixture to enable
future testing of the remote object with both `virtual_host` and `path`
styles.
The command don't need arguments.
Cloud storage tests will now use member variables from the base
`s3_imposter` class for bucket names, as well as ensuring
expectations are set correctly per the URL style.
Migrating the existing remote `FIXTURE_TEST`s to GTest allows for
easily parameterizing with the `url_style`. Now, both `virtual_host`
and `path` url styles will be tested in `remote_test.cc`.

Additionally, a new macro is added to `test_macros.h`, due to the fact
that GTest's `FAIL()` macro can only be called in a function returning
`void`. `RPTEST_ADD_FAIL()` will instead call into `ADD_FAILURE()` in
a GTest environment.
When parameterizing ducktape tests, it makes sense to join the cloud
storage type with the url style to ensure compatability- if the two
options were to be parameterized separately in a @matrix decorator,
we might run, for example, an ABS test with path style addressing.

`get_cloud_storage_type_and_url_style()` will generate the proper
list of compatible cloud storage types and url styles.
Parameterizes the following ducktape tests with url_style:
 * ArchivalTest.test_write
 * TestReadReplicaService.test_simple_end_to_end
 * TieredStorageTest.test_tiered_storage
If neither of state machines managed by the state machine manager can
accept the foreground apply there is no need to read the range of
batches from the log. In this case we can skip the foreground apply read
and move on directly to background apply dispatch.

Signed-off-by: Michał Maślanka <[email protected]>
If an apply results in error the stm manager should yield to prevent
busy looping and consuming a lot of CPU cycles. Fixed yielding in
background apply fiber and added a yield if foreground apply didn't make
progress.

Signed-off-by: Michał Maślanka <[email protected]>
In state machine manager the background apply fiber is there to apply
batches to state machines which are behind the main apply fiber. The
background apply fiber is only active if an stm is behind what is
currently being applied by the main fiber. A bug in reader configuration
lead to a situation in which a background apply could read up to `_next`
offset of the state machine manager leading to the situation in which a
stm `last_applied_offset` is equal to `_next` therefore the stm next
offset is actually equal to `state_machine_manager::_next + 1`. This
lead to a situation in which there may be no stms suitable to be
handled in main apply fiber but also they are not recoverable by
background apply fiber as their `_next` offset is greater of the one
from the state machine manager. This causes a stall in the
`state_machine_manager` and state machines stop making progress.

Signed-off-by: Michał Maślanka <[email protected]>
…chine-manager

Fixed `state_machine_manager` stall
…ode-mutability-default

schema_registry: Make mode_mutability: true by default
Was not using the correct instance of node_config when determining
whether to put the OpenSSL context service into FIPS mode or not.

Signed-off-by: Michael Boquard <[email protected]>
@pgellert pgellert force-pushed the quotas/quota-store-stm branch from 33ac836 to 1e8602e Compare May 31, 2024 09:23
@pgellert pgellert force-pushed the quotas/quota-store-stm branch from 1e8602e to f0e5e83 Compare May 31, 2024 11:21
@pgellert pgellert closed this May 31, 2024
pgellert pushed a commit that referenced this pull request Oct 2, 2024
=================================================================
==524==ERROR: AddressSanitizer: container-overflow on address 0x52100028a100 at pc 0x7f29c0bf51fd bp 0x7ffda75397d0 sp 0x7ffda75397c8
READ of size 4 at 0x52100028a100 thread T0
    #0 0x7f29c0bf51fc in util::mem_tracker::pretty_print_ascii() const /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tracking_allocator.cc:75:31
    #1 0x55be1ac7eb7b in mem_tracker_pretty_printing::test_method() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:82:5
    #2 0x55be1ac7d651 in mem_tracker_pretty_printing_invoker() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:50:1

Signed-off-by: Noah Watkins <[email protected]>
pgellert pushed a commit that referenced this pull request Oct 29, 2024
=================================================================
==524==ERROR: AddressSanitizer: container-overflow on address 0x52100028a100 at pc 0x7f29c0bf51fd bp 0x7ffda75397d0 sp 0x7ffda75397c8
READ of size 4 at 0x52100028a100 thread T0
    #0 0x7f29c0bf51fc in util::mem_tracker::pretty_print_ascii() const /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tracking_allocator.cc:75:31
    #1 0x55be1ac7eb7b in mem_tracker_pretty_printing::test_method() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:82:5
    #2 0x55be1ac7d651 in mem_tracker_pretty_printing_invoker() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:50:1

Signed-off-by: Noah Watkins <[email protected]>
(cherry picked from commit 71c352c)
pgellert pushed a commit that referenced this pull request Nov 27, 2024
Previously I consistently saw the following UB crash:

    #0 0x798cfa38a419 in serde::parquet::(anonymous namespace)::record_shredder::process_group_node(serde::parquet::schema_element const*, serde::parquet::(anonymous namespace)::traversal_levels, std::__1::variant<serde::parquet::null_value, serde::parquet::boolean_value, serde::parquet::int32_value, serde::parquet::int64_value, serde::parquet::float32_value, serde::parquet::float64_value, serde::parquet::byte_array_value, serde::parquet::fixed_byte_array_value, fragmented_vector<serde::parquet::group_member, 18446744073709551615ul>, fragmented_vector<serde::parquet::repeated_element, 18446744073709551615ul> >) /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:145:9
    #1 0x798cfa38a419 in serde::parquet::(anonymous namespace)::record_shredder::shred() (.resume) /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:60:26
    #2 0x798d04e84f26 in std::__1::coroutine_handle<seastar::internal::coroutine_traits_base<void>::promise_type>::resume[abi:ne180100]() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__coroutine/coroutine_handle.h:143:5
    #3 0x798d04e84f26 in seastar::internal::coroutine_traits_base<void>::promise_type::run_and_dispose() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/rp_deps_install/include/seastar/core/coroutine.hh:125:20
    #4 0x798cd42e82c6 in seastar::reactor::run_tasks(seastar::reactor::task_queue&) /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:2800:14
    redpanda-data#5 0x798cd42ee05e in seastar::reactor::run_some_tasks() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:3262:9
    redpanda-data#6 0x798cd42f04d2 in seastar::reactor::do_run() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:3445:9
    redpanda-data#7 0x798cd4353ff3 in seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2::operator()() const /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:4702:22
    redpanda-data#8 0x798cd4353ff3 in decltype(std::declval<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>()()) std::__1::__invoke[abi:ne180100]<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__type_traits/invoke.h:344:25
    redpanda-data#9 0x798cd4353ff3 in void std::__1::__invoke_void_return_wrapper<void, true>::__call[abi:ne180100]<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__type_traits/invoke.h:419:5
    redpanda-data#10 0x798cd4353ff3 in 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:ne180100]() /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:169:12
    redpanda-data#11 0x798cd4353ff3 in 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()() /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:311:10
    redpanda-data#12 0x798cd41fb2eb in std::__1::__function::__value_func<void ()>::operator()[abi:ne180100]() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:428:12
    redpanda-data#13 0x798cd41fb2eb in std::__1::function<void ()>::operator()() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:981:10
    redpanda-data#14 0x798cd41fb2eb in seastar::posix_thread::start_routine(void*) /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/posix.cc:90:5
    redpanda-data#15 0x600bf2dbed78 in asan_thread_start(void*) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/src/compiler-rt/lib/asan/asan_interceptors.cpp:239:28
    redpanda-data#16 0x798cd2094ac2  (/home/andrew/xfs/vbuild/redpanda/dev/debug/clang/dist/local/redpanda/lib/libc.so.6+0x94ac2) (BuildId: 490fef8403240c91833978d494d39e537409b92e)
    redpanda-data#17 0x798cd212684f  (/home/andrew/xfs/vbuild/redpanda/dev/debug/clang/dist/local/redpanda/lib/libc.so.6+0x12684f) (BuildId: 490fef8403240c91833978d494d39e537409b92e)

SUMMARY: UndefinedBehaviorSanitizer: undefined-behavior /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:136:18
pgellert added a commit that referenced this pull request Dec 16, 2024
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Dec 16, 2024
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 10, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 10, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 10, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 10, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 13, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 14, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 15, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 15, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert added a commit that referenced this pull request Jan 15, 2025
```
ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded:
Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4
```
pgellert pushed a commit that referenced this pull request Jan 16, 2025
=================================================================
==524==ERROR: AddressSanitizer: container-overflow on address 0x52100028a100 at pc 0x7f29c0bf51fd bp 0x7ffda75397d0 sp 0x7ffda75397c8
READ of size 4 at 0x52100028a100 thread T0
    #0 0x7f29c0bf51fc in util::mem_tracker::pretty_print_ascii() const /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tracking_allocator.cc:75:31
    #1 0x55be1ac7eb7b in mem_tracker_pretty_printing::test_method() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:82:5
    #2 0x55be1ac7d651 in mem_tracker_pretty_printing_invoker() /var/lib/buildkite-agent/builds/buildkite-amd64-builders-i-024f8676eb7b139e9-1/redpanda/vtools/src/v/utils/tests/tracking_allocator_tests.cc:50:1

Signed-off-by: Noah Watkins <[email protected]>
(cherry picked from commit 71c352c)
pgellert pushed a commit that referenced this pull request Mar 10, 2025
Previously I consistently saw the following UB crash:

    #0 0x798cfa38a419 in serde::parquet::(anonymous namespace)::record_shredder::process_group_node(serde::parquet::schema_element const*, serde::parquet::(anonymous namespace)::traversal_levels, std::__1::variant<serde::parquet::null_value, serde::parquet::boolean_value, serde::parquet::int32_value, serde::parquet::int64_value, serde::parquet::float32_value, serde::parquet::float64_value, serde::parquet::byte_array_value, serde::parquet::fixed_byte_array_value, fragmented_vector<serde::parquet::group_member, 18446744073709551615ul>, fragmented_vector<serde::parquet::repeated_element, 18446744073709551615ul> >) /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:145:9
    #1 0x798cfa38a419 in serde::parquet::(anonymous namespace)::record_shredder::shred() (.resume) /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:60:26
    #2 0x798d04e84f26 in std::__1::coroutine_handle<seastar::internal::coroutine_traits_base<void>::promise_type>::resume[abi:ne180100]() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__coroutine/coroutine_handle.h:143:5
    #3 0x798d04e84f26 in seastar::internal::coroutine_traits_base<void>::promise_type::run_and_dispose() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/rp_deps_install/include/seastar/core/coroutine.hh:125:20
    #4 0x798cd42e82c6 in seastar::reactor::run_tasks(seastar::reactor::task_queue&) /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:2800:14
    redpanda-data#5 0x798cd42ee05e in seastar::reactor::run_some_tasks() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:3262:9
    redpanda-data#6 0x798cd42f04d2 in seastar::reactor::do_run() /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:3445:9
    redpanda-data#7 0x798cd4353ff3 in seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2::operator()() const /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/reactor.cc:4702:22
    redpanda-data#8 0x798cd4353ff3 in decltype(std::declval<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>()()) std::__1::__invoke[abi:ne180100]<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__type_traits/invoke.h:344:25
    redpanda-data#9 0x798cd4353ff3 in void std::__1::__invoke_void_return_wrapper<void, true>::__call[abi:ne180100]<seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&>(seastar::smp::configure(seastar::smp_options const&, seastar::reactor_options const&)::$_2&) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__type_traits/invoke.h:419:5
    redpanda-data#10 0x798cd4353ff3 in 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:ne180100]() /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:169:12
    redpanda-data#11 0x798cd4353ff3 in 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()() /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:311:10
    redpanda-data#12 0x798cd41fb2eb in std::__1::__function::__value_func<void ()>::operator()[abi:ne180100]() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:428:12
    redpanda-data#13 0x798cd41fb2eb in std::__1::function<void ()>::operator()() const /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/install/bin/../include/c++/v1/__functional/function.h:981:10
    redpanda-data#14 0x798cd41fb2eb in seastar::posix_thread::start_routine(void*) /home/andrew/xfs/vbuild/redpanda/dev/debug/clang/v_deps_build/seastar-prefix/src/seastar-build/../seastar/src/core/posix.cc:90:5
    redpanda-data#15 0x600bf2dbed78 in asan_thread_start(void*) /home/andrew/xfs/vbuild/llvm/3b5b5c1ec4a3095ab096dd780e84d7ab81f3d7ff/src/compiler-rt/lib/asan/asan_interceptors.cpp:239:28
    redpanda-data#16 0x798cd2094ac2  (/home/andrew/xfs/vbuild/redpanda/dev/debug/clang/dist/local/redpanda/lib/libc.so.6+0x94ac2) (BuildId: 490fef8403240c91833978d494d39e537409b92e)
    redpanda-data#17 0x798cd212684f  (/home/andrew/xfs/vbuild/redpanda/dev/debug/clang/dist/local/redpanda/lib/libc.so.6+0x12684f) (BuildId: 490fef8403240c91833978d494d39e537409b92e)

SUMMARY: UndefinedBehaviorSanitizer: undefined-behavior /home/awong/Repos/redpanda/src/v/serde/parquet/shredder.cc:136:18
(cherry picked from commit 7560dc9)
pgellert pushed a commit that referenced this pull request Jul 3, 2025
…imestamp

`storage`: add `self_compact_timestamp` related details to `segment` [SC&R #2]
pgellert pushed a commit that referenced this pull request Aug 1, 2025
In a previous PR[1] we began to rely on the archiver loop to retry, and
moved away from relying on `cloud_io::remote` for retries in two ways:
1. setting an explicit `disallow` retry policy on the retry node passed
   to the remote, and
2. setting the `max_retries` passed to `remote::upload_segment()` to 1.

In practice, we saw that _not_ relying on the remote resulted in an
uptick in the `vectorized_cloud_storage_failed_uploads` metric, which is
monitored and alerted on. In [2] we reverted #1, but didn't notice #2.
This commit reverts #2.

[1] redpanda-data#25951
[2] redpanda-data#26969
pgellert pushed a commit that referenced this pull request Aug 5, 2025
In a previous PR[1] we began to rely on the archiver loop to retry, and
moved away from relying on `cloud_io::remote` for retries in two ways:
1. setting an explicit `disallow` retry policy on the retry node passed
   to the remote, and
2. setting the `max_retries` passed to `remote::upload_segment()` to 1.

In practice, we saw that _not_ relying on the remote resulted in an
uptick in the `vectorized_cloud_storage_failed_uploads` metric, which is
monitored and alerted on. In [2] we reverted #1, but didn't notice #2.
This commit reverts #2.

[1] redpanda-data#25951
[2] redpanda-data#26969

(cherry picked from commit 7f409da)
pgellert pushed a commit that referenced this pull request Aug 8, 2025
This reorganizes cloud_topics::app so that:
1. it is a single non-sharded entity,
2. it includes all cloud_topics-related sharded services

Previously, when cloud_topics::app was sharded, #2 was difficult because
there are some cloud_topics-related services that need to be sharded,
but app itself was sharded in application.cc; having a sharded service
own another sharded service seems like a bad idea.

In doing this, I needed to wrap the data_plane_api in another class in
order to shard it, which I've done with the recently added
state_accessors. This lets us continue to use the data_plane_api instead
of the data_plane_impl, as we were before, as a sharded service.

Using the state_accessors also lets us plumb the data plane API through
to cluster::partition and to the partition_proxy without introducing a
circular dependency. Previously we plumbed down cloud_topics::app, which
made it easy to trip over dependencies.
pgellert pushed a commit that referenced this pull request Sep 19, 2025
`ct/l1`: add `committing_policy` & `compaction_committer` [CT - Compaction #2]
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.