Skip to content

Commit

Permalink
wasm: fix CPU profiler hang
Browse files Browse the repository at this point in the history
Currently trying to take a backtrace while the CPU profiler is running
results in a deadlock, because taking a backtrace segfaults, as we don't
write any debug symbols during JIT compilation (which causes other
deadlocks in libgcc). To fix this, disable the profile's backtracing
when within Wasm. In the future we should use Wasmtime's profiling APIs
to get a stacktrace within the guest program that is running. For more
information on that API see:
https://docs.wasmtime.dev/api/wasmtime/struct.GuestProfiler.html

Signed-off-by: Tyler Rockwood <rockwood@redpanda.com>
  • Loading branch information
rockwotj committed Apr 16, 2024
1 parent 75a1f2a commit d6aa1e2
Show file tree
Hide file tree
Showing 2 changed files with 27 additions and 1 deletion.
19 changes: 18 additions & 1 deletion src/v/wasm/tests/wasm_transform_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,10 +11,10 @@

#include "bytes/bytes.h"
#include "pandaproxy/schema_registry/types.h"
#include "serde/rw/rw.h"
#include "wasm/errc.h"
#include "wasm/tests/wasm_fixture.h"

#include <seastar/core/internal/cpu_profiler.hh>
#include <seastar/core/reactor.hh>

#include <absl/strings/str_cat.h>
Expand Down Expand Up @@ -134,3 +134,20 @@ TEST_F(WasmTestFixture, LogsAreEmitted) {
expected.append(reinterpret_cast<const char*>(bytes.data()), bytes.size());
EXPECT_THAT(log_lines(), ElementsAre(expected));
}

TEST_F(WasmTestFixture, WorksWithCpuProfiler) {
bool original_enabled = ss::engine().get_cpu_profiler_enabled();
std::chrono::nanoseconds original_period
= ss::engine().get_cpu_profiler_period();
ss::engine().set_cpu_profiler_enabled(true);
ss::engine().set_cpu_profiler_period(10ns);
load_wasm("dynamic.wasm");
EXPECT_THROW(execute_command("loop", 0), wasm::wasm_exception);
ss::engine().set_cpu_profiler_enabled(original_enabled);
ss::engine().set_cpu_profiler_period(original_period);
std::vector<ss::cpu_profiler_trace> traces;
ss::engine().profiler_results(traces);
for (const auto& t : traces) {
std::cout << t.user_backtrace << "\n";
}
}
9 changes: 9 additions & 0 deletions src/v/wasm/wasmtime.cc
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@

#include <seastar/core/align.hh>
#include <seastar/core/future.hh>
#include <seastar/core/internal/cpu_profiler.hh>
#include <seastar/core/posix.hh>
#include <seastar/core/sharded.hh>
#include <seastar/core/shared_ptr.hh>
Expand Down Expand Up @@ -599,7 +600,13 @@ class wasmtime_engine : public engine {
// Poll the call future to completion, yielding to the scheduler when
// the future yields.
auto start = ss::steady_clock_type::now();
// Disable profiling backtraces inside the VM - at the time of writing
// backtraces lead to segfaults causing deadlock in Seastar's signal
// handlers.
auto _ = ss::internal::scoped_disable_profile_temporarily();
while (!wasmtime_call_future_poll(fut.get())) {
// Re-enable stacktraces before we yield control to the scheduler.
ss::internal::profiler_drop_stacktraces(false);
auto end = ss::steady_clock_type::now();
_probe.increment_cpu_time(end - start);
if (_pending_host_function) {
Expand All @@ -609,6 +616,8 @@ class wasmtime_engine : public engine {
co_await ss::coroutine::maybe_yield();
}
start = ss::steady_clock_type::now();
// Disable stacktraces as we enter back into Wasmtime
ss::internal::profiler_drop_stacktraces(true);
}
auto end = ss::steady_clock_type::now();
_probe.increment_cpu_time(end - start);
Expand Down

0 comments on commit d6aa1e2

Please sign in to comment.