Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BugFix] abort compaction task properly during shutdown #55503

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 42 additions & 0 deletions be/src/storage/lake/compaction_scheduler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -165,11 +165,20 @@ void CompactionScheduler::stop() {
bool expected = false;
if (_stopped.compare_exchange_strong(expected, true)) {
_threads->shutdown();
abort_all();
}
}

void CompactionScheduler::compact(::google::protobuf::RpcController* controller, const CompactRequest* request,
CompactResponse* response, ::google::protobuf::Closure* done) {
if (_stopped) {
brpc::ClosureGuard guard(done);
auto st = Status::Aborted("Compaction shutdown in progress!");
LOG(WARNING) << "Fail to compact num_of_tablets= " << request->tablet_ids().size()
<< ". version=" << request->version() << " txn_id=" << request->txn_id() << " : " << st;
st.to_protobuf(response->mutable_status());
return;
}
// when FE request a compaction, CN may not have any key cached yet, so pass an encryption_meta to refresh cache
if (!request->encryption_meta().empty()) {
Status st = KeyCache::instance().refresh_keys(request->encryption_meta());
Expand Down Expand Up @@ -197,6 +206,9 @@ void CompactionScheduler::compact(::google::protobuf::RpcController* controller,
}
// initialize last check time, compact request is received right after FE sends it, so consider it valid now
cb->set_last_check_time(time(nullptr));
// FIXME: be noticed that even the `_stopped` is checked in the beginning, it is still possible that the scheduler
// turns to stop status after the check and before this point. In case of the conflict happens, the contexts_vec
// may never be processed which leads to rpc hanging without response.
_task_queues.put_by_txn_id(request->txn_id(), contexts_vec);
// DO NOT touch `contexts_vec` from here!
TEST_SYNC_POINT("CompactionScheduler::compact:return");
Expand Down Expand Up @@ -414,6 +426,21 @@ Status CompactionScheduler::do_compaction(std::unique_ptr<CompactionTaskContext>
return status;
}

void CompactionScheduler::abort_compaction(std::unique_ptr<CompactionTaskContext> context) {
const auto start_time = ::time(nullptr);
const auto tablet_id = context->tablet_id;
const auto txn_id = context->txn_id;
const auto version = context->version;

int64_t in_queue_time_sec = start_time > context->enqueue_time_sec ? (start_time - context->enqueue_time_sec) : 0;
context->stats->in_queue_time_sec += in_queue_time_sec;
context->status = Status::Aborted("Aborted due to shutdown!");
LOG(WARNING) << "Fail to compact tablet " << tablet_id << ". version=" << version << " txn_id=" << txn_id << " : "
<< context->status;
// make sure every task can be finished no matter it is succeeded or failed.
context->callback->finish_task(std::move(context));
}

Status CompactionScheduler::abort(int64_t txn_id) {
std::unique_lock l(_contexts_lock);
for (butil::LinkNode<CompactionTaskContext>* node = _contexts.head(); node != _contexts.end();
Expand All @@ -432,6 +459,21 @@ Status CompactionScheduler::abort(int64_t txn_id) {
return Status::NotFound(fmt::format("no compaction task with txn id {}", txn_id));
}

void CompactionScheduler::abort_all() {
for (int i = 0; i < _task_queues.task_queue_size(); ++i) {
// drain _task_queues, ensure every tasks in queue are properly aborted
bool done = false;
while (!done) {
CompactionContextPtr context;
if (_task_queues.try_get(i, &context)) {
abort_compaction(std::move(context));
} else {
done = true;
}
}
}
}

// If `lake_compaction_max_concurrency` is reduced during runtime, `id` may exceed it.
// Reschedule all the tasks in _task_queues where idx ranges from [new_val, old_val-1].
// return true means current thread id is beyond target size, current thread shoud exist.
Expand Down
5 changes: 5 additions & 0 deletions be/src/storage/lake/compaction_scheduler.h
Original file line number Diff line number Diff line change
Expand Up @@ -227,12 +227,17 @@ class CompactionScheduler {
private:
friend class CompactionTaskCallback;

// abort all the compaction tasks in the task queue. Only expected to be invoked during stop()
void abort_all();

void remove_states(const std::vector<std::unique_ptr<CompactionTaskContext>>& contexes);

void thread_task(int id);

Status do_compaction(std::unique_ptr<CompactionTaskContext> context);

void abort_compaction(std::unique_ptr<CompactionTaskContext> context);

bool reschedule_task_if_needed(int id);

TabletManager* _tablet_mgr;
Expand Down
103 changes: 102 additions & 1 deletion be/test/storage/lake/compaction_scheduler_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,107 @@ TEST_F(LakeCompactionSchedulerTest, test_list_tasks) {
bthread_join(tid, nullptr);
}

TEST_F(LakeCompactionSchedulerTest, test_abort_all) {
// set to single thread mode, so all the tasks will be in the same thread
_compaction_scheduler.update_compact_threads(1);
std::vector<CompactionTaskInfo> tasks;
_compaction_scheduler.list_tasks(&tasks);
EXPECT_EQ(0, tasks.size());

int num_tasks = 16;
auto l0 = std::make_shared<CountDownLatch>(1);
auto l1 = std::make_shared<CountDownLatch>(num_tasks);
auto l2 = std::make_shared<CountDownLatch>(1);
auto l3 = std::make_shared<CountDownLatch>(num_tasks);
EXPECT_EQ(num_tasks, l1->count());

std::vector<bthread_t> tids;
// preserve requests and responses life time
std::vector<std::shared_ptr<CompactRequest>> requests;
std::vector<std::shared_ptr<CompactResponse>> responses;
{ // task 0: block the execution done until l2.count_down()
auto txn_id = next_id();
auto request = std::make_shared<CompactRequest>();
requests.push_back(request);
auto response = std::make_shared<CompactResponse>();
responses.push_back(response);
auto meta = generate_simple_tablet_metadata(DUP_KEYS);
CHECK_OK(_tablet_mgr->put_tablet_metadata(meta));
request->add_tablet_ids(meta->id());
request->set_timeout_ms(60 * 1000); // 60 seconds
request->set_txn_id(txn_id);
request->set_version(1);
// wait l2, count down l0
ASSIGN_OR_ABORT(auto tid, bthreads::start_bthread([&, l1, l2, request, response]() {
auto cb = ::google::protobuf::NewCallback(notify_and_wait_latch, l0, l2);
_compaction_scheduler.compact(nullptr, request.get(), response.get(), cb);
}));
tids.push_back(tid);
}
// Wait for task0 complete
l0->wait();
// repeatedly submit num_tasks into the queue, make the thread busy before stop() invoked.
for (int i = 0; i < num_tasks; ++i) {
auto txn_id = next_id();
auto request = std::make_shared<CompactRequest>();
requests.push_back(request);
auto response = std::make_shared<CompactResponse>();
responses.push_back(response);
auto meta = generate_simple_tablet_metadata(DUP_KEYS);
CHECK_OK(_tablet_mgr->put_tablet_metadata(meta));
request->add_tablet_ids(meta->id());
request->set_timeout_ms(60 * 1000); // 60 seconds
request->set_txn_id(txn_id);
request->set_version(1);
// wait l2, count down l1
ASSIGN_OR_ABORT(auto tid, bthreads::start_bthread([&, l1, l2, l3, request, response]() {
auto cb = ::google::protobuf::NewCallback(notify_and_wait_latch, l1, l2);
l3->count_down();
_compaction_scheduler.compact(nullptr, request.get(), response.get(), cb);
}));
tids.push_back(tid);
}
// wait until all bthreads run
l3->wait();
// Allow all tasks to be executed
// because the first task is blocked by the l2 countdown, rest are all in task queue.
l2->count_down();
// expect remain tasks in task queue aborted during stop
_compaction_scheduler.stop();
// l1 should be properly count down by all the tasks
l1->wait();

for (auto tid : tids) {
bthread_join(tid, nullptr);
}

int aborted = 0;
for (auto response : responses) {
if (response->status().status_code() == TStatusCode::ABORTED) {
++aborted;
}
}
// total num_tasks + 1 compact tasks submitted.
// expect the first one success, and then the remain `num_tasks` aborted between [1, num_tasks]
EXPECT_GE(aborted, 1);
EXPECT_LE(aborted, num_tasks);
}

TEST_F(LakeCompactionSchedulerTest, test_submit_compact_after_stop) {
_compaction_scheduler.stop();
auto l1 = std::make_shared<CountDownLatch>(1);
CompactRequest request;
CompactResponse response;
request.add_tablet_ids(_tablet_metadata->id());
request.set_timeout_ms(60 * 1000);
request.set_txn_id(next_id());
request.set_version(1);
auto cb = ::google::protobuf::NewCallback(notify, l1);
_compaction_scheduler.compact(nullptr, &request, &response, cb);
l1->wait();
EXPECT_EQ(response.status().status_code(), TStatusCode::ABORTED);
}

TEST_F(LakeCompactionSchedulerTest, test_compaction_cancel) {
CompactRequest request;
CompactResponse response;
Expand Down Expand Up @@ -160,4 +261,4 @@ TEST_F(LakeCompactionSchedulerTest, test_issue44136) {
latch->wait();
}

} // namespace starrocks::lake
} // namespace starrocks::lake
Loading